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.from__future__importannotationsimportosimportrefromcollections.abcimportSequencefromfunctoolsimportcached_propertyfromtypingimportTYPE_CHECKING,Anyfromairflow.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 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. :param hive_cli_params: parameters passed to hive CLO :param auth: optional authentication option passed for the Hive connection :param proxy_user: Run HQL code as this user. """
job_name_template=conf.get_mandatory_value("hive","mapred_job_name_template",fallback="Airflow HiveOperator task for {hostname}.{dag_id}.{task_id}.{logical_date}",)
[docs]defexecute(self,context:Context)->None:self.log.info("Executing: %s",self.hql)# set the mapred_job_name if it's not set with dag, task, execution time infoifnotself.mapred_job_name:ti=context["ti"]logical_date=context["logical_date"]iflogical_dateisNone:raiseRuntimeError("logical_date is None")hostname=ti.hostnameor""self.hook.mapred_job_name=self.mapred_job_name_template.format(dag_id=ti.dag_id,task_id=ti.task_id,logical_date=logical_date.isoformat(),hostname=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.test_hql(hql=self.hql)