-
Notifications
You must be signed in to change notification settings - Fork 13.7k
/
hive.py
996 lines (864 loc) · 38.2 KB
/
hive.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
#
# 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 contextlib
import os
import re
import socket
import subprocess
import time
from collections import OrderedDict
from tempfile import NamedTemporaryFile, TemporaryDirectory
import unicodecsv as csv
from airflow.configuration import conf
from airflow.exceptions import AirflowException
from airflow.hooks.base_hook import BaseHook
from airflow.security import utils
from airflow.utils.helpers import as_flattened_list
from airflow.utils.operator_helpers import AIRFLOW_VAR_NAME_FORMAT_MAPPING
HIVE_QUEUE_PRIORITIES = ['VERY_HIGH', 'HIGH', 'NORMAL', 'LOW', 'VERY_LOW']
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()}
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
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)
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
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 d.items()])
)
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
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")
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,
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)
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)
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()
class HiveMetastoreHook(BaseHook):
""" Wrapper to interact with the Hive Metastore"""
# java short max val
MAX_PART_COUNT = 32767
def __init__(self, metastore_conn_id='metastore_default'):
self.conn_id = metastore_conn_id
self.metastore = self.get_metastore_client()
def __getstate__(self):
# This is for pickling to work despite the thirft hive client not
# being pickable
d = dict(self.__dict__)
del d['metastore']
return d
def __setstate__(self, d):
self.__dict__.update(d)
self.__dict__['metastore'] = self.get_metastore_client()
def get_metastore_client(self):
"""
Returns a Hive thrift client.
"""
import hmsclient
from thrift.transport import TSocket, TTransport
from thrift.protocol import TBinaryProtocol
ms = self._find_valid_server()
if ms is None:
raise AirflowException("Failed to locate the valid server.")
auth_mechanism = ms.extra_dejson.get('authMechanism', 'NOSASL')
if conf.get('core', 'security') == 'kerberos':
auth_mechanism = ms.extra_dejson.get('authMechanism', 'GSSAPI')
kerberos_service_name = ms.extra_dejson.get('kerberos_service_name', 'hive')
conn_socket = TSocket.TSocket(ms.host, ms.port)
if conf.get('core', 'security') == 'kerberos' \
and auth_mechanism == 'GSSAPI':
try:
import saslwrapper as sasl
except ImportError:
import sasl
def sasl_factory():
sasl_client = sasl.Client()
sasl_client.setAttr("host", ms.host)
sasl_client.setAttr("service", kerberos_service_name)
sasl_client.init()
return sasl_client
from thrift_sasl import TSaslClientTransport
transport = TSaslClientTransport(sasl_factory, "GSSAPI", conn_socket)
else:
transport = TTransport.TBufferedTransport(conn_socket)
protocol = TBinaryProtocol.TBinaryProtocol(transport)
return hmsclient.HMSClient(iprot=protocol)
def _find_valid_server(self):
conns = self.get_connections(self.conn_id)
for conn in conns:
host_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
self.log.info("Trying to connect to %s:%s", conn.host, conn.port)
if host_socket.connect_ex((conn.host, conn.port)) == 0:
self.log.info("Connected to %s:%s", conn.host, conn.port)
host_socket.close()
return conn
else:
self.log.info("Could not connect to %s:%s", conn.host, conn.port)
def get_conn(self):
return self.metastore
def check_for_partition(self, schema, table, partition):
"""
Checks whether a partition exists
:param schema: Name of hive schema (database) @table belongs to
:type schema: str
:param table: Name of hive table @partition belongs to
:type schema: str
:partition: Expression that matches the partitions to check for
(eg `a = 'b' AND c = 'd'`)
:type schema: str
:rtype: bool
>>> hh = HiveMetastoreHook()
>>> t = 'static_babynames_partitioned'
>>> hh.check_for_partition('airflow', t, "ds='2015-01-01'")
True
"""
with self.metastore as client:
partitions = client.get_partitions_by_filter(
schema, table, partition, 1)
if partitions:
return True
else:
return False
def check_for_named_partition(self, schema, table, partition_name):
"""
Checks whether a partition with a given name exists
:param schema: Name of hive schema (database) @table belongs to
:type schema: str
:param table: Name of hive table @partition belongs to
:type schema: str
:partition: Name of the partitions to check for (eg `a=b/c=d`)
:type schema: str
:rtype: bool
>>> hh = HiveMetastoreHook()
>>> t = 'static_babynames_partitioned'
>>> hh.check_for_named_partition('airflow', t, "ds=2015-01-01")
True
>>> hh.check_for_named_partition('airflow', t, "ds=xxx")
False
"""
with self.metastore as client:
return client.check_for_named_partition(schema, table, partition_name)
def get_table(self, table_name, db='default'):
"""Get a metastore table object
>>> hh = HiveMetastoreHook()
>>> t = hh.get_table(db='airflow', table_name='static_babynames')
>>> t.tableName
'static_babynames'
>>> [col.name for col in t.sd.cols]
['state', 'year', 'name', 'gender', 'num']
"""
if db == 'default' and '.' in table_name:
db, table_name = table_name.split('.')[:2]
with self.metastore as client:
return client.get_table(dbname=db, tbl_name=table_name)
def get_tables(self, db, pattern='*'):
"""
Get a metastore table object
"""
with self.metastore as client:
tables = client.get_tables(db_name=db, pattern=pattern)
return client.get_table_objects_by_name(db, tables)
def get_databases(self, pattern='*'):
"""
Get a metastore table object
"""
with self.metastore as client:
return client.get_databases(pattern)
def get_partitions(
self, schema, table_name, filter=None):
"""
Returns a list of all partitions in a table. Works only
for tables with less than 32767 (java short max val).
For subpartitioned table, the number might easily exceed this.
>>> hh = HiveMetastoreHook()
>>> t = 'static_babynames_partitioned'
>>> parts = hh.get_partitions(schema='airflow', table_name=t)
>>> len(parts)
1
>>> parts
[{'ds': '2015-01-01'}]
"""
with self.metastore as client:
table = client.get_table(dbname=schema, tbl_name=table_name)
if len(table.partitionKeys) == 0:
raise AirflowException("The table isn't partitioned")
else:
if filter:
parts = client.get_partitions_by_filter(
db_name=schema, tbl_name=table_name,
filter=filter, max_parts=HiveMetastoreHook.MAX_PART_COUNT)
else:
parts = client.get_partitions(
db_name=schema, tbl_name=table_name,
max_parts=HiveMetastoreHook.MAX_PART_COUNT)
pnames = [p.name for p in table.partitionKeys]
return [dict(zip(pnames, p.values)) for p in parts]
@staticmethod
def _get_max_partition_from_part_specs(part_specs, partition_key, filter_map):
"""
Helper method to get max partition of partitions with partition_key
from part specs. key:value pair in filter_map will be used to
filter out partitions.
:param part_specs: list of partition specs.
:type part_specs: list
:param partition_key: partition key name.
:type partition_key: str
:param filter_map: partition_key:partition_value map used for partition filtering,
e.g. {'key1': 'value1', 'key2': 'value2'}.
Only partitions matching all partition_key:partition_value
pairs will be considered as candidates of max partition.
:type filter_map: map
:return: Max partition or None if part_specs is empty.
"""
if not part_specs:
return None
# Assuming all specs have the same keys.
if partition_key not in part_specs[0].keys():
raise AirflowException("Provided partition_key {} "
"is not in part_specs.".format(partition_key))
if filter_map:
is_subset = set(filter_map.keys()).issubset(set(part_specs[0].keys()))
if filter_map and not is_subset:
raise AirflowException("Keys in provided filter_map {} "
"are not subset of part_spec keys: {}"
.format(', '.join(filter_map.keys()),
', '.join(part_specs[0].keys())))
candidates = [p_dict[partition_key] for p_dict in part_specs
if filter_map is None or
all(item in p_dict.items() for item in filter_map.items())]
if not candidates:
return None
else:
return max(candidates).encode('utf-8')
def max_partition(self, schema, table_name, field=None, filter_map=None):
"""
Returns the maximum value for all partitions with given field in a table.
If only one partition key exist in the table, the key will be used as field.
filter_map should be a partition_key:partition_value map and will be used to
filter out partitions.
:param schema: schema name.
:type schema: str
:param table_name: table name.
:type table_name: str
:param field: partition key to get max partition from.
:type field: str
:param filter_map: partition_key:partition_value map used for partition filtering.
:type filter_map: map
>>> hh = HiveMetastoreHook()
>>> filter_map = {'ds': '2015-01-01', 'ds': '2014-01-01'}
>>> t = 'static_babynames_partitioned'
>>> hh.max_partition(schema='airflow',\
... table_name=t, field='ds', filter_map=filter_map)
'2015-01-01'
"""
with self.metastore as client:
table = client.get_table(dbname=schema, tbl_name=table_name)
key_name_set = {key.name for key in table.partitionKeys}
if len(table.partitionKeys) == 1:
field = table.partitionKeys[0].name
elif not field:
raise AirflowException("Please specify the field you want the max "
"value for.")
elif field not in key_name_set:
raise AirflowException("Provided field is not a partition key.")
if filter_map and not set(filter_map.keys()).issubset(key_name_set):
raise AirflowException("Provided filter_map contains keys "
"that are not partition key.")
part_names = \
client.get_partition_names(schema,
table_name,
max_parts=HiveMetastoreHook.MAX_PART_COUNT)
part_specs = [client.partition_name_to_spec(part_name)
for part_name in part_names]
return HiveMetastoreHook._get_max_partition_from_part_specs(part_specs,
field,
filter_map)
def table_exists(self, table_name, db='default'):
"""
Check if table exists
>>> hh = HiveMetastoreHook()
>>> hh.table_exists(db='airflow', table_name='static_babynames')
True
>>> hh.table_exists(db='airflow', table_name='does_not_exist')
False
"""
try:
self.get_table(table_name, db)
return True
except Exception:
return False
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
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')
def _get_results(self, hql, schema='default', fetch_size=None, hive_conf=None):
from pyhive.exc import ProgrammingError
if isinstance(hql, str):
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('show') 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
yield from cur
except ProgrammingError:
self.log.debug("get_results returned no records")
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
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 file:
writer = csv.writer(file,
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)
def get_records(self, hql, schema='default', hive_conf=None):
"""
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, hive_conf=hive_conf)['data']
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