kaxil commented on code in PR #62790: URL: https://github.com/apache/airflow/pull/62790#discussion_r2963359572
########## providers/ibm/mq/src/airflow/providers/ibm/mq/hooks/mq.py: ########## @@ -0,0 +1,358 @@ +# +# 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 annotations + +import asyncio +import json +import threading +from contextlib import contextmanager, suppress +from typing import Any + +from asgiref.sync import sync_to_async + +from airflow.sdk.bases.hook import BaseHook Review Comment: The hook imports `BaseHook` directly from `airflow.sdk.bases.hook`. Other providers in this repo import through the compat layer: ```python from airflow.providers.common.compat.sdk import BaseHook ``` This breaks Airflow 2.x compatibility. Also, `apache-airflow-providers-common-compat` is missing from the main `dependencies` list in `pyproject.toml` (only present in dev dependencies). ########## providers/ibm/mq/src/airflow/providers/ibm/mq/hooks/mq.py: ########## @@ -0,0 +1,358 @@ +# +# 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 annotations + +import asyncio +import json +import threading +from contextlib import contextmanager, suppress +from typing import Any + +from asgiref.sync import sync_to_async + +from airflow.sdk.bases.hook import BaseHook + +# Backoff parameters for transient consume failures +_BACKOFF_BASE: float = 1.0 +_BACKOFF_MAX: float = 60.0 +_BACKOFF_FACTOR: float = 2.0 + + +class IBMMQHook(BaseHook): + conn_name_attr = "conn_id" + default_conn_name = "mq_default" + conn_type = "mq" + hook_name = "IBM MQ" + default_open_options = "MQOO_INPUT_SHARED" + + def __init__( + self, + conn_id: str = default_conn_name, + queue_manager: str | None = None, + channel: str | None = None, + open_options: int | None = None, + ): + super().__init__() + self.conn_id = conn_id + self.queue_manager = queue_manager + self.channel = channel + self.open_options = open_options + self._conn = None + + @classmethod + def get_ui_field_behaviour(cls) -> dict[str, Any]: + """Return custom UI field behaviour for IBM MQ Connection.""" + return { + "hidden_fields": ["schema"], + "placeholders": { + "host": "mq.example.com", + "port": "1414", + "login": "app_user", + "extra": json.dumps( + { + "queue_manager": "QM1", + "channel": "DEV.APP.SVRCONN", + "open_options": cls.default_open_options, + }, + indent=2, + ), + }, + } + + @classmethod + def get_open_options_flags(cls, open_options: int) -> list[str]: + """ + Return the symbolic MQ open option flags set in a given bitmask. + + Each flag corresponds to a constant in 'ibmmq.CMQC' that starts with 'MQOO_'. + + :param open_options: The integer bitmask used when opening an MQ queue + (e.g., 'MQOO_INPUT_EXCLUSIVE | MQOO_FAIL_IF_QUIESCING'). + + :return: A list of the names of the MQ open flags that are set in the bitmask. + For example, '['MQOO_INPUT_EXCLUSIVE', 'MQOO_FAIL_IF_QUIESCING']'. + + Example: + >>> open_options = ibmmq.CMQC.MQOO_INPUT_SHARED | ibmmq.CMQC.MQOO_FAIL_IF_QUIESCING + >>> cls.get_open_options_flags(open_options) + ['MQOO_INPUT_SHARED', 'MQOO_FAIL_IF_QUIESCING'] + """ + import ibmmq + + return [ + name + for name, value in vars(ibmmq.CMQC).items() + if name.startswith("MQOO_") and (open_options & value) + ] + + @contextmanager + def get_conn(self): + """ + Sync context manager for IBM MQ connection lifecycle. + + Must be called from the executor thread (not the event loop thread). + Retrieves the Airflow connection, extracts MQ parameters, and manages + the IBM MQ connection lifecycle. + + :yield: IBM MQ connection object + """ + import ibmmq + + connection = BaseHook.get_connection(self.conn_id) + config = connection.extra_dejson + queue_manager = self.queue_manager or config.get("queue_manager") + channel = self.channel or config.get("channel") + Review Comment: `self.open_options` is mutated on the instance inside `get_conn()`. Since `get_conn()` can be called multiple times, and `getattr` falls back to `MQOO_INPUT_EXCLUSIVE` when the config string doesn't match a CMQC attribute, the first call permanently overwrites whatever was passed in `__init__`. Consider using a local variable instead of writing to `self.open_options`: ```python open_options = self.open_options or getattr( ibmmq.CMQC, config.get("open_options", self.default_open_options), ibmmq.CMQC.MQOO_INPUT_EXCLUSIVE, ) ``` Then use `open_options` (local) throughout the rest of the method. ########## providers/ibm/mq/src/airflow/providers/ibm/mq/queues/mq.py: ########## @@ -0,0 +1,93 @@ +# +# 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 annotations + +import re +from typing import Any +from urllib.parse import urlparse + +from airflow.providers.common.messaging.providers.base_provider import BaseMessageQueueProvider +from airflow.providers.ibm.mq.triggers.mq import AwaitMessageTrigger +from airflow.providers.ibm.mq.version_compat import AIRFLOW_V_3_0_PLUS + +if AIRFLOW_V_3_0_PLUS: Review Comment: The `BaseEventTrigger` import with the Airflow version check runs at module level, which means a version mismatch crashes at import time rather than at call time. The Kafka provider handles this differently — it puts the import in a `TYPE_CHECKING` block and uses `type: ignore` on the return type. Worth matching that pattern for consistency and resilience. ########## providers/ibm/mq/src/airflow/providers/ibm/mq/get_provider_info.py: ########## @@ -0,0 +1,45 @@ +# 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. + +# NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE OVERWRITTEN! +# +# IF YOU WANT TO MODIFY THIS FILE, YOU SHOULD MODIFY THE TEMPLATE +# `get_provider_info_TEMPLATE.py.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY + + +def get_provider_info(): + return { + "package-name": "apache-airflow-providers-ibm-mq", + "name": "IBM MQ", + "description": "`IBM MQ <https://www.ibm.com/products/mq/>`__\n", + "integrations": [ + { + "integration-name": "IBM MQ", + "external-doc-url": "https://www.ibm.com/products/mq", + "logo": "/docs/integration-logos/ibm-mq.png", + "tags": ["apache"], + } Review Comment: `"tags": ["apache"]` doesn't match `provider.yaml` which has `tags: [software]`. This file is auto-generated, but the mismatch suggests it was generated before the yaml was updated (or hand-edited). IBM MQ isn't an Apache project, so `"apache"` is wrong here. ########## providers/ibm/mq/pyproject.toml: ########## @@ -0,0 +1,124 @@ +# 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. + +# NOTE! THIS FILE IS AUTOMATICALLY GENERATED AND WILL BE OVERWRITTEN! + +# IF YOU WANT TO MODIFY THIS FILE EXCEPT DEPENDENCIES, YOU SHOULD MODIFY THE TEMPLATE +# `pyproject_TEMPLATE.toml.jinja2` IN the `dev/breeze/src/airflow_breeze/templates` DIRECTORY +[build-system] +requires = ["flit_core==3.12.0"] +build-backend = "flit_core.buildapi" + +[project] +name = "apache-airflow-providers-ibm-mq" +version = "0.1.0" +description = "Provider package apache-airflow-providers-ibm-mq for Apache Airflow" +readme = "README.rst" +license = "Apache-2.0" +license-files = ['LICENSE', 'NOTICE'] +authors = [ + {name="Apache Software Foundation", email="[email protected]"}, +] +maintainers = [ + {name="Apache Software Foundation", email="[email protected]"}, +] +keywords = [ "airflow-provider", "ibm.mq", "airflow", "integration" ] +classifiers = [ + "Development Status :: 5 - Production/Stable", + "Environment :: Console", + "Environment :: Web Environment", + "Intended Audience :: Developers", + "Intended Audience :: System Administrators", + "Framework :: Apache Airflow", + "Framework :: Apache Airflow :: Provider", + "Programming Language :: Python :: 3.10", + "Programming Language :: Python :: 3.11", + "Programming Language :: Python :: 3.12", + "Programming Language :: Python :: 3.13", + "Topic :: System :: Monitoring", +] +requires-python = ">=3.10" + +# The dependencies should be modified in place in the generated file. +# Any change in the dependencies is preserved when the file is regenerated +# Make sure to run ``prek update-providers-dependencies --all-files`` +# After you modify the dependencies, and rebuild your Breeze CI image with ``breeze ci-image build`` +dependencies = [ + "apache-airflow>=2.11.0", + "asgiref>=2.3.0", +] + +# The optional dependencies should be modified in place in the generated file +# Any change in the dependencies is preserved when the file is regenerated +[project.optional-dependencies] +"ibmmq" = [ + # Required at Runtime Review Comment: The main `dependencies` list is missing `apache-airflow-providers-common-compat`. The hook needs the compat layer for `BaseHook` import, and `version_compat.py` is used in both the trigger and queue modules. The Kafka provider (similar pattern) includes `common-compat` as a hard dependency. Without it, `pip install apache-airflow-providers-ibm-mq` would fail on import. ########## providers/ibm/mq/src/airflow/providers/ibm/mq/hooks/mq.py: ########## @@ -0,0 +1,358 @@ +# +# 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 annotations + +import asyncio +import json +import threading +from contextlib import contextmanager, suppress +from typing import Any + +from asgiref.sync import sync_to_async + +from airflow.sdk.bases.hook import BaseHook + +# Backoff parameters for transient consume failures +_BACKOFF_BASE: float = 1.0 +_BACKOFF_MAX: float = 60.0 +_BACKOFF_FACTOR: float = 2.0 + + +class IBMMQHook(BaseHook): + conn_name_attr = "conn_id" + default_conn_name = "mq_default" + conn_type = "mq" + hook_name = "IBM MQ" + default_open_options = "MQOO_INPUT_SHARED" + + def __init__( + self, + conn_id: str = default_conn_name, + queue_manager: str | None = None, + channel: str | None = None, + open_options: int | None = None, + ): + super().__init__() + self.conn_id = conn_id + self.queue_manager = queue_manager + self.channel = channel + self.open_options = open_options + self._conn = None + + @classmethod + def get_ui_field_behaviour(cls) -> dict[str, Any]: + """Return custom UI field behaviour for IBM MQ Connection.""" + return { + "hidden_fields": ["schema"], + "placeholders": { + "host": "mq.example.com", + "port": "1414", + "login": "app_user", + "extra": json.dumps( + { + "queue_manager": "QM1", + "channel": "DEV.APP.SVRCONN", + "open_options": cls.default_open_options, + }, + indent=2, + ), + }, + } + + @classmethod + def get_open_options_flags(cls, open_options: int) -> list[str]: + """ + Return the symbolic MQ open option flags set in a given bitmask. + + Each flag corresponds to a constant in 'ibmmq.CMQC' that starts with 'MQOO_'. + + :param open_options: The integer bitmask used when opening an MQ queue + (e.g., 'MQOO_INPUT_EXCLUSIVE | MQOO_FAIL_IF_QUIESCING'). + + :return: A list of the names of the MQ open flags that are set in the bitmask. + For example, '['MQOO_INPUT_EXCLUSIVE', 'MQOO_FAIL_IF_QUIESCING']'. + + Example: + >>> open_options = ibmmq.CMQC.MQOO_INPUT_SHARED | ibmmq.CMQC.MQOO_FAIL_IF_QUIESCING + >>> cls.get_open_options_flags(open_options) + ['MQOO_INPUT_SHARED', 'MQOO_FAIL_IF_QUIESCING'] + """ + import ibmmq + + return [ + name + for name, value in vars(ibmmq.CMQC).items() + if name.startswith("MQOO_") and (open_options & value) + ] + + @contextmanager + def get_conn(self): + """ + Sync context manager for IBM MQ connection lifecycle. + + Must be called from the executor thread (not the event loop thread). + Retrieves the Airflow connection, extracts MQ parameters, and manages + the IBM MQ connection lifecycle. + + :yield: IBM MQ connection object + """ + import ibmmq + + connection = BaseHook.get_connection(self.conn_id) + config = connection.extra_dejson + queue_manager = self.queue_manager or config.get("queue_manager") + channel = self.channel or config.get("channel") + + if not queue_manager: + raise ValueError("queue_manager must be set in Connection extra config or hook init") + if not channel: + raise ValueError("channel must be set in Connection extra config or hook init") + + self.open_options = self.open_options or getattr( + ibmmq.CMQC, + config.get("open_options", self.default_open_options), + ibmmq.CMQC.MQOO_INPUT_EXCLUSIVE, + ) + + csp = ibmmq.CSP() + csp.CSPUserId = connection.login + csp.CSPPassword = connection.password + + conn = None + try: + conn = ibmmq.connect( + queue_manager, + channel, + f"{connection.host}({connection.port})", + csp=csp, + ) + yield conn + finally: + if conn: + with suppress(Exception): + conn.disconnect() + + def _process_message(self, message: bytes) -> str: + """ + Process a raw MQ message. Review Comment: No error handling around the `ibmmq.connect()` call. If `connect()` raises (wrong credentials, network issues), the exception type and message from the IBM MQ C client can be cryptic. For consistency with `_consume_sync` and `_produce_sync` (which catch `ibmmq.MQMIError`), consider catching and wrapping connection errors with a more helpful message that includes the queue manager name and host. ########## providers/ibm/mq/src/airflow/providers/ibm/mq/hooks/mq.py: ########## @@ -0,0 +1,358 @@ +# +# 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 annotations + +import asyncio +import json +import threading +from contextlib import contextmanager, suppress +from typing import Any + +from asgiref.sync import sync_to_async + +from airflow.sdk.bases.hook import BaseHook + +# Backoff parameters for transient consume failures +_BACKOFF_BASE: float = 1.0 +_BACKOFF_MAX: float = 60.0 +_BACKOFF_FACTOR: float = 2.0 + + +class IBMMQHook(BaseHook): + conn_name_attr = "conn_id" + default_conn_name = "mq_default" + conn_type = "mq" + hook_name = "IBM MQ" + default_open_options = "MQOO_INPUT_SHARED" + + def __init__( + self, + conn_id: str = default_conn_name, + queue_manager: str | None = None, + channel: str | None = None, + open_options: int | None = None, + ): + super().__init__() + self.conn_id = conn_id + self.queue_manager = queue_manager + self.channel = channel + self.open_options = open_options + self._conn = None + + @classmethod + def get_ui_field_behaviour(cls) -> dict[str, Any]: + """Return custom UI field behaviour for IBM MQ Connection.""" + return { + "hidden_fields": ["schema"], + "placeholders": { + "host": "mq.example.com", + "port": "1414", + "login": "app_user", + "extra": json.dumps( + { + "queue_manager": "QM1", + "channel": "DEV.APP.SVRCONN", + "open_options": cls.default_open_options, + }, + indent=2, + ), + }, + } + + @classmethod + def get_open_options_flags(cls, open_options: int) -> list[str]: + """ + Return the symbolic MQ open option flags set in a given bitmask. + + Each flag corresponds to a constant in 'ibmmq.CMQC' that starts with 'MQOO_'. + + :param open_options: The integer bitmask used when opening an MQ queue + (e.g., 'MQOO_INPUT_EXCLUSIVE | MQOO_FAIL_IF_QUIESCING'). + + :return: A list of the names of the MQ open flags that are set in the bitmask. + For example, '['MQOO_INPUT_EXCLUSIVE', 'MQOO_FAIL_IF_QUIESCING']'. + + Example: + >>> open_options = ibmmq.CMQC.MQOO_INPUT_SHARED | ibmmq.CMQC.MQOO_FAIL_IF_QUIESCING + >>> cls.get_open_options_flags(open_options) + ['MQOO_INPUT_SHARED', 'MQOO_FAIL_IF_QUIESCING'] + """ + import ibmmq + + return [ + name + for name, value in vars(ibmmq.CMQC).items() + if name.startswith("MQOO_") and (open_options & value) + ] + + @contextmanager + def get_conn(self): + """ + Sync context manager for IBM MQ connection lifecycle. + + Must be called from the executor thread (not the event loop thread). + Retrieves the Airflow connection, extracts MQ parameters, and manages + the IBM MQ connection lifecycle. + + :yield: IBM MQ connection object + """ + import ibmmq + + connection = BaseHook.get_connection(self.conn_id) + config = connection.extra_dejson + queue_manager = self.queue_manager or config.get("queue_manager") + channel = self.channel or config.get("channel") + + if not queue_manager: + raise ValueError("queue_manager must be set in Connection extra config or hook init") + if not channel: + raise ValueError("channel must be set in Connection extra config or hook init") + + self.open_options = self.open_options or getattr( + ibmmq.CMQC, + config.get("open_options", self.default_open_options), + ibmmq.CMQC.MQOO_INPUT_EXCLUSIVE, + ) + + csp = ibmmq.CSP() + csp.CSPUserId = connection.login + csp.CSPPassword = connection.password + + conn = None + try: + conn = ibmmq.connect( + queue_manager, + channel, + f"{connection.host}({connection.port})", + csp=csp, + ) + yield conn + finally: + if conn: + with suppress(Exception): + conn.disconnect() + + def _process_message(self, message: bytes) -> str: + """ + Process a raw MQ message. + + If the message contains an RFH2 header, the header is unpacked and the + payload following the header is returned. If unpacking fails, the raw + message is returned decoded as UTF-8. + + :param message: Raw message received from IBM MQ. + :return: Decoded message payload. + """ + import ibmmq + + try: + rfh2 = ibmmq.RFH2() + rfh2.unpack(message) + + payload_offset = rfh2.get_length() + payload = message[payload_offset:] + + decoded = payload.decode("utf-8", errors="ignore") + self.log.info("Message received from MQ (RFH2 decoded): %s", decoded) + return decoded + except ibmmq.PYIFError as error: # RFH2 header not present or unpack failed + self.log.warning( + "Failed to unpack RFH2 header (%s). Returning raw message payload: %s", error, message + ) + return message.decode("utf-8", errors="ignore") + + async def consume(self, queue_name: str, poll_interval: float = 5) -> str: + """ + Wait for a single message from the specified IBM MQ queue and return its decoded payload. + + The method retries with exponential back-off whenever the underlying + ``_consume_sync`` returns ``None`` (connection broken, timeout) or raises + an unexpected exception, so that an AssetWatcher is never silently killed + by a transient failure. + + All blocking IBM MQ operations ('connect', 'open', 'get', 'close', 'disconnect') run in a + separate thread via 'sync_to_async' to satisfy the IBM MQ C client's thread-affinity + requirement — every operation on a connection must happen from the thread that created it. + + A :class:`threading.Event` stop signal is passed to the worker so that, when this + coroutine is canceled (e.g. because the Airflow triggerer reassigns the watcher to + another pod), the background thread exits cleanly after the current 'q.get' call + times out (at most 'poll_interval' seconds). Without this, an orphaned thread could + silently consume a message after cancellation, causing the event to be lost and the + DAG never to run. + + :param queue_name: Name of the IBM MQ queue to consume messages from. + :param poll_interval: Interval in seconds used to wait for messages and to control + how long the underlying MQ 'get' operation blocks before checking again. + :return: The decoded message payload. + """ + backoff = _BACKOFF_BASE Review Comment: A new `threading.Event` is created on every retry iteration. If `sync_to_async` spawns a thread that outlives the iteration (e.g. the background thread is still blocked in `q.get`), the old stop event is lost because nothing holds a reference to it after the next loop. Moving `stop_event = threading.Event()` before the `while True` loop and reusing it across retries would let all spawned threads receive the cancellation signal. ########## providers/ibm/mq/src/airflow/providers/ibm/mq/hooks/mq.py: ########## @@ -0,0 +1,358 @@ +# +# 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 annotations + +import asyncio +import json +import threading +from contextlib import contextmanager, suppress +from typing import Any + +from asgiref.sync import sync_to_async + +from airflow.sdk.bases.hook import BaseHook + +# Backoff parameters for transient consume failures +_BACKOFF_BASE: float = 1.0 +_BACKOFF_MAX: float = 60.0 +_BACKOFF_FACTOR: float = 2.0 + + +class IBMMQHook(BaseHook): + conn_name_attr = "conn_id" + default_conn_name = "mq_default" + conn_type = "mq" + hook_name = "IBM MQ" + default_open_options = "MQOO_INPUT_SHARED" + + def __init__( + self, + conn_id: str = default_conn_name, + queue_manager: str | None = None, + channel: str | None = None, + open_options: int | None = None, + ): + super().__init__() + self.conn_id = conn_id + self.queue_manager = queue_manager + self.channel = channel + self.open_options = open_options + self._conn = None + + @classmethod + def get_ui_field_behaviour(cls) -> dict[str, Any]: + """Return custom UI field behaviour for IBM MQ Connection.""" + return { + "hidden_fields": ["schema"], + "placeholders": { + "host": "mq.example.com", + "port": "1414", + "login": "app_user", + "extra": json.dumps( + { + "queue_manager": "QM1", + "channel": "DEV.APP.SVRCONN", + "open_options": cls.default_open_options, + }, + indent=2, + ), + }, + } + + @classmethod + def get_open_options_flags(cls, open_options: int) -> list[str]: + """ + Return the symbolic MQ open option flags set in a given bitmask. + + Each flag corresponds to a constant in 'ibmmq.CMQC' that starts with 'MQOO_'. + + :param open_options: The integer bitmask used when opening an MQ queue + (e.g., 'MQOO_INPUT_EXCLUSIVE | MQOO_FAIL_IF_QUIESCING'). + + :return: A list of the names of the MQ open flags that are set in the bitmask. + For example, '['MQOO_INPUT_EXCLUSIVE', 'MQOO_FAIL_IF_QUIESCING']'. + + Example: + >>> open_options = ibmmq.CMQC.MQOO_INPUT_SHARED | ibmmq.CMQC.MQOO_FAIL_IF_QUIESCING + >>> cls.get_open_options_flags(open_options) + ['MQOO_INPUT_SHARED', 'MQOO_FAIL_IF_QUIESCING'] + """ + import ibmmq + + return [ + name + for name, value in vars(ibmmq.CMQC).items() + if name.startswith("MQOO_") and (open_options & value) + ] + + @contextmanager + def get_conn(self): + """ + Sync context manager for IBM MQ connection lifecycle. + + Must be called from the executor thread (not the event loop thread). + Retrieves the Airflow connection, extracts MQ parameters, and manages + the IBM MQ connection lifecycle. + + :yield: IBM MQ connection object + """ + import ibmmq + + connection = BaseHook.get_connection(self.conn_id) + config = connection.extra_dejson + queue_manager = self.queue_manager or config.get("queue_manager") + channel = self.channel or config.get("channel") + + if not queue_manager: + raise ValueError("queue_manager must be set in Connection extra config or hook init") + if not channel: + raise ValueError("channel must be set in Connection extra config or hook init") + + self.open_options = self.open_options or getattr( + ibmmq.CMQC, + config.get("open_options", self.default_open_options), + ibmmq.CMQC.MQOO_INPUT_EXCLUSIVE, + ) + + csp = ibmmq.CSP() + csp.CSPUserId = connection.login + csp.CSPPassword = connection.password + + conn = None + try: + conn = ibmmq.connect( + queue_manager, + channel, + f"{connection.host}({connection.port})", + csp=csp, + ) + yield conn + finally: + if conn: + with suppress(Exception): + conn.disconnect() + + def _process_message(self, message: bytes) -> str: + """ + Process a raw MQ message. + + If the message contains an RFH2 header, the header is unpacked and the + payload following the header is returned. If unpacking fails, the raw + message is returned decoded as UTF-8. + + :param message: Raw message received from IBM MQ. + :return: Decoded message payload. + """ + import ibmmq + + try: + rfh2 = ibmmq.RFH2() Review Comment: `_process_message` logs the full message payload at INFO level. For IBM MQ, messages can be large binary blobs or contain sensitive data (financial transactions, PII). This should be DEBUG level, or at least truncated. ########## providers/ibm/mq/src/airflow/providers/ibm/mq/hooks/__init__.py: ########## Review Comment: This file was created via `rename from airflow-core/src/airflow/_shared/__init__.py`. This means the PR deletes `_shared/__init__.py` from `airflow-core`. Even though the file contents (just a `pkgutil.extend_path` line) are identical, this removes a file from the core package. This looks like an accident in the diff — probably from a rebase or file copy gone wrong. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
