Source code for airflow.providers.apache.hive.operators.hive
## 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.importosimportrefromtypingimportTYPE_CHECKING,Any,Dict,Optional,Sequencefromairflow.configurationimportconffromairflow.modelsimportBaseOperatorfromairflow.providers.apache.hive.hooks.hiveimportHiveCliHookfromairflow.utilsimportoperator_helpersfromairflow.utils.operator_helpersimportcontext_to_airflow_varsifTYPE_CHECKING:fromairflow.utils.contextimportContext
[docs]classHiveOperator(BaseOperator):""" Executes hql code or hive script in a specific Hive database. :param hql: the hql to be executed. Note that you may also use a relative path from the dag file of a (template) hive script. (templated) :param hive_cli_conn_id: Reference to the :ref:`Hive CLI connection id <howto/connection:hive_cli>`. (templated) :param hiveconfs: if defined, these key value pairs will be passed to hive as ``-hiveconf "key"="value"`` :param hiveconf_jinja_translate: when True, hiveconf-type templating ${var} gets translated into jinja-type templating {{ var }} and ${hiveconf:var} gets translated into jinja-type templating {{ var }}. Note that you may want to use this along with the ``DAG(user_defined_macros=myargs)`` parameter. View the DAG object documentation for more details. :param script_begin_tag: If defined, the operator will get rid of the part of the script before the first occurrence of `script_begin_tag` :param run_as_owner: Run HQL code as a DAG's owner. :param mapred_queue: queue used by the Hadoop CapacityScheduler. (templated) :param mapred_queue_priority: priority within CapacityScheduler queue. Possible settings include: VERY_HIGH, HIGH, NORMAL, LOW, VERY_LOW :param mapred_job_name: This name will appear in the jobtracker. This can make monitoring easier. """
def__init__(self,*,hql:str,hive_cli_conn_id:str='hive_cli_default',schema:str='default',hiveconfs:Optional[Dict[Any,Any]]=None,hiveconf_jinja_translate:bool=False,script_begin_tag:Optional[str]=None,run_as_owner:bool=False,mapred_queue:Optional[str]=None,mapred_queue_priority:Optional[str]=None,mapred_job_name:Optional[str]=None,**kwargs:Any,)->None:super().__init__(**kwargs)self.hql=hqlself.hive_cli_conn_id=hive_cli_conn_idself.schema=schemaself.hiveconfs=hiveconfsor{}self.hiveconf_jinja_translate=hiveconf_jinja_translateself.script_begin_tag=script_begin_tagself.run_as=Noneifrun_as_owner:self.run_as=self.dag.ownerself.mapred_queue=mapred_queueself.mapred_queue_priority=mapred_queue_priorityself.mapred_job_name=mapred_job_namejob_name_template=conf.get('hive','mapred_job_name_template',fallback="Airflow HiveOperator task for {hostname}.{dag_id}.{task_id}.{execution_date}",)ifjob_name_templateisNone:raiseValueError("Job name template should be set !")self.mapred_job_name_template:str=job_name_template# assigned lazily - just for consistency we can create the attribute with a# `None` initial value, later it will be populated by the execute method.# This also makes `on_kill` implementation consistent since it assumes `self.hook`# is defined.self.hook:Optional[HiveCliHook]=None
[docs]defexecute(self,context:"Context")->None:self.log.info('Executing: %s',self.hql)self.hook=self.get_hook()# set the mapred_job_name if it's not set with dag, task, execution time infoifnotself.mapred_job_name:ti=context['ti']self.hook.mapred_job_name=self.mapred_job_name_template.format(dag_id=ti.dag_id,task_id=ti.task_id,execution_date=ti.execution_date.isoformat(),hostname=ti.hostname.split('.')[0],)ifself.hiveconf_jinja_translate:self.hiveconfs=context_to_airflow_vars(context)else:self.hiveconfs.update(context_to_airflow_vars(context))self.log.info('Passing HiveConf: %s',self.hiveconfs)self.hook.run_cli(hql=self.hql,schema=self.schema,hive_conf=self.hiveconfs)
[docs]defdry_run(self)->None:# Reset airflow environment variables to prevent# existing env vars from impacting behavior.self.clear_airflow_vars()self.hook=self.get_hook()self.hook.test_hql(hql=self.hql)