You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2022/08/30 07:36:44 UTC

[GitHub] [airflow] phanikumv commented on a diff in pull request #26038: Implement Azure synapse operator

phanikumv commented on code in PR #26038:
URL: https://github.com/apache/airflow/pull/26038#discussion_r958112111


##########
tests/providers/microsoft/azure/hooks/test_azure_synapse.py:
##########
@@ -0,0 +1,186 @@
+# 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 json
+from typing import Type
+from unittest.mock import MagicMock, patch
+
+import pytest
+from azure.identity import ClientSecretCredential, DefaultAzureCredential
+from azure.synapse.spark import SparkClient
+from pytest import fixture
+
+from airflow.models.connection import Connection
+from airflow.providers.microsoft.azure.hooks.synapse import AzureSynapseHook, AzureSynapseSparkBatchRunStatus
+from airflow.utils import db
+
+DEFAULT_SPARK_POOL = "defaultSparkPool"
+
+DEFAULT_SYNAPSE = "defaultSynapse"
+SYNAPSE = "testSynapse"
+
+DEFAULT_CONNECTION_CLIENT_SECRET = "azure_synapse_test_client_secret"
+DEFAULT_CONNECTION_DEFAULT_CREDENTIAL = "azure_synapse_test_default_credential"
+
+MODEL = object()
+NAME = "testName"
+ID = "testId"
+JOB_ID = 1
+
+
+def setup_module():
+    connection_client_secret = Connection(
+        conn_id=DEFAULT_CONNECTION_CLIENT_SECRET,
+        conn_type="azure_synapse",
+        host="https://testsynapse.dev.azuresynapse.net",
+        login="clientId",
+        password="clientSecret",
+        extra=json.dumps(
+            {
+                "extra__azure_synapse__tenantId": "tenantId",
+                "extra__azure_synapse__subscriptionId": "subscriptionId",
+                "extra__azure_synapse__spark_pool": DEFAULT_SPARK_POOL,

Review Comment:
   it should be removed from the connection here. It was  leftover from a previous implementation. I made the changes.



##########
docs/apache-airflow-providers-microsoft-azure/connections/azure_synapse.rst:
##########
@@ -0,0 +1,70 @@
+.. 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.
+
+
+
+.. _howto/connection:synapse:
+
+Microsoft Azure Synapse
+=======================
+
+The Microsoft Azure Synapse connection type enables the Azure Synapse Integrations.
+
+Authenticating to Azure Synapse
+-------------------------------
+
+There are multiple ways to connect to Azure Synapse using Airflow.
+
+1. Use `token credentials
+   <https://docs.microsoft.com/en-us/azure/developer/python/azure-sdk-authenticate?tabs=cmd#authenticate-with-token-credentials>`_
+   i.e. add specific credentials (client_id, secret, tenant) and subscription id to the Airflow connection.
+2. Fallback on `DefaultAzureCredential
+   <https://docs.microsoft.com/en-us/python/api/overview/azure/identity-readme?view=azure-python#defaultazurecredential>`_.
+   This includes a mechanism to try different options to authenticate: Managed System Identity, environment variables, authentication through Azure CLI...
+
+Default Connection IDs
+----------------------
+
+All hooks and operators related to Microsoft Azure Data Factory use ``azure_synapse_default`` by default.
+
+Configuring the Connection
+--------------------------
+
+Client ID
+    Specify the ``client_id`` used for the initial connection.
+    This is needed for *token credentials* authentication mechanism.
+    It can be left out to fall back on ``DefaultAzureCredential``.
+
+Secret
+    Specify the ``secret`` used for the initial connection.
+    This is needed for *token credentials* authentication mechanism.
+    It can be left out to fall back on ``DefaultAzureCredential``.
+
+Tenant ID
+    Specify the Azure tenant ID used for the initial connection.
+    This is needed for *token credentials* authentication mechanism.
+    It can be left out to fall back on ``DefaultAzureCredential``.
+    Use the key ``extra__azure_synapse__tenantId`` to pass in the tenant ID.
+
+Subscription ID
+    Specify the ID of the subscription used for the initial connection.
+    This is needed for all authentication mechanisms.
+    Use the key ``extra__azure_synapse__subscriptionId`` to pass in the Azure subscription ID.
+
+Synapse Workspace URL
+    Specify the Azure Synapse endpoint to interface with.
+    If not specified in the connection, this needs to be passed in directly to hooks, operators, and sensors.

Review Comment:
   removed the last sentence.



-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org