You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by sa...@apache.org on 2018/05/08 18:13:11 UTC

incubator-airflow git commit: [AIRFLOW-1978] Add WinRM windows operator and hook

Repository: incubator-airflow
Updated Branches:
  refs/heads/master b18b437c2 -> e5f2a38d6


[AIRFLOW-1978] Add WinRM windows operator and hook

Closes #3316 from cloneluke/winrm_connector2


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/e5f2a38d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/e5f2a38d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/e5f2a38d

Branch: refs/heads/master
Commit: e5f2a38d6aff69bb4d31c792f7e9234b51646675
Parents: b18b437
Author: Luke Bodeen <lb...@docutap.com>
Authored: Tue May 8 11:12:52 2018 -0700
Committer: r39132 <si...@yahoo.com>
Committed: Tue May 8 11:12:59 2018 -0700

----------------------------------------------------------------------
 .../example_dags/example_winrm_operator.py      |  73 ++++++++++
 airflow/contrib/hooks/__init__.py               |   1 +
 airflow/contrib/hooks/winrm_hook.py             | 137 +++++++++++++++++++
 airflow/contrib/operators/__init__.py           |   1 +
 airflow/contrib/operators/winrm_operator.py     | 110 +++++++++++++++
 docs/code.rst                                   |   6 +-
 setup.py                                        |  76 +++++-----
 7 files changed, 370 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/e5f2a38d/airflow/contrib/example_dags/example_winrm_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/example_dags/example_winrm_operator.py b/airflow/contrib/example_dags/example_winrm_operator.py
new file mode 100644
index 0000000..195bf5d
--- /dev/null
+++ b/airflow/contrib/example_dags/example_winrm_operator.py
@@ -0,0 +1,73 @@
+# -*- coding: utf-8 -*-
+#
+# 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.
+# --------------------------------------------------------------------------------
+# Written By: Ekhtiar Syed
+# Last Update: 8th April 2016
+# Caveat: This Dag will not run because of missing scripts.
+# The purpose of this is to give you a sample of a real world example DAG!
+# --------------------------------------------------------------------------------
+
+# --------------------------------------------------------------------------------
+# Load The Dependencies
+# --------------------------------------------------------------------------------
+import airflow
+from airflow.operators.dummy_operator import DummyOperator
+from airflow.models import DAG
+from datetime import timedelta
+
+from airflow.contrib.hooks import WinRMHook
+from airflow.contrib.operators.winrm_operator import WinRMOperator
+
+
+args = {
+    'owner': 'airflow',
+    'start_date': airflow.utils.dates.days_ago(2)
+}
+
+dag = DAG(
+    dag_id='POC_winrm_parallel', default_args=args,
+    schedule_interval='0 0 * * *',
+    dagrun_timeout=timedelta(minutes=60))
+
+cmd = 'ls -l'
+run_this_last = DummyOperator(task_id='run_this_last', dag=dag)
+
+winRMHook = WinRMHook(ssh_conn_id='ssh_POC1')
+
+t1 = WinRMOperator(
+    task_id="wintask1",
+    command='ls -altr',
+    winrm_hook=winRMHook,
+    dag=dag)
+
+t2 = WinRMOperator(
+    task_id="wintask2",
+    command='sleep 60',
+    winrm_hook=winRMHook,
+    dag=dag)
+
+t3 = WinRMOperator(
+    task_id="wintask3",
+    command='echo \'luke test\' ',
+    winrm_hook=winRMHook,
+    dag=dag)
+
+t1.set_downstream(run_this_last)
+t2.set_downstream(run_this_last)
+t3.set_downstream(run_this_last)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/e5f2a38d/airflow/contrib/hooks/__init__.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/__init__.py b/airflow/contrib/hooks/__init__.py
index 7f9d59e..81ce8f0 100644
--- a/airflow/contrib/hooks/__init__.py
+++ b/airflow/contrib/hooks/__init__.py
@@ -42,6 +42,7 @@ _hooks = {
     'ftps_hook': ['FTPSHook'],
     'vertica_hook': ['VerticaHook'],
     'ssh_hook': ['SSHHook'],
+    'winrm_hook': ['WinRMHook'],
     'sftp_hook': ['SFTPHook'],
     'bigquery_hook': ['BigQueryHook'],
     'qubole_hook': ['QuboleHook'],

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/e5f2a38d/airflow/contrib/hooks/winrm_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/winrm_hook.py b/airflow/contrib/hooks/winrm_hook.py
new file mode 100644
index 0000000..0be904b
--- /dev/null
+++ b/airflow/contrib/hooks/winrm_hook.py
@@ -0,0 +1,137 @@
+# -*- coding: utf-8 -*-
+#
+# 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 getpass
+from winrm.protocol import Protocol
+from airflow.exceptions import AirflowException
+from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class WinRMHook(BaseHook, LoggingMixin):
+
+    """
+    Hook for winrm remote execution using pywinrm.
+
+    :param ssh_conn_id: connection id from airflow Connections from where all
+        the required parameters can be fetched like username, password or key_file.
+        Thought the priority is given to the param passed during init
+    :type ssh_conn_id: str
+    :param remote_host: remote host to connect
+    :type remote_host: str
+    :param username: username to connect to the remote_host
+    :type username: str
+    :param password: password of the username to connect to the remote_host
+    :type password: str
+    :param key_file: key file to use to connect to the remote_host.
+    :type key_file: str
+    :param timeout: timeout for the attempt to connect to the remote_host.
+    :type timeout: int
+    :param keepalive_interval: send a keepalive packet to remote host
+        every keepalive_interval seconds
+    :type keepalive_interval: int
+    """
+
+    def __init__(self,
+                 ssh_conn_id=None,
+                 remote_host=None,
+                 username=None,
+                 password=None,
+                 key_file=None,
+                 timeout=10,
+                 keepalive_interval=30
+                 ):
+        super(WinRMHook, self).__init__(ssh_conn_id)
+        # TODO make new win rm connection class
+        self.ssh_conn_id = ssh_conn_id
+        self.remote_host = remote_host
+        self.username = username
+        self.password = password
+        self.key_file = key_file
+        self.timeout = timeout
+        self.keepalive_interval = keepalive_interval
+        # Default values, overridable from Connection
+        self.compress = True
+        self.no_host_key_check = True
+        self.client = None
+        self.winrm_protocol = None
+
+    def get_conn(self):
+        if not self.client:
+            self.log.debug('Creating WinRM client for conn_id: %s', self.ssh_conn_id)
+            if self.ssh_conn_id is not None:
+                conn = self.get_connection(self.ssh_conn_id)
+                if self.username is None:
+                    self.username = conn.login
+                if self.password is None:
+                    self.password = conn.password
+                if self.remote_host is None:
+                    self.remote_host = conn.host
+                if conn.extra is not None:
+                    extra_options = conn.extra_dejson
+                    self.key_file = extra_options.get("key_file")
+
+                    if "timeout" in extra_options:
+                        self.timeout = int(extra_options["timeout"], 10)
+
+                    if "compress" in extra_options \
+                            and extra_options["compress"].lower() == 'false':
+                        self.compress = False
+                    if "no_host_key_check" in extra_options \
+                            and extra_options["no_host_key_check"].lower() == 'false':
+                        self.no_host_key_check = False
+
+            if not self.remote_host:
+                raise AirflowException("Missing required param: remote_host")
+
+            # Auto detecting username values from system
+            if not self.username:
+                self.log.debug(
+                    "username to ssh to host: %s is not specified for connection id"
+                    " %s. Using system's default provided by getpass.getuser()",
+                    self.remote_host, self.ssh_conn_id
+                )
+                self.username = getpass.getuser()
+
+            try:
+
+                if self.password and self.password.strip():
+                    self.winrm_protocol = Protocol(
+                        # TODO pass in port from ssh conn
+                        endpoint='http://' + self.remote_host + ':5985/wsman',
+                        # TODO get cert transport working
+                        # transport='certificate',
+                        transport='plaintext',
+                        # cert_pem=r'publickey.pem',
+                        # cert_key_pem=r'dev.pem',
+                        read_timeout_sec=70,
+                        operation_timeout_sec=60,
+                        username=self.username,
+                        password=self.password,
+                        server_cert_validation='ignore')
+
+                self.log.info("Opening WinRM shell")
+                self.client = self.winrm_protocol.open_shell()
+
+            except Exception as error:
+                self.log.error(
+                    "Error connecting to host: %s, error: %s",
+                    self.remote_host, error
+                )
+        return self.client

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/e5f2a38d/airflow/contrib/operators/__init__.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/__init__.py b/airflow/contrib/operators/__init__.py
index a982db7..436878b 100644
--- a/airflow/contrib/operators/__init__.py
+++ b/airflow/contrib/operators/__init__.py
@@ -38,6 +38,7 @@ import sys
 # ------------------------------------------------------------------------
 _operators = {
     'ssh_operator': ['SSHOperator'],
+    'winrm_operator': ['WinRMOperator'],
     'vertica_operator': ['VerticaOperator'],
     'vertica_to_hive': ['VerticaToHiveTransfer'],
     'qubole_operator': ['QuboleOperator'],

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/e5f2a38d/airflow/contrib/operators/winrm_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/winrm_operator.py b/airflow/contrib/operators/winrm_operator.py
new file mode 100644
index 0000000..c426c02
--- /dev/null
+++ b/airflow/contrib/operators/winrm_operator.py
@@ -0,0 +1,110 @@
+# -*- coding: utf-8 -*-
+#
+# 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 airflow.contrib.hooks.winrm_hook import WinRMHook
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.utils.decorators import apply_defaults
+
+
+class WinRMOperator(BaseOperator):
+
+    """
+    WinRMOperator to execute commands on given remote host using the winrm_hook.
+
+    :param winrm_hook: predefined ssh_hook to use for remote execution
+    :type winrm_hook: :class:`WinRMHook`
+    :param ssh_conn_id: connection id from airflow Connections
+    :type ssh_conn_id: str
+    :param remote_host: remote host to connect
+    :type remote_host: str
+    :param command: command to execute on remote host
+    :type command: str
+    :param timeout: timeout for executing the command.
+    :type timeout: int
+    :param do_xcom_push: return the stdout which also get set in xcom by airflow platform
+    :type do_xcom_push: bool
+    """
+
+    template_fields = ('command',)
+
+    @apply_defaults
+    def __init__(self,
+                 winrm_hook=None,
+                 ssh_conn_id=None,
+                 remote_host=None,
+                 command=None,
+                 timeout=10,
+                 do_xcom_push=False,
+                 *args,
+                 **kwargs):
+        super(WinRMOperator, self).__init__(*args, **kwargs)
+        self.winrm_hook = winrm_hook
+        self.ssh_conn_id = ssh_conn_id
+        self.remote_host = remote_host
+        self.command = command
+        self.timeout = timeout
+        self.do_xcom_push = do_xcom_push
+
+    def execute(self, context):
+        try:
+            if self.ssh_conn_id and not self.winrm_hook:
+                self.log.info("hook not found, creating")
+                self.winrm_hook = WinRMHook(ssh_conn_id=self.ssh_conn_id)
+
+            if not self.winrm_hook:
+                raise AirflowException("can not operate without ssh_hook or ssh_conn_id")
+
+            if self.remote_host is not None:
+                self.winrm_hook.remote_host = self.remote_host
+
+            winrm_client = self.winrm_hook.get_conn()
+            self.log.info("Established WinRM connection")
+
+            if not self.command:
+                raise AirflowException("no command specified so nothing to execute here.")
+
+            self.log.info(
+                "Starting command: '{command}' on remote host: {remotehost}".
+                format(command=self.command, remotehost=self.winrm_hook.remote_host)
+            )
+            command_id = self.winrm_hook.winrm_protocol. \
+                run_command(winrm_client, self.command)
+            std_out, std_err, status_code = self.winrm_hook.winrm_protocol. \
+                get_command_output(winrm_client, command_id)
+
+            self.log.info("std out: " + std_out.decode())
+            self.log.info("std err: " + std_err.decode())
+            self.log.info("exit code: " + str(status_code))
+            self.log.info("Cleaning up WinRM command")
+            self.winrm_hook.winrm_protocol.cleanup_command(winrm_client, command_id)
+            self.log.info("Cleaning up WinRM protocol shell")
+            self.winrm_hook.winrm_protocol.close_shell(winrm_client)
+            if status_code is 0:
+                return std_out.decode()
+
+            else:
+                error_msg = std_err.decode()
+                raise AirflowException("error running cmd: {0}, error: {1}"
+                                       .format(self.command, error_msg))
+
+        except Exception as e:
+            raise AirflowException("WinRM operator error: {0}".format(str(e)))
+
+        return True

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/e5f2a38d/docs/code.rst
----------------------------------------------------------------------
diff --git a/docs/code.rst b/docs/code.rst
index c979f26..596e9c3 100644
--- a/docs/code.rst
+++ b/docs/code.rst
@@ -108,6 +108,7 @@ Community-contributed Operators
 
 Operators
 ^^^^^^^^^
+.. Alphabetize this list
 
 .. autoclass:: airflow.contrib.operators.awsbatch_operator.AWSBatchOperator
 .. autoclass:: airflow.contrib.operators.bigquery_check_operator.BigQueryCheckOperator
@@ -181,6 +182,7 @@ Operators
 .. autoclass:: airflow.contrib.operators.ssh_operator.SSHOperator
 .. autoclass:: airflow.contrib.operators.vertica_operator.VerticaOperator
 .. autoclass:: airflow.contrib.operators.vertica_to_hive.VerticaToHiveTransfer
+.. autoclass:: airflow.contrib.operators.winrm_operator.WinRMOperator
 
 Sensors
 ^^^^^^^
@@ -343,7 +345,7 @@ interface when possible and acting as building blocks for operators.
 
 Community contributed hooks
 '''''''''''''''''''''''''''
-
+.. Alphabetize this list
 .. autoclass:: airflow.contrib.hooks.aws_dynamodb_hook.AwsDynamoDBHook
 .. autoclass:: airflow.contrib.hooks.aws_hook.AwsHook
 .. autoclass:: airflow.contrib.hooks.aws_lambda_hook.AwsLambdaHook
@@ -379,8 +381,8 @@ Community contributed hooks
 .. autoclass:: airflow.contrib.hooks.sqoop_hook.SqoopHook
 .. autoclass:: airflow.contrib.hooks.ssh_hook.SSHHook
 .. autoclass:: airflow.contrib.hooks.vertica_hook.VerticaHook
-.. autoclass:: airflow.contrib.hooks.spark_jdbc_hook.SparkJDBCHook
 .. autoclass:: airflow.contrib.hooks.wasb_hook.WasbHook
+.. autoclass:: airflow.contrib.hooks.winrm_hook.WinRMHook
 
 Executors
 ---------

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/e5f2a38d/setup.py
----------------------------------------------------------------------
diff --git a/setup.py b/setup.py
index 8e4ad5c..3e0b376 100644
--- a/setup.py
+++ b/setup.py
@@ -33,17 +33,21 @@ version = imp.load_source(
 
 PY3 = sys.version_info[0] == 3
 
+
 class Tox(TestCommand):
     user_options = [('tox-args=', None, "Arguments to pass to tox")]
+
     def initialize_options(self):
         TestCommand.initialize_options(self)
         self.tox_args = ''
+
     def finalize_options(self):
         TestCommand.finalize_options(self)
         self.test_args = []
         self.test_suite = True
+
     def run_tests(self):
-        #import here, cause outside the eggs aren't loaded
+        # import here, cause outside the eggs aren't loaded
         import tox
         errno = tox.cmdline(args=self.tox_args.split())
         sys.exit(errno)
@@ -52,10 +56,13 @@ class Tox(TestCommand):
 class CleanCommand(Command):
     """Custom clean command to tidy up the project root."""
     user_options = []
+
     def initialize_options(self):
         pass
+
     def finalize_options(self):
         pass
+
     def run(self):
         os.system('rm -vrf ./build ./dist ./*.pyc ./*.tgz ./*.egg-info')
 
@@ -66,8 +73,8 @@ def git_version(version):
     indicate whether the head of the current git-backed working directory is tied to a
     release tag or not : it will indicate the former with a 'release:{version}' prefix
     and the latter with a 'dev0' prefix. Following the prefix will be a sha of the current
-    branch head. Finally, a "dirty" suffix is appended to indicate that uncommitted changes
-    are present.
+    branch head. Finally, a "dirty" suffix is appended to indicate that uncommitted
+    changes are present.
     """
     repo = None
     try:
@@ -109,10 +116,12 @@ celery = [
 cgroups = [
     'cgroupspy>=0.1.4',
 ]
+# major update coming soon, clamp to 0.x
+cloudant = ['cloudant>=0.5.9,<2.0']
 crypto = ['cryptography>=0.9.3']
 dask = [
     'distributed>=1.17.1, <2'
-    ]
+]
 databricks = ['requests>=2.5.1, <3']
 datadog = ['datadog>=0.14.0']
 doc = [
@@ -135,10 +144,8 @@ gcp_api = [
     'PyOpenSSL',
     'pandas-gbq'
 ]
+github_enterprise = ['Flask-OAuthlib>=0.9.1']
 hdfs = ['snakebite>=2.7.8']
-webhdfs = ['hdfs[dataframe,avro,kerberos]>=2.0.4']
-jenkins = ['python-jenkins>=0.4.15']
-jira = ['JIRA>1.0.7']
 hive = [
     'hmsclient>=0.1.0',
     'pyhive>=0.1.3',
@@ -146,36 +153,38 @@ hive = [
     'unicodecsv>=0.14.1'
 ]
 jdbc = ['jaydebeapi>=1.1.1']
-mssql = ['pymssql>=2.1.1', 'unicodecsv>=0.14.1']
-mysql = ['mysqlclient>=1.3.6']
-rabbitmq = ['librabbitmq>=1.6.1']
-oracle = ['cx_Oracle>=5.1.2']
-postgres = ['psycopg2-binary>=2.7.4']
-pinot = ['pinotdb>=0.1.1']
-ssh = ['paramiko>=2.1.1', 'pysftp>=0.2.9']
-salesforce = ['simple-salesforce>=0.72']
-s3 = ['boto3>=1.7.0']
-samba = ['pysmbclient>=0.1.3']
-slack = ['slackclient>=1.0.0']
-statsd = ['statsd>=3.0.1, <4.0']
-vertica = ['vertica-python>=0.5.1']
-ldap = ['ldap3>=0.9.9.1']
+jenkins = ['python-jenkins>=0.4.15']
+jira = ['JIRA>1.0.7']
 kerberos = ['pykerberos>=1.1.13',
             'requests_kerberos>=0.10.0',
             'thrift_sasl>=0.2.0',
             'snakebite[kerberos]>=2.7.8']
+kubernetes = ['kubernetes>=3.0.0',
+              'cryptography>=2.0.0']
+ldap = ['ldap3>=0.9.9.1']
+mssql = ['pymssql>=2.1.1', 'unicodecsv>=0.14.1']
+mysql = ['mysqlclient>=1.3.6']
+oracle = ['cx_Oracle>=5.1.2']
 password = [
     'bcrypt>=2.0.0',
     'flask-bcrypt>=0.7.1',
 ]
-github_enterprise = ['Flask-OAuthlib>=0.9.1']
+pinot = ['pinotdb>=0.1.1']
+postgres = ['psycopg2-binary>=2.7.4']
 qds = ['qds-sdk>=1.9.6']
-cloudant = ['cloudant>=0.5.9,<2.0'] # major update coming soon, clamp to 0.x
+rabbitmq = ['librabbitmq>=1.6.1']
 redis = ['redis>=2.10.5']
-kubernetes = ['kubernetes>=3.0.0',
-              'cryptography>=2.0.0']
+s3 = ['boto3>=1.7.0']
+salesforce = ['simple-salesforce>=0.72']
+samba = ['pysmbclient>=0.1.3']
+slack = ['slackclient>=1.0.0']
 snowflake = ['snowflake-connector-python>=1.5.2',
              'snowflake-sqlalchemy>=1.1.0']
+ssh = ['paramiko>=2.1.1', 'pysftp>=0.2.9']
+statsd = ['statsd>=3.0.1, <4.0']
+vertica = ['vertica-python>=0.5.1']
+webhdfs = ['hdfs[dataframe,avro,kerberos]>=2.0.4']
+winrm = ['pywinrm==0.2.2']
 zendesk = ['zdesk']
 
 all_dbs = postgres + mysql + hive + mssql + hdfs + vertica + cloudant + druid + pinot
@@ -190,10 +199,11 @@ devel = [
     'nose-ignore-docstring==0.2',
     'nose-timer',
     'parameterized',
-    'qds-sdk>=1.9.6',
-    'rednose',
     'paramiko',
     'pysftp',
+    'pywinrm',
+    'qds-sdk>=1.9.6',
+    'rednose',
     'requests_mock'
 ]
 devel_minreq = devel + kubernetes + mysql + doc + password + s3 + cgroups
@@ -210,6 +220,7 @@ if PY3:
 else:
     devel_ci = devel_all
 
+
 def do_setup():
     write_version()
     setup(
@@ -288,7 +299,9 @@ def do_setup():
             'hdfs': hdfs,
             'hive': hive,
             'jdbc': jdbc,
+            'jira': jira,
             'kerberos': kerberos,
+            'kubernetes': kubernetes,
             'ldap': ldap,
             'mssql': mssql,
             'mysql': mysql,
@@ -298,19 +311,18 @@ def do_setup():
             'postgres': postgres,
             'qds': qds,
             'rabbitmq': rabbitmq,
+            'redis': redis,
             's3': s3,
             'salesforce': salesforce,
             'samba': samba,
-            'sendgrid' : sendgrid,
+            'sendgrid': sendgrid,
             'slack': slack,
+            'snowflake': snowflake,
             'ssh': ssh,
             'statsd': statsd,
             'vertica': vertica,
             'webhdfs': webhdfs,
-            'jira': jira,
-            'redis': redis,
-            'kubernetes': kubernetes,
-            'snowflake': snowflake
+            'winrm': winrm
         },
         classifiers=[
             'Development Status :: 5 - Production/Stable',