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 2021/01/19 17:08:56 UTC

[GitHub] [airflow] potiuk opened a new pull request #13767: Implement target provider versioning tools

potiuk opened a new pull request #13767:
URL: https://github.com/apache/airflow/pull/13767


   This change implements per-provider versioning tools. Version of the
   providers is retrieved from provider.yaml file (top-level verion).
   Documentation is generated in the documentation folder rather than
   in sources and embedded in provider's index. Backport providers
   remain as they were until we delete all the backport references in
   April 2021 nd then the code can be simplified and the
   backport functionality can be removed then.
   
   When generating multiple providers, only those that have version
   that has no corresponding `providers-<PROVIDER>/<VERSION>` are
   generated. Other providers are skipped with warnings.
   
   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/master/UPDATING.md).
   


----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561425618



##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -1225,184 +1195,272 @@ def get_provider_info_from_provider_yaml(provider_package_id: str) -> Dict[str,
     return provider_info
 
 
-def update_generated_files_for_package(
+def get_backport_current_changes_table(previous_release_commit_ref: str, source_provider_package_path: str):
+    git_cmd = get_git_log_command(previous_release_commit_ref)
+    try:
+        changes = subprocess.check_output(git_cmd, cwd=source_provider_package_path, universal_newlines=True)
+        changes_table = convert_git_changes_to_table(
+            previous_release_commit_ref, changes, base_url="https://github.com/apache/airflow/commit/"
+        )
+    except subprocess.CalledProcessError:
+        changes_table = ''
+    return changes_table
+
+
+def get_version_tag(version: str, provider_package_id: str, version_suffix: str):
+    return f"providers-{provider_package_id.replace('.','-')}/{version}{version_suffix}"
+
+
+def print_changes_table(changes_table):
+    syntax = Syntax(changes_table, "rst", theme="light")
+    console = Console(width=200)
+    console.print(syntax)
+
+
+def get_all_changes_for_regular_packages(
+    versions: List[str],
     provider_package_id: str,
-    current_release_version: str,
+    source_provider_package_path: str,
     version_suffix: str,
-    imported_classes: List[str],
-    backport_packages: bool,
-    update_release_notes: bool,
-    update_setup: bool,
-) -> Tuple[int, int]:
-    """
-    Updates release notes (BACKPORT_PROVIDER_README.md/README.md) for the package.
-    Returns Tuple of total number of entities and badly named entities.
+) -> Tuple[bool, str]:
+    current_version = versions[0]
+    current_tag = get_version_tag(current_version, provider_package_id, version_suffix)
+    print(f"Checking if tag `{current_tag}` exists.")
+    if not subprocess.call(
+        get_git_tag_check_command(current_tag),
+        cwd=source_provider_package_path,
+        stderr=subprocess.DEVNULL,
+    ):
+        print(f"The tag {current_tag} exists.")
+        # The tag already exists
+        changes = subprocess.check_output(
+            get_git_log_command(current_tag, 'HEAD'),
+            cwd=source_provider_package_path,
+            universal_newlines=True,
+        )
+        if len(changes) > 0:
+            print(
+                f"[yellow]The provider {provider_package_id} has changes"
+                f" since last release but version is not updated[/]"
+            )
+            print()
+            print(
+                f'Please update version in `airflow/providers/{provider_package_id.replace("-","/")}/'
+                'provider.yaml` to prepare release.\n'
+            )
+            changes_table = convert_git_changes_to_table(
+                "UNKNOWN", changes, base_url="https://github.com/apache/airflow/commit/", markdown=False
+            )
+            print_changes_table(changes_table)
+            return False, changes_table
+        else:
+            print(f"No changes for {provider_package_id}")
+            return False, ""
+    print("The tag does not exist. ")
+    if len(versions) == 1:
+        print("This provider has never been released!\n")
+    else:
+        print(f"The {provider_package_id} is ready to be released!\n")
+    current_version = 'HEAD'
+    changes_table = ''
+    print_version = versions[0]
+    for version in versions[1:]:
+        version_tag = get_version_tag(version, provider_package_id, version_suffix)
+        changes = subprocess.check_output(
+            get_git_log_command(version_tag, current_version),
+            cwd=source_provider_package_path,
+            universal_newlines=True,
+        )
+        changes_table += convert_git_changes_to_table(
+            print_version, changes, base_url="https://github.com/apache/airflow/commit/", markdown=False
+        )
+        current_version = version
+        print_version = version
+    print_changes_table(changes_table)
+    return True, changes_table
+
+
+def get_provider_details(provider_package_id: str) -> ProviderPackageDetails:
+    return ProviderPackageDetails(
+        provider_package_id=provider_package_id,
+        full_package_name=f"airflow.providers.{provider_package_id}",
+        source_provider_package_path=get_source_package_path(provider_package_id),
+        documentation_provider_package_path=get_documentation_package_path(provider_package_id),
+    )
 
-    :param provider_package_id: id of the package
-    :param current_release_version: release version:
-    :param version_suffix: version suffix corresponding to the version in the code
-    :param imported_classes - entities that have been imported from providers
-    :param backport_packages: whether to prepare regular (False) or backport (True) packages
-    :param update_release_notes: whether to update release notes
-    :param update_setup: whether to update setup files
 
-    :return: Tuple of total/bad number of entities
-    """
-    verify_provider_package(provider_package_id)
-    full_package_name = f"airflow.providers.{provider_package_id}"
-    source_provider_package_path = get_source_package_path(provider_package_id)
-    entity_summaries = get_package_class_summary(full_package_name, imported_classes)
-    past_releases = get_all_releases(
-        provider_package_path=source_provider_package_path, backport_packages=backport_packages
+def get_provider_jinja_context(
+    provider_details: ProviderPackageDetails,
+    current_release_version: str,
+    version_suffix: str,
+    backport_packages: bool,
+):
+    verify_provider_package(provider_details.provider_package_id)
+    cross_providers_dependencies = get_cross_provider_dependent_packages(
+        provider_package_id=provider_details.provider_package_id
     )
-    current_release_version, previous_release = check_if_release_version_ok(
-        past_releases, current_release_version, backport_packages
+    release_version_no_leading_zeros = strip_leading_zeros(current_release_version)
+    pip_requirements_table = convert_pip_requirements_to_table(
+        PROVIDERS_REQUIREMENTS[provider_details.provider_package_id]
     )
-    cross_providers_dependencies = get_cross_provider_dependent_packages(
-        provider_package_id=provider_package_id
+    pip_requirements_table_rst = convert_pip_requirements_to_table(
+        PROVIDERS_REQUIREMENTS[provider_details.provider_package_id], markdown=False
     )
-    previous_release = get_previous_release_info(
-        previous_release_version=previous_release,
-        past_releases=past_releases,
-        current_release_version=current_release_version,
+    cross_providers_dependencies_table = convert_cross_package_dependencies_to_table(
+        cross_providers_dependencies,
+        backport_packages=backport_packages,
     )
-    release_version_no_leading_zeros = (
-        strip_leading_zeros_in_calver(current_release_version)
-        if backport_packages
-        else current_release_version
+    cross_providers_dependencies_table_rst = convert_cross_package_dependencies_to_table(
+        cross_providers_dependencies, backport_packages=backport_packages, markdown=False
     )
     context: Dict[str, Any] = {
         "ENTITY_TYPES": list(EntityType),
         "README_FILE": "BACKPORT_PROVIDER_README.md" if backport_packages else "README.md",
-        "PROVIDER_PACKAGE_ID": provider_package_id,
-        "PACKAGE_PIP_NAME": get_pip_package_name(provider_package_id, backport_packages),
-        "FULL_PACKAGE_NAME": full_package_name,
-        "PROVIDER_PATH": full_package_name.replace(".", "/"),
+        "PROVIDER_PACKAGE_ID": provider_details.provider_package_id,
+        "PACKAGE_PIP_NAME": get_pip_package_name(
+            provider_details.provider_package_id, backport_packages=backport_packages
+        ),
+        "FULL_PACKAGE_NAME": provider_details.full_package_name,
+        "PROVIDER_PATH": provider_details.full_package_name.replace(".", "/"),
         "RELEASE": current_release_version,
         "RELEASE_NO_LEADING_ZEROS": release_version_no_leading_zeros,
         "VERSION_SUFFIX": version_suffix,
-        "ADDITIONAL_INFO": get_additional_package_info(provider_package_path=source_provider_package_path),
+        "ADDITIONAL_INFO": get_additional_package_info(
+            provider_package_path=provider_details.source_provider_package_path
+        ),
+        "CHANGELOG": get_changelog_for_package(
+            provider_package_path=provider_details.source_provider_package_path
+        ),
         "CROSS_PROVIDERS_DEPENDENCIES": cross_providers_dependencies,
-        "PIP_REQUIREMENTS": PROVIDERS_REQUIREMENTS[provider_package_id],
-        "PROVIDER_TYPE": "Backport provider" if BACKPORT_PACKAGES else "Provider",
-        "PROVIDERS_FOLDER": "backport-providers" if BACKPORT_PACKAGES else "providers",
+        "PIP_REQUIREMENTS": PROVIDERS_REQUIREMENTS[provider_details.provider_package_id],
+        "PROVIDER_TYPE": "Backport provider" if backport_packages else "Provider",
+        "PROVIDERS_FOLDER": "backport-providers" if backport_packages else "providers",
         "INSTALL_REQUIREMENTS": get_install_requirements(
-            provider_package_id=provider_package_id, backport_packages=backport_packages
+            provider_package_id=provider_details.provider_package_id, backport_packages=backport_packages
         ),
         "SETUP_REQUIREMENTS": get_setup_requirements(),
         "EXTRAS_REQUIREMENTS": get_package_extras(
-            provider_package_id=provider_package_id, backport_packages=backport_packages
+            provider_package_id=provider_details.provider_package_id, backport_packages=backport_packages
         ),
-        "PROVIDER_INFO": get_provider_info_from_provider_yaml(provider_package_id),
+        "CROSS_PROVIDERS_DEPENDENCIES_TABLE": cross_providers_dependencies_table,
+        "CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST": cross_providers_dependencies_table_rst,
+        "PIP_REQUIREMENTS_TABLE": pip_requirements_table,
+        "PIP_REQUIREMENTS_TABLE_RST": pip_requirements_table_rst,
     }
-    if update_release_notes:
-        git_cmd = get_git_command(previous_release)
-        try:
-            changes = subprocess.check_output(
-                git_cmd, cwd=source_provider_package_path, universal_newlines=True
-            )
-            changes_table = convert_git_changes_to_table(
-                changes, base_url="https://github.com/apache/airflow/commit/"
-            )
-        except subprocess.CalledProcessError:
-            # TODO(potiuk) fix me for both backport/provider package check
-            changes_table = ''
-        context["CURRENT_CHANGES_TABLE"] = changes_table
-        pip_requirements_table = convert_pip_requirements_to_table(
-            PROVIDERS_REQUIREMENTS[provider_package_id]
-        )
-        cross_providers_dependencies_table = convert_cross_package_dependencies_to_table(
-            cross_providers_dependencies,
-            backport_packages=backport_packages,
-        )
-        context["CROSS_PROVIDERS_DEPENDENCIES_TABLE"] = cross_providers_dependencies_table
-        context["PIP_REQUIREMENTS_TABLE"] = pip_requirements_table
+    return context
 
-    total, bad = check_if_classes_are_properly_named(entity_summaries)
+
+def prepare_readme_file(
+    context,
+):
+    readme_content = LICENCE
+    readme_template_name = PROVIDER_TEMPLATE_PREFIX + "README"
+    readme_content += render_template(template_name=readme_template_name, context=context, extension='.md')
+    readme_file_path = os.path.join(TARGET_PROVIDER_PACKAGES_PATH, "README.md")
+    with open(readme_file_path, "wt") as readme_file:
+        readme_file.write(readme_content)
+
+
+def update_generated_files_for_regular_package(
+    provider_package_id: str,
+    version_suffix: str,
+    update_release_notes: bool,
+    update_setup: bool,
+) -> None:
+    """
+    Updates generated files (readme, changes and/or setup.cfg/setup.py/manifest.in/provider_info)
+
+    :param provider_package_id: id of the package
+    :param version_suffix: version suffix corresponding to the version in the code
+    :param update_release_notes: whether to update release notes
+    :param update_setup: whether to update setup files
+    """
+    verify_provider_package(provider_package_id)
+    provider_details = get_provider_details(provider_package_id)
+    provider_info = get_provider_info_from_provider_yaml(provider_package_id)
+    versions: List[str] = provider_info['versions']
+    current_release_version = versions[0]
+    # previous_release = versions[1] if len(versions) > 1 else None
+    # previous_release_commit_ref = f"providers-{provider_package_id.replace('.','-')}/{previous_release}"

Review comment:
       this can go I guess




----------------------------------------------------------------
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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561410905



##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       @kaxil For now, I can see that we have two changelogs, one with only a list of breaking changes and the detailed changelog with a list of all commits.
   http://gabby-cough.surge.sh/#changelog
   We don't have any updating guide yet.
   
   I only suggest changing the name from "Changelog" to "Upgrading guide" and from "Detailed changelog" to "Changelog" because I am concerned that we will be able to keep a changelog written by hand if we have a lot of these contributions. I only suggest changing the name from "Changelog" to "Upgrading guide" and from "Detailed changelog" to "Changelog", because I am concerned that we will be able to maintain the changelog if we have as many contributions as we do now. Over time, it will just copy selected changes that are repeated. in detailed changelog.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r560629925



##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       In my opinion, this page has too much content and some are missing (but the missing ones do not need to be added now).  I like the README files. which is available in the documentation of Google libraries. https://googleapis.dev/python/firestore/latest/index.html
   It is standardized too, but contains all the information you need.
   
   1. Starting at the top. So I think it's worth adding a description of the package itself.  This information is available in the `provider.yaml` file, although so far the descriptions are not very. precise.h ttps://github.com/apache/airflow/blob/1602ec97c8d5bc7a7a8b42e850ac6c7a7030e47d/airflow/provider.yaml.schema.json#L13-L16
   I hope that we will be able to prepare descriptions in the future gradually, but so far I have made a minimum step and added a link to the product page for each provider.
   
   2. The next step is to link to the documentation, if we want to use the same file in setup.py and in the documentation. This will make it easier to find the documentation when you are in pypi.
   
   We have these links in the side menu, but this is quite a new feature and not everyone uses this menu. Especially on mobile devices, I start reading from the package description, not the side menu.
   ![Screenshot 2021-01-20 at 03 10 13](https://user-images.githubusercontent.com/12058428/105117662-2924b880-5acd-11eb-987b-445ed4334f9d.png)
   
   3.  The division into a detailed changelog and a normal changelog is strange for me. As for me, the contents of the ADDITIONAL_INFO.rst file should be a normal documentation page and have the title "Migration guide" because this is exactly what this file contains. As this file is fully human written, I don't think other scripts need to process it as well.
   
   4.  I think the detailed changelog and migration guides worth moving to new subpages, because it is not important for the new user.  This then will look similar to the Google documentation
   ![Screenshot 2021-01-20 at 03 16 53](https://user-images.githubusercontent.com/12058428/105118117-1959a400-5ace-11eb-8d74-1cdc3afe3680.png).
   When we move the guides to new pages, we will also be able to easily prepare the migration guide from Airflow 1.10, because we already have a table that facilitates these migrations. 
   
   ![Uploading Screenshot 2021-01-20 at 03.21.33.png…]()
   
   5. The links in the table use markdown syntax instead of rst.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-770434666


   [The Workflow run](https://github.com/apache/airflow/actions/runs/525784682) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567720055



##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -1088,6 +1078,60 @@ def get_additional_package_info(provider_package_path: str) -> str:
     return ""
 
 
+def get_changelog_for_package(provider_package_path: str) -> str:
+    """
+    Returns changelog_for the package.
+
+    :param provider_package_path: path for the package
+    :return: additional information for the path (empty string if missing)
+    """
+    changelog_path = os.path.join(provider_package_path, "CHANGELOG.rst")
+    if os.path.isfile(changelog_path):
+        with open(changelog_path) as changelog_file:
+            return changelog_file.read()
+    else:
+        print(f"[red]ERROR: Missing ${changelog_path}[/]")
+        print("Please add the file with initial content:")
+        print()
+        syntax = Syntax(

Review comment:
       Would it make sense to extract this to a constant to move it outside of the function?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567933609



##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -1225,184 +1269,308 @@ def get_provider_info_from_provider_yaml(provider_package_id: str) -> Dict[str,
     return provider_info
 
 
-def update_generated_files_for_package(
+def get_backport_current_changes_table(previous_release_commit_ref: str, source_provider_package_path: str):
+    git_cmd = get_git_log_command(HEAD_OF_HTTPS_REMOTE, previous_release_commit_ref)
+    try:
+        changes = subprocess.check_output(git_cmd, cwd=source_provider_package_path, universal_newlines=True)
+        changes_table = convert_git_changes_to_table(
+            previous_release_commit_ref, changes, base_url="https://github.com/apache/airflow/commit/"
+        )
+    except subprocess.CalledProcessError:
+        changes_table = ''
+    return changes_table
+
+
+def get_version_tag(version: str, provider_package_id: str, version_suffix: str = ''):
+    return f"providers-{provider_package_id.replace('.','-')}/{version}{version_suffix}"
+
+
+def print_changes_table(changes_table):
+    syntax = Syntax(changes_table, "rst", theme="ansi_dark")
+    console = Console(width=200)
+    console.print(syntax)
+
+
+def get_all_changes_for_regular_packages(
+    versions: List[str],
     provider_package_id: str,
-    current_release_version: str,
+    source_provider_package_path: str,
     version_suffix: str,
-    imported_classes: List[str],
-    backport_packages: bool,
-    update_release_notes: bool,
-    update_setup: bool,
-) -> Tuple[int, int]:
-    """
-    Updates release notes (BACKPORT_PROVIDER_README.md/README.md) for the package.
-    Returns Tuple of total number of entities and badly named entities.
+) -> Tuple[bool, str]:
+    current_version = versions[0]
+    current_tag_no_suffix = get_version_tag(current_version, provider_package_id)
+    print(f"Checking if tag '{current_tag_no_suffix}' exist.")
+    if not subprocess.call(
+        get_git_tag_check_command(current_tag_no_suffix),
+        cwd=source_provider_package_path,
+        stderr=subprocess.DEVNULL,
+    ):
+        print(f"The tag {current_tag_no_suffix} exists.")
+        # The tag already exists
+        changes = subprocess.check_output(
+            get_git_log_command(HEAD_OF_HTTPS_REMOTE, current_tag_no_suffix),
+            cwd=source_provider_package_path,
+            universal_newlines=True,
+        )
+        if len(changes) > 0:

Review comment:
       Comparing `len` to 0 is not pylint friendly :) 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561722394



##########
File path: airflow/providers/CHANGELOG.rst
##########
@@ -0,0 +1,25 @@
+ .. 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.
+
+
+Changelog
+---------
+
+1.0.0
+.....
+
+Initial version of the provider.

Review comment:
       Hmmmmm . Strange indeed :)




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-770292425


   [The Workflow run](https://github.com/apache/airflow/actions/runs/524131018) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-764912882


   [The Workflow run](https://github.com/apache/airflow/actions/runs/502077400) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-770276791


   [The Workflow run](https://github.com/apache/airflow/actions/runs/523950162) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r562096132



##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       I will also add a few hyperlinks here and there, so that we will be able to freely jump between those different pages - 
   * README in PyPI, 
   * index.rst in docs
   * commit changes in docs
   
   
   Those will be interlinked with version numbers so that we will be able to jump from PYPI 2.0.0 package straight to detailed commit log for 2.0.0 and 2.0.0 Changelog (and all the cross-links between those).
   
   




----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
kaxil commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-767085261


   CI is sad :(


----------------------------------------------------------------
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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r560618091



##########
File path: dev/README_RELEASE_BACKPORT_PROVIDER_PACKAGES.md
##########
@@ -0,0 +1,711 @@
+<!--
+ 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.
+-->
+<!-- START doctoc generated TOC please keep comment here to allow auto update -->
+<!-- DON'T EDIT THIS SECTION, INSTEAD RE-RUN doctoc TO UPDATE -->
+**Table of contents**
+
+- [Backport provider packages](#backport-provider-packages)
+- [Decide when to release](#decide-when-to-release)
+- [Backport provider packages versioning](#backport-provider-packages-versioning)
+- [Prepare Backport Provider Packages (RC)](#prepare-backport-provider-packages-rc)
+  - [Generate release notes](#generate-release-notes)
+  - [Build an RC release for SVN apache upload](#build-an-rc-release-for-svn-apache-upload)
+  - [Build and sign the source and convenience packages](#build-and-sign-the-source-and-convenience-packages)
+  - [Commit the source packages to Apache SVN repo](#commit-the-source-packages-to-apache-svn-repo)
+  - [Publish the RC convenience package to PyPI](#publish-the-rc-convenience-package-to-pypi)
+  - [Prepare voting email for Backport Providers release candidate](#prepare-voting-email-for-backport-providers-release-candidate)
+  - [Verify the release by PMC members](#verify-the-release-by-pmc-members)
+  - [Verify by Contributors](#verify-by-contributors)
+  - [Publish the final releases of backport packages](#publish-the-final-releases-of-backport-packages)
+  - [Update Announcements page](#update-announcements-page)
+
+<!-- END doctoc generated TOC please keep comment here to allow auto update -->
+
+------------------------------------------------------------------------------------------------------------
+
+# Backport provider packages
+
+The prerequisites to release Apache Airflow are described in [README.md](README.md).
+
+You can read more about the command line tools used to generate the packages and the two types of
+packages we have (Backport and Regular Provider Packages) in [Provider packages](PROVIDER_PACKAGES.md).
+
+# Decide when to release
+
+You can release provider packages separately from the main Airflow on an ad-hoc basis, whenever we find that
+a given provider needs to be released - due to new features or due to bug fixes.
+You can release each provider package separately, but due to voting and release overhead we try to group
+releases of provider packages together.
+
+# Backport provider packages versioning
+
+We are using the [CALVER](https://calver.org/) versioning scheme for the backport packages. We also have an
+automated way to prepare and build the packages, so it should be very easy to release the packages often and
+separately. Backport packages will be maintained for three months after 2.0.0 version of Airflow, and it is
+really a bridge, allowing people to migrate to Airflow 2.0 in stages, so the overhead of maintaining
+semver versioning does not apply there - subsequent releases might be backward-incompatible, and it is
+not indicated by the version of the packages.
+
+# Prepare Backport Provider Packages (RC)
+
+## Generate release notes
+
+Prepare release notes for all the packages you plan to release. Where YYYY.MM.DD is the CALVER
+date for the packages.
+
+```shell script
+./breeze --backports prepare-provider-documentation YYYY.MM.DD [packages]
+```
+
+If you iterate with merges and release candidates you can update the release date without providing
+the date (to update the existing release notes)
+
+```shell script
+./breeze --backports prepare-provider-documentation 2020.01.01 google
+```
+
+Generated readme files should be eventually committed to the repository.
+
+## Build an RC release for SVN apache upload
+
+The Release Candidate artifacts we vote upon should be the exact ones we vote against, without any
+modification than renaming i.e. the contents of the files must be the same between voted
+release candidate and final release. Because of this the version in the built artifacts
+that will become the official Apache releases must not include the rcN suffix. They also need
+to be signed and have checksum files. You can generate the checksum/signature files by running
+the "dev/sign.sh" script (assuming you have the right PGP key set-up for signing). The script
+generates corresponding .asc and .sha512 files for each file to sign.
+
+## Build and sign the source and convenience packages
+
+* Set environment variables (version and root of airflow repo)
+
+```shell script
+export VERSION=2020.5.20rc2
+export AIRFLOW_REPO_ROOT=$(pwd)
+
+```
+
+* Build the source package:
+
+```shell script
+${AIRFLOW_REPO_ROOT}/dev/provider_packages/build_source_package.sh --backports
+```
+
+It will generate `apache-airflow-backport-providers-${VERSION}-source.tar.gz`
+
+* Generate the packages - since we are preparing packages for SVN repo, we should use the right switch. Note
+  that this will clean up dist folder before generating the packages, so it will only contain the packages
+  you intended to build.
+
+```shell script
+./breeze --backports prepare-provider-packages --package-format both --version-suffix-for-svn rc1
+```
+
+if you ony build few packages, run:
+
+```shell script
+./breeze --backports prepare-provider-packages --package-format both --version-suffix-for-svn rc1 PACKAGE PACKAGE ....
+```
+
+* Move the source tarball to dist folder
+
+```shell script
+mv apache-airflow-backport-providers-${VERSION}-source.tar.gz dist
+```
+
+* Sign all your packages
+
+```shell script
+pushd dist
+../dev/sign.sh *
+popd
+```
+
+* Push tags to Apache repository (assuming that you have apache remote pointing to apache/airflow repo)]
+
+```shell script
+git push apache backport-providers-${VERSION}
+```
+
+## Commit the source packages to Apache SVN repo
+
+* Push the artifacts to ASF dev dist repo
+
+```shell script
+# First clone the repo if you do not have it
+svn checkout https://dist.apache.org/repos/dist/dev/airflow airflow-dev
+
+# update the repo in case you have it already
+cd airflow-dev
+svn update
+
+# Create a new folder for the release.
+cd airflow-dev/backport-providers
+svn mkdir ${VERSION}
+
+# Move the artifacts to svn folder
+mv ${AIRFLOW_REPO_ROOT}/dist/* ${VERSION}/
+
+# Add and commit
+svn add ${VERSION}/*
+svn commit -m "Add artifacts for Airflow ${VERSION}"
+
+cd ${AIRFLOW_REPO_ROOT}
+```
+
+Verify that the files are available at
+[backport-providers](https://dist.apache.org/repos/dist/dev/airflow/backport-providers/)
+
+## Publish the RC convenience package to PyPI
+
+In order to publish to PyPI you just need to build and release packages. The packages should however
+contain the rcN suffix in the version name as well, so you need to use `--version-suffix-for-pypi` switch
+to prepare those packages. Note that these are different packages than the ones used for SVN upload
+though they should be generated from the same sources.
+
+* Generate the packages with the right RC version (specify the version suffix with PyPI switch). Note that
+this will clean up dist folder before generating the packages, so you will only have the right packages there.
+
+```shell script
+./breeze --backports prepare-provider-packages --package-format both --version-suffix-for-pypi rc1
+```
+
+if you ony build few packages, run:
+
+```shell script
+./breeze --backports prepare-provider-packages --package-format both --version-suffix-for-pypi rc1 PACKAGE PACKAGE ....
+```
+
+* Verify the artifacts that would be uploaded:
+
+```shell script
+twine check dist/*
+```
+
+* Upload the package to PyPi's test environment:
+
+```shell script
+twine upload -r pypitest dist/*
+```
+
+* Verify that the test packages look good by downloading it and installing them into a virtual environment.
+Twine prints the package links as output - separately for each package.
+
+* Upload the package to PyPi's production environment:
+
+```shell script
+twine upload -r pypi dist/*
+```
+
+* Copy the list of links to the uploaded packages - they will be useful in preparing VOTE email.
+
+* Again, confirm that the packages are available under the links printed.
+
+## Prepare voting email for Backport Providers release candidate
+
+Make sure the packages are in https://dist.apache.org/repos/dist/dev/airflow/backport-providers/
+
+Send out a vote to the dev@airflow.apache.org mailing list. Here you can prepare text of the
+email using the ${VERSION} variable you already set in the command line.
+
+subject:
+
+
+```shell script
+cat <<EOF
+[VOTE] Airflow Backport Providers ${VERSION}
+EOF
+```
+
+```shell script
+cat <<EOF
+Hey all,
+
+I have cut Airflow Backport Providers ${VERSION}. This email is calling a vote on the release,
+which will last for 72 hours - which means that it will end on $(date -d '+3 days').
+
+Consider this my (binding) +1.
+
+Airflow Backport Providers ${VERSION} are available at:
+https://dist.apache.org/repos/dist/dev/airflow/backport-providers/${VERSION}/
+
+*apache-airflow-backport-providers-${VERSION}-source.tar.gz* is a source release that comes
+ with INSTALL instructions.
+
+*apache-airflow-backport-providers-<PROVIDER>-${VERSION}-bin.tar.gz* are the binary
+ Python "sdist" release.
+
+The test procedure for PMCs and Contributors who would like to test the RC candidates are described in
+https://github.com/apache/airflow/blob/master/dev/README.md#vote-and-verify-the-backport-providers-release-candidate
+
+
+Public keys are available at:
+https://dist.apache.org/repos/dist/release/airflow/KEYS
+
+Please vote accordingly:
+
+[ ] +1 approve
+[ ] +0 no opinion
+[ ] -1 disapprove with the reason
+
+
+Only votes from PMC members are binding, but members of the community are
+encouraged to test the release and vote with "(non-binding)".
+
+Please note that the version number excludes the 'rcX' string, so it's now
+simply ${VERSION%rc?}. This will allow us to rename the artifact without modifying
+the artifact checksums when we actually release.
+
+Each of the packages contains detailed changelog. Here is the list of links to
+the released packages and changelogs:
+
+<PASTE TWINE UPLOAD LINKS HERE. SORT THEM BEFORE!>
+
+Cheers,
+<TODO: Your Name>
+
+EOF
+```
+
+Due to the nature of backport packages, not all packages have to be released as convenience
+packages in the final release. During the voting process
+the voting PMCs might decide to exclude certain packages from the release if some critical
+problems have been found in some packages.
+
+Please modify the message above accordingly to clearly exclude those packages.
+
+## Verify the release by PMC members
+
+### SVN check
+
+The files should be present in the sub-folder of
+[Airflow dist](https://dist.apache.org/repos/dist/dev/airflow/backport-providers/)
+
+The following files should be present (9 files):
+
+* -source.tar.gz + .asc + .sha512 (one set of files)
+* -bin-tar.gz + .asc + .sha512 (one set of files per provider)
+* -.whl + .asc + .sha512 (one set of files per provider)
+
+As a PMC you should be able to clone the SVN repository:
+
+```shell script
+svn co https://dist.apache.org/repos/dist/dev/airflow/
+```
+
+Or update it if you already checked it out:
+
+```shell script
+svn update .
+```
+
+### Licences check
+
+This can be done with the Apache RAT tool.
+
+* Download the latest jar from https://creadur.apache.org/rat/download_rat.cgi (unpack the sources,
+  the jar is inside)
+* Unpack the -source.tar.gz to a folder
+* Enter the folder and run the check (point to the place where you extracted the .jar)
+
+```shell script
+java -jar ../../apache-rat-0.13/apache-rat-0.13.jar -E .rat-excludes -d .
+```
+
+### Signature check
+
+Make sure you have the key of person signed imported in your GPG. You can find the valid keys in
+[KEYS](https://dist.apache.org/repos/dist/release/airflow/KEYS).
+
+You can import the whole KEYS file:
+
+```shell script
+gpg --import KEYS
+```
+
+You can also import the keys individually from a keyserver. The below one uses Kaxil's key and
+retrieves it from the default GPG keyserver
+[OpenPGP.org](https://keys.openpgp.org):
+
+```shell script
+gpg --receive-keys 12717556040EEF2EEAF1B9C275FCCD0A25FA0E4B
+```
+
+You should choose to import the key when asked.
+
+Note that by being default, the OpenPGP server tends to be overloaded often and might respond with
+errors or timeouts. Many of the release managers also uploaded their keys to the
+[GNUPG.net](https://keys.gnupg.net) keyserver, and you can retrieve it from there.
+
+```shell script
+gpg --keyserver keys.gnupg.net --receive-keys 12717556040EEF2EEAF1B9C275FCCD0A25FA0E4B
+```
+
+Once you have the keys, the signatures can be verified by running this:
+
+```shell script
+for i in *.asc
+do
+   echo "Checking $i"; gpg --verify $i
+done
+```
+
+This should produce results similar to the below. The "Good signature from ..." is indication
+that the signatures are correct. Do not worry about the "not certified with a trusted signature"
+warning. Most of the certificates used by release managers are self signed, that's why you get this
+warning. By importing the server in the previous step and importing it via ID from
+[KEYS](https://dist.apache.org/repos/dist/release/airflow/KEYS) page, you know that
+this is a valid Key already.
+
+```
+Checking apache-airflow-1.10.12rc4-bin.tar.gz.asc
+gpg: assuming signed data in 'apache-airflow-1.10.12rc4-bin.tar.gz'
+gpg: Signature made sob, 22 sie 2020, 20:28:28 CEST
+gpg:                using RSA key 12717556040EEF2EEAF1B9C275FCCD0A25FA0E4B
+gpg: Good signature from "Kaxil Naik <ka...@gmail.com>" [unknown]
+gpg: WARNING: This key is not certified with a trusted signature!
+gpg:          There is no indication that the signature belongs to the owner.
+Primary key fingerprint: 1271 7556 040E EF2E EAF1  B9C2 75FC CD0A 25FA 0E4B
+Checking apache_airflow-1.10.12rc4-py2.py3-none-any.whl.asc
+gpg: assuming signed data in 'apache_airflow-1.10.12rc4-py2.py3-none-any.whl'
+gpg: Signature made sob, 22 sie 2020, 20:28:31 CEST
+gpg:                using RSA key 12717556040EEF2EEAF1B9C275FCCD0A25FA0E4B
+gpg: Good signature from "Kaxil Naik <ka...@gmail.com>" [unknown]
+gpg: WARNING: This key is not certified with a trusted signature!
+gpg:          There is no indication that the signature belongs to the owner.
+Primary key fingerprint: 1271 7556 040E EF2E EAF1  B9C2 75FC CD0A 25FA 0E4B
+Checking apache-airflow-1.10.12rc4-source.tar.gz.asc
+gpg: assuming signed data in 'apache-airflow-1.10.12rc4-source.tar.gz'
+gpg: Signature made sob, 22 sie 2020, 20:28:25 CEST
+gpg:                using RSA key 12717556040EEF2EEAF1B9C275FCCD0A25FA0E4B
+gpg: Good signature from "Kaxil Naik <ka...@gmail.com>" [unknown]
+gpg: WARNING: This key is not certified with a trusted signature!
+gpg:          There is no indication that the signature belongs to the owner.
+Primary key fingerprint: 1271 7556 040E EF2E EAF1  B9C2 75FC CD0A 25FA 0E4B
+```
+
+### SHA512 check
+
+Run this:
+
+```shell script
+for i in *.sha512
+do
+    echo "Checking $i"; shasum -a 512 `basename $i .sha512 ` | diff - $i
+done
+```
+
+You should get output similar to:
+
+```
+Checking apache-airflow-1.10.12rc4-bin.tar.gz.sha512
+Checking apache_airflow-1.10.12rc4-py2.py3-none-any.whl.sha512
+Checking apache-airflow-1.10.12rc4-source.tar.gz.sha512
+```
+
+## Verify by Contributors
+
+This can be done (and we encourage to) by any of the Contributors. In fact, it's best if the
+actual users of Apache Airflow test it in their own staging/test installations. Each release candidate
+is available on PyPI apart from SVN packages, so everyone should be able to install
+the release candidate version of Airflow via simply (<VERSION> is 1.10.12 for example, and <X> is
+release candidate number 1,2,3,....).
+
+You can use any of the installation methods you prefer (you can even install it via the binary wheels
+downloaded from the SVN).
+
+### Installing in your local virtualenv
+
+You have to make sure you have Airflow 1.10.* installed in your PIP virtualenv
+(the version you want to install providers with).
+
+```shell script
+pip install apache-airflow-backport-providers-<provider>==<VERSION>rc<X>
+```
+
+### Installing with Breeze
+
+There is also an easy way of installation with Breeze if you have the latest sources of Apache Airflow.
+Here is a typical scenario.
+
+First copy all the provider packages .whl files to the `dist` folder.
+
+```shell script
+./breeze start-airflow --install-airflow-version <VERSION>rc<X> \
+    --python 3.7 --backend postgres --install-packages-from-dist
+```
+
+For 1.10 releases you can also use `--no-rbac-ui` flag disable RBAC UI of Airflow:
+
+```shell script
+./breeze start-airflow --install-airflow-version <VERSION>rc<X> \
+    --python 3.7 --backend postgres --install-packages-from-dist --no-rbac-ui
+```
+
+### Building your own docker image
+
+If you prefer to build your own image, you can also use the official image andipi PyPI packages to test

Review comment:
       ```suggestion
   If you prefer to build your own image, you can also use the official image and PyPI packages to test
   ```




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561811331



##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       Also one small comment here.
   
   Looking at https://apache-airflow.slack.com/archives/CCR6P6JRL/p1611225089006700 - this is why I think we should have the detailed commit log (as a separate document linked from the index). Users are sometimes aware of the PRs they are interested in and want to know in which version they were included. And it's not very easy to find it out. We will have a separate set of tags for each provider, so for some people, it would be great to have it in this nicely formatted document rather than having to run some complex git commands to find out. 
   
   




----------------------------------------------------------------
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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561478559



##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       > (happy to name it differently if someone has a good proposal).
   
   Alternatively, we can call it "commits"

##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       > (happy to name it differently if someone has a good proposal).
   
   Alternatively, we can call it "Commits"




----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
kaxil commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-763181343


   will take a look tomorrow


----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561413886



##########
File path: airflow/providers/amazon/README.md
##########
@@ -1,483 +0,0 @@
-<!--
- 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.
- -->
-
-
-# Package apache-airflow-providers-amazon

Review comment:
       We don't want this Readme, but I thought we will generate the new readme -- is that only during release ? Would be good to have it here




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r560756343



##########
File path: dev/provider_packages/copy_provider_package_sources.py
##########
@@ -767,27 +781,38 @@ def do_refactor(self, in_process: bool = False) -> None:  # noqa
         self.refactor_kubernetes_pod_operator()
         # In order to debug Bowler - set in_process to True
         self.qry.execute(write=True, silent=False, interactive=False, in_process=in_process)
+        print("[green]OK[/]")
+
+
+def get_parser():
+    cli_parser = argparse.ArgumentParser(
+        description="Copies sources and optionally refactors provider code to be Airflow 1.10 compatible.",
+        formatter_class=argparse.RawTextHelpFormatter,
+    )
+    cli_parser.add_argument(
+        "--backports",
+        action='store_true',
+        help=textwrap.dedent("Includes refactoring to prepare backport packages rather than regular ones"),
+    )
+    cli_parser.add_argument(
+        "--debug",
+        action='store_true',
+        help=textwrap.dedent(
+            "Run bowler refactoring in single process. Makes it debuggable with regular"
+            " IDE debugger (much slower)"
+        ),
+    )
+    return cli_parser
 
 
 if __name__ == '__main__':
-    BACKPORT_PACKAGES = os.getenv('BACKPORT_PACKAGES') == "true"
-    in_process = False
-    if len(sys.argv) > 1:
-        if sys.argv[1] in ['--help', '-h']:
-            print()
-            print("Refactors provider packages to be Airflow 1.10 compatible.")
-            print()
-            print(f"Usage: {sys.argv[0]} [--debug] | [-h] | [--help]")
-            print()
-            print("You can use --debug flag in order to run bowler refactoring in process.")
-            print("This allows you to debug bowler process as usual using your IDE debugger")
-            print("Otherwise it heavily uses multi-processing and is next-to-impossible to debug")
-            print()
-            print("Note - Bowler is also a lot slower in this mode.")
-            print()
-            sys.exit(0)
-        if sys.argv[1] == '--debug':
-            in_process = True
-    copy_provider_sources()
-    if BACKPORT_PACKAGES:
+    parser = get_parser()
+    args = parser.parse_args()
+
+    in_process = args.debug
+    copy_provider_sources(args.backports)
+    os.chdir(get_target_providers_folder())
+    if args.backports:
+        print("\nRefactoring code to be Airflow 1.10 - compatible\n")

Review comment:
       Not sure but maybe we may use `end='\n\n'` instead of new lines in string?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567717505



##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -830,87 +859,17 @@ def convert_cross_package_dependencies_to_table(
 )
 
 
-def strip_leading_zeros_in_calver(calver_version: str) -> str:
+def strip_leading_zeros(version: str) -> str:
     """
-    Strips leading zeros from calver version number.
+    Strips leading zeros from version number.
 
     This converts 1974.04.03 to 1974.4.3 as the format with leading month and day zeros is not accepted
     by PIP versioning.
 
-    :param calver_version: version number in calver format (potentially with leading 0s in date and month)
+    :param version: version number in calver format (potentially with leading 0s in date and month)
     :return: string with leading 0s after dot replaced.
     """
-    return calver_version.replace(".0", ".")
-
-
-def get_provider_changes_prefix(backport_packages: bool) -> str:
-    """
-    Returns prefix for provider CHANGES files.
-    """
-    if backport_packages:
-        return "BACKPORT_PROVIDER_CHANGES_"
-    else:
-        return "PROVIDER_CHANGES_"
-
-
-def get_all_releases(provider_package_path: str, backport_packages: bool) -> List[ReleaseInfo]:
-    """
-    Returns information about past releases (retrieved from *changes_*md files stored in the
-    package folder.
-    :param provider_package_path: path of the package
-    :param backport_packages: whether to prepare regular (False) or backport (True) packages
-    :return: list of releases made so far.
-    """
-    changes_file_prefix = get_provider_changes_prefix(backport_packages=backport_packages)
-    past_releases: List[ReleaseInfo] = []
-    changes_file_names = listdir(provider_package_path)
-    for file_name in sorted(changes_file_names, reverse=True):
-        if file_name.startswith(changes_file_prefix) and file_name.endswith(".md"):
-            changes_file_path = os.path.join(provider_package_path, file_name)
-            with open(changes_file_path) as changes_file:
-                content = changes_file.read()
-            found = re.search(r'/([a-z0-9]*)\)', content, flags=re.MULTILINE)
-            if not found:
-                print("[yellow]No commit found. This seems to be first time you run it[/]", file=sys.stderr)
-            else:
-                last_commit_hash = found.group(1)
-                release_version = file_name[len(changes_file_prefix) :][:-3]
-                release_version_no_leading_zeros = (
-                    strip_leading_zeros_in_calver(release_version) if backport_packages else release_version
-                )
-                past_releases.append(
-                    ReleaseInfo(
-                        release_version=release_version,
-                        release_version_no_leading_zeros=release_version_no_leading_zeros,
-                        last_commit_hash=last_commit_hash,
-                        content=content,
-                        file_name=file_name,
-                    )
-                )
-    return past_releases
-
-
-def get_latest_release(provider_package_path: str, backport_packages: bool) -> ReleaseInfo:
-    """
-    Gets information about the latest release.
-
-    :param provider_package_path: path of package
-    :param backport_packages: whether to prepare regular (False) or backport (True) packages
-    :return: latest release information
-    """
-    releases = get_all_releases(
-        provider_package_path=provider_package_path, backport_packages=backport_packages
-    )
-    if len(releases) == 0:
-        return ReleaseInfo(
-            release_version="0.0.0",
-            release_version_no_leading_zeros="0.0.0",
-            last_commit_hash="no_hash",
-            content="empty",
-            file_name="no_file",
-        )
-    else:
-        return releases[0]
+    return ".".join(str(int(i)) for i in version.split("."))

Review comment:
       Is there need to cast to `int` if we then cast to `str`?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561395893



##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       > human description on what changed in each version. Which I think CHANGELOG is a great name for.
   
   I am afraid nobody will write it by hand. Google automatically generates CHANGELOG.md files. Everyone has similar references to Github. 
   https://github.com/googleapis/python-bigquery/blob/master/CHANGELOG.md
   https://github.com/googleapis/python-bigquery/commit/b0e074f7522710886be1da2f117ea22de411b408
   To generate the changelog, they use the[googleapis/release-please](https://github.com/googleapis/python-bigquery/commit/b0e074f7522710886be1da2f117ea22de411b408) tool, which analyzes the message and generates a changelog based on it.
   
   > Release Please assumes you are using Conventional Commit messages.
   > 
   > The most important prefixes you should have in mind are:
   > 
   > fix: which represents bug fixes, and correlates to a SemVer patch.
   > feat: which represents a new feature, and correlates to a SemVer minor.
   > feat!:, or fix!:, refactor!:, etc., which represent a breaking change (indicated by the !) and will result in a SemVer major.
   
   In the first version, I think that we don't need the changes to be divided into several categories, although that would be super functional, we can limit ourselves to listing the changes. If we publish these packages regularly, each release will not have a very long list of changes.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r562039147



##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       Oh we should definitely have a Changelog which contains the commit log whether it is automated or done manually




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r560993238



##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       > In my opinion, this page has too much content and some are missing (but the missing ones do not need to be added now). I like the README files. which is available in the documentation of Google libraries. https://googleapis.dev/python/firestore/latest/index.html
   > It is standardized too, but contains all the information you need.
   > 
   > 1. Starting at the top. So I think it's worth adding a description of the package itself.  This information is available in the `provider.yaml` file, although so far the descriptions are not very. precise.h ttps://github.com/apache/airflow/blob/1602ec97c8d5bc7a7a8b42e850ac6c7a7030e47d/airflow/provider.yaml.schema.json#L13-L16
   
   Sure. Happy to add it. This will be easy to incorporate.
   
   > 2. The next step is to link to the documentation, if we want to use the same file in setup.py and in the documentation. This will make it easier to find the documentation when you are in pypi.
   
   I thiink the PyPI and documentation should be different than documentation. PyPI should be more focused on how to install packages and what are the limitations/requirements, but the "full" documentation" should be reached (including the ChangeLog which I think should not be part of README as it will grow over tim). This is how I approached it and I think this is a better idea than to keep the same documentation index and README.
   
   > We have these links in the side menu, but this is quite a new feature and not everyone uses this menu. Especially on mobile devices, I start reading from the package description, not the side menu.
   
   Yeah. I will modify setup.cfg to include few more links (Documentation especially).
    
   > 1. The division into a detailed changelog and a normal changelog is strange for me. As for me, the contents of the ADDITIONAL_INFO.rst file should be a normal documentation page and have the title "Migration guide" because this is exactly what this file contains. As this file is fully human written, I don't think other scripts need to process it as well.
   
   ADDITIONAL_INFO is for backport providers. IMHO this README is something that will evolve evertime. ADDITIONAL_INFO was more about migrating from 1.0 to backports. Effectively CHANGELOG should be written by human as a summary of changes in each version. The assumption is that there will be very rare "migration needed" for those.  Google provider is a bit different than the others, because we migrate python APIs. But in most other packages this will not be a migration guide but human description on what changed in each version. Which I think CHANGELOG is a great name for.
   
   > 2. I think the detailed changelog and migration guides worth moving to new subpages, because it is not important for the new user.  This then will look similar to the Google documentation
   
   Sure. The Detailed guide can be added as a separate page in the docs and linked to. I thought about it as well. Happy to hear why others think but I think we should have this:
   
   * README.md in PyPI describing gerneral "what it is about" + all things important to installing PyPI + link to the Provider "index" documentation
   
   * Provider's index documentation with the same information added + link to detailed commit changelog. It might contain more information (but each Provider is free to add it in the index.rst above the automatically generated part). I do not want to discuss "standard" for this documentation here - we can discuss it separately. In this  change I only want to focus on automatically generated part.
   
   * Detailed changelog - this might be separate page (happy to name it differently if someone has a good proposal).
   
   >    When we move the guides to new pages, we will also be able to easily prepare the migration guide from Airflow 1.10, because we already have a table that facilitates these migrations.
    
   We already have it in Backport providers. I think Provider's documentation should be more "forward" looking rather than focuing in the migration from 1.10. 
   We might instead point out to backport's PYPI readme and mention that for migration from 1.10 people 
   
   > 1. The links in the table use markdown syntax instead of rst.
   
   I think I fixed that, but I can take a look. 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-766452624


   [The Workflow run](https://github.com/apache/airflow/actions/runs/507984080) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567775848



##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -1225,184 +1269,308 @@ def get_provider_info_from_provider_yaml(provider_package_id: str) -> Dict[str,
     return provider_info
 
 
-def update_generated_files_for_package(
+def get_backport_current_changes_table(previous_release_commit_ref: str, source_provider_package_path: str):
+    git_cmd = get_git_log_command(HEAD_OF_HTTPS_REMOTE, previous_release_commit_ref)
+    try:
+        changes = subprocess.check_output(git_cmd, cwd=source_provider_package_path, universal_newlines=True)
+        changes_table = convert_git_changes_to_table(
+            previous_release_commit_ref, changes, base_url="https://github.com/apache/airflow/commit/"
+        )
+    except subprocess.CalledProcessError:
+        changes_table = ''
+    return changes_table
+
+
+def get_version_tag(version: str, provider_package_id: str, version_suffix: str = ''):
+    return f"providers-{provider_package_id.replace('.','-')}/{version}{version_suffix}"
+
+
+def print_changes_table(changes_table):
+    syntax = Syntax(changes_table, "rst", theme="ansi_dark")
+    console = Console(width=200)
+    console.print(syntax)
+
+
+def get_all_changes_for_regular_packages(
+    versions: List[str],
     provider_package_id: str,
-    current_release_version: str,
+    source_provider_package_path: str,
     version_suffix: str,
-    imported_classes: List[str],
-    backport_packages: bool,
-    update_release_notes: bool,
-    update_setup: bool,
-) -> Tuple[int, int]:
-    """
-    Updates release notes (BACKPORT_PROVIDER_README.md/README.md) for the package.
-    Returns Tuple of total number of entities and badly named entities.
+) -> Tuple[bool, str]:
+    current_version = versions[0]
+    current_tag_no_suffix = get_version_tag(current_version, provider_package_id)
+    print(f"Checking if tag '{current_tag_no_suffix}' exist.")
+    if not subprocess.call(
+        get_git_tag_check_command(current_tag_no_suffix),
+        cwd=source_provider_package_path,
+        stderr=subprocess.DEVNULL,
+    ):
+        print(f"The tag {current_tag_no_suffix} exists.")
+        # The tag already exists
+        changes = subprocess.check_output(
+            get_git_log_command(HEAD_OF_HTTPS_REMOTE, current_tag_no_suffix),
+            cwd=source_provider_package_path,
+            universal_newlines=True,
+        )
+        if len(changes) > 0:

Review comment:
       Fine for me :) . I often prefer explicit len() check but in this case it makes perfect sense.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-770369444


   [The Workflow run](https://github.com/apache/airflow/actions/runs/525146210) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-767154886


   Yeah. I have to test it in "master' of my fork as it requires changes in "buid image workflow" . Once I get it finallzed (I hope tomorrow) I will link to successful build there :(


----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567891751



##########
File path: airflow/providers/res.txt
##########
@@ -0,0 +1,61 @@
+./salesforce
+./celery
+./amazon
+./papermill

Review comment:
       aaaaaaaa




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567718220



##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -977,70 +933,94 @@ def get_cross_provider_dependent_packages(provider_package_id: str) -> List[str]
 
 def make_sure_remote_apache_exists_and_fetch():
     """
-    Make sure that apache remote exist in git. We need to take a log from the master of apache
-    repository - not locally - because when we commit this change and run it, our log will include the
-    current commit - which is going to have different commit id once we merge. So it is a bit
-    catch-22.
+    Make sure that apache remote exist in git. We need to take a log from the apache
+    repository - not locally.
+
+    Also the the local repo might be shallow so we need to unshallow it.
+
+    This will:
+    * check if the remote exists and add if it does not
+    * check if the local repo is shallow, markit to be unshallowed in this case
+    * fetch from the remote including all tags and overriding local taags in case they are set differently
 
-    :return:
     """
     try:
+        check_remote_command = ["git", "remote", "get-url", HTTPS_REMOTE]
         subprocess.check_call(
-            ["git", "remote", "add", "apache-https-for-providers", "https://github.com/apache/airflow.git"],
+            check_remote_command,
             stdout=subprocess.DEVNULL,
             stderr=subprocess.DEVNULL,
         )
-    except subprocess.CalledProcessError as e:
-        if e.returncode == 128:
-            print(
-                "[yellow]The remote `apache-https-for-providers` already exists. If you have trouble running "
-                "git log delete the remote[/]",
-                file=sys.stderr,
-            )
+        print(f"Running command: '{' '.join(check_remote_command)}'")

Review comment:
       Should this print be before `check_call`?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-770422063


   [The Workflow run](https://github.com/apache/airflow/actions/runs/525666493) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567889793



##########
File path: airflow/providers/res.txt
##########
@@ -0,0 +1,61 @@
+./salesforce
+./celery
+./amazon
+./papermill

Review comment:
       ping @potiuk ^^ 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561721706



##########
File path: BREEZE.rst
##########
@@ -2056,6 +2065,28 @@ This is the current syntax for  `./breeze <./breeze>`_:
 
   Flags:
 
+  -S, --version-suffix-for-pypi SUFFIX
+          Adds optional suffix to the version in the generated backport package. It can be used

Review comment:
       For regular providers too




----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
kaxil commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-767085261


   CI is sad :(


----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561725620



##########
File path: airflow/providers/res.txt
##########
@@ -0,0 +1,61 @@
+./salesforce
+./celery
+./amazon

Review comment:
       Debug file :)




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561726275



##########
File path: dev/provider_packages/BACKPORT_PROVIDER_README_TEMPLATE.md.jinja2
##########
@@ -88,6 +95,6 @@ You can install such cross-provider dependencies when installing from PyPI. For
 pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
 ```
 
-{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE }}
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE | safe }}

Review comment:
       Avoids escaping < > into `&gt;` `&lt;`




----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561398276



##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       I like what you have suggested here Jarek.
   
   - Concise README
   - Link to commit changelog
   - detailed changelog is the Updating guide right?

##########
File path: BREEZE.rst
##########
@@ -2056,6 +2065,28 @@ This is the current syntax for  `./breeze <./breeze>`_:
 
   Flags:
 
+  -S, --version-suffix-for-pypi SUFFIX
+          Adds optional suffix to the version in the generated backport package. It can be used

Review comment:
       Is this only for backports or regular providers too?

##########
File path: BREEZE.rst
##########
@@ -2056,6 +2065,28 @@ This is the current syntax for  `./breeze <./breeze>`_:
 
   Flags:
 
+  -S, --version-suffix-for-pypi SUFFIX
+          Adds optional suffix to the version in the generated backport package. It can be used
+          to generate rc1/rc2 ... versions of the packages to be uploaded to PyPI.
+
+  -N, --version-suffix-for-svn SUFFIX
+          Adds optional suffix to the generated names of package. It can be used to generate
+          rc1/rc2 ... versions of the packages to be uploaded to SVN.
+
+  --package-format PACKAGE_FORMAT
+
+          Chooses format of packages to prepare.
+
+          One of:
+
+                 wheel,sdist,both
+
+          Default: 

Review comment:
       is there NO default?

##########
File path: airflow/providers/CHANGELOG.rst
##########
@@ -0,0 +1,25 @@
+ .. 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.
+
+
+Changelog
+---------
+
+1.0.0
+.....
+
+Initial version of the provider.

Review comment:
       Don't  think this is intended -- we don't want to have Changelog.rst in `airflow/providers`

##########
File path: airflow/providers/README.md
##########
@@ -1,28 +0,0 @@
-<!--
- 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.
- -->
-
-# Airflow Providers

Review comment:
       is this deletion intentional?

##########
File path: airflow/providers/amazon/README.md
##########
@@ -1,483 +0,0 @@
-<!--
- 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.
- -->
-
-
-# Package apache-airflow-providers-amazon

Review comment:
       We don't want this Readme, but I thought we will generate the new readme -- is that only during release ? Would be good to have it here

##########
File path: airflow/providers/res.txt
##########
@@ -0,0 +1,61 @@
+./salesforce
+./celery
+./amazon

Review comment:
       yeah not sure what this file is about

##########
File path: dev/provider_packages/BACKPORT_PROVIDER_README_TEMPLATE.md.jinja2
##########
@@ -88,6 +95,6 @@ You can install such cross-provider dependencies when installing from PyPI. For
 pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
 ```
 
-{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE }}
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE | safe }}

Review comment:
       what's the reason for `| safe`

##########
File path: dev/provider_packages/copy_provider_package_sources.py
##########
@@ -767,27 +781,38 @@ def do_refactor(self, in_process: bool = False) -> None:  # noqa
         self.refactor_kubernetes_pod_operator()
         # In order to debug Bowler - set in_process to True
         self.qry.execute(write=True, silent=False, interactive=False, in_process=in_process)
+        print("[green]OK[/]")
+
+
+def get_parser():
+    cli_parser = argparse.ArgumentParser(
+        description="Copies sources and optionally refactors provider code to be Airflow 1.10 compatible.",
+        formatter_class=argparse.RawTextHelpFormatter,
+    )
+    cli_parser.add_argument(
+        "--backports",
+        action='store_true',
+        help=textwrap.dedent("Includes refactoring to prepare backport packages rather than regular ones"),
+    )
+    cli_parser.add_argument(
+        "--debug",
+        action='store_true',
+        help=textwrap.dedent(
+            "Run bowler refactoring in single process. Makes it debuggable with regular"
+            " IDE debugger (much slower)"
+        ),
+    )
+    return cli_parser
 
 
 if __name__ == '__main__':
-    BACKPORT_PACKAGES = os.getenv('BACKPORT_PACKAGES') == "true"
-    in_process = False
-    if len(sys.argv) > 1:
-        if sys.argv[1] in ['--help', '-h']:
-            print()
-            print("Refactors provider packages to be Airflow 1.10 compatible.")
-            print()
-            print(f"Usage: {sys.argv[0]} [--debug] | [-h] | [--help]")
-            print()
-            print("You can use --debug flag in order to run bowler refactoring in process.")
-            print("This allows you to debug bowler process as usual using your IDE debugger")
-            print("Otherwise it heavily uses multi-processing and is next-to-impossible to debug")
-            print()
-            print("Note - Bowler is also a lot slower in this mode.")
-            print()
-            sys.exit(0)
-        if sys.argv[1] == '--debug':
-            in_process = True
-    copy_provider_sources()
-    if BACKPORT_PACKAGES:
+    parser = get_parser()
+    args = parser.parse_args()
+
+    in_process = args.debug
+    copy_provider_sources(args.backports)
+    os.chdir(get_target_providers_folder())
+    if args.backports:
+        print("\nRefactoring code to be Airflow 1.10 - compatible\n")
         RefactorBackportPackages().do_refactor(in_process=in_process)
+        print("\n[green]Refactored code successfully[/]\n")

Review comment:
       ```suggestion
       copy_provider_sources(args.backports)
       os.chdir(get_target_providers_folder())
       if args.backports:
           print("\nRefactoring code to be Airflow 1.10 - compatible\n")
           RefactorBackportPackages().do_refactor(in_process=args.debug)
           print("\n[green]Refactored code successfully[/]\n")
   ```

##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -1225,184 +1195,272 @@ def get_provider_info_from_provider_yaml(provider_package_id: str) -> Dict[str,
     return provider_info
 
 
-def update_generated_files_for_package(
+def get_backport_current_changes_table(previous_release_commit_ref: str, source_provider_package_path: str):
+    git_cmd = get_git_log_command(previous_release_commit_ref)
+    try:
+        changes = subprocess.check_output(git_cmd, cwd=source_provider_package_path, universal_newlines=True)
+        changes_table = convert_git_changes_to_table(
+            previous_release_commit_ref, changes, base_url="https://github.com/apache/airflow/commit/"
+        )
+    except subprocess.CalledProcessError:
+        changes_table = ''
+    return changes_table
+
+
+def get_version_tag(version: str, provider_package_id: str, version_suffix: str):
+    return f"providers-{provider_package_id.replace('.','-')}/{version}{version_suffix}"
+
+
+def print_changes_table(changes_table):
+    syntax = Syntax(changes_table, "rst", theme="light")
+    console = Console(width=200)
+    console.print(syntax)
+
+
+def get_all_changes_for_regular_packages(
+    versions: List[str],
     provider_package_id: str,
-    current_release_version: str,
+    source_provider_package_path: str,
     version_suffix: str,
-    imported_classes: List[str],
-    backport_packages: bool,
-    update_release_notes: bool,
-    update_setup: bool,
-) -> Tuple[int, int]:
-    """
-    Updates release notes (BACKPORT_PROVIDER_README.md/README.md) for the package.
-    Returns Tuple of total number of entities and badly named entities.
+) -> Tuple[bool, str]:
+    current_version = versions[0]
+    current_tag = get_version_tag(current_version, provider_package_id, version_suffix)
+    print(f"Checking if tag `{current_tag}` exists.")
+    if not subprocess.call(
+        get_git_tag_check_command(current_tag),
+        cwd=source_provider_package_path,
+        stderr=subprocess.DEVNULL,
+    ):
+        print(f"The tag {current_tag} exists.")
+        # The tag already exists
+        changes = subprocess.check_output(
+            get_git_log_command(current_tag, 'HEAD'),
+            cwd=source_provider_package_path,
+            universal_newlines=True,
+        )
+        if len(changes) > 0:
+            print(
+                f"[yellow]The provider {provider_package_id} has changes"
+                f" since last release but version is not updated[/]"
+            )
+            print()
+            print(
+                f'Please update version in `airflow/providers/{provider_package_id.replace("-","/")}/'
+                'provider.yaml` to prepare release.\n'
+            )
+            changes_table = convert_git_changes_to_table(
+                "UNKNOWN", changes, base_url="https://github.com/apache/airflow/commit/", markdown=False
+            )
+            print_changes_table(changes_table)
+            return False, changes_table
+        else:
+            print(f"No changes for {provider_package_id}")
+            return False, ""
+    print("The tag does not exist. ")
+    if len(versions) == 1:
+        print("This provider has never been released!\n")
+    else:
+        print(f"The {provider_package_id} is ready to be released!\n")
+    current_version = 'HEAD'
+    changes_table = ''
+    print_version = versions[0]
+    for version in versions[1:]:
+        version_tag = get_version_tag(version, provider_package_id, version_suffix)
+        changes = subprocess.check_output(
+            get_git_log_command(version_tag, current_version),
+            cwd=source_provider_package_path,
+            universal_newlines=True,
+        )
+        changes_table += convert_git_changes_to_table(
+            print_version, changes, base_url="https://github.com/apache/airflow/commit/", markdown=False
+        )
+        current_version = version
+        print_version = version
+    print_changes_table(changes_table)
+    return True, changes_table
+
+
+def get_provider_details(provider_package_id: str) -> ProviderPackageDetails:
+    return ProviderPackageDetails(
+        provider_package_id=provider_package_id,
+        full_package_name=f"airflow.providers.{provider_package_id}",
+        source_provider_package_path=get_source_package_path(provider_package_id),
+        documentation_provider_package_path=get_documentation_package_path(provider_package_id),
+    )
 
-    :param provider_package_id: id of the package
-    :param current_release_version: release version:
-    :param version_suffix: version suffix corresponding to the version in the code
-    :param imported_classes - entities that have been imported from providers
-    :param backport_packages: whether to prepare regular (False) or backport (True) packages
-    :param update_release_notes: whether to update release notes
-    :param update_setup: whether to update setup files
 
-    :return: Tuple of total/bad number of entities
-    """
-    verify_provider_package(provider_package_id)
-    full_package_name = f"airflow.providers.{provider_package_id}"
-    source_provider_package_path = get_source_package_path(provider_package_id)
-    entity_summaries = get_package_class_summary(full_package_name, imported_classes)
-    past_releases = get_all_releases(
-        provider_package_path=source_provider_package_path, backport_packages=backport_packages
+def get_provider_jinja_context(
+    provider_details: ProviderPackageDetails,
+    current_release_version: str,
+    version_suffix: str,
+    backport_packages: bool,
+):
+    verify_provider_package(provider_details.provider_package_id)
+    cross_providers_dependencies = get_cross_provider_dependent_packages(
+        provider_package_id=provider_details.provider_package_id
     )
-    current_release_version, previous_release = check_if_release_version_ok(
-        past_releases, current_release_version, backport_packages
+    release_version_no_leading_zeros = strip_leading_zeros(current_release_version)
+    pip_requirements_table = convert_pip_requirements_to_table(
+        PROVIDERS_REQUIREMENTS[provider_details.provider_package_id]
     )
-    cross_providers_dependencies = get_cross_provider_dependent_packages(
-        provider_package_id=provider_package_id
+    pip_requirements_table_rst = convert_pip_requirements_to_table(
+        PROVIDERS_REQUIREMENTS[provider_details.provider_package_id], markdown=False
     )
-    previous_release = get_previous_release_info(
-        previous_release_version=previous_release,
-        past_releases=past_releases,
-        current_release_version=current_release_version,
+    cross_providers_dependencies_table = convert_cross_package_dependencies_to_table(
+        cross_providers_dependencies,
+        backport_packages=backport_packages,
     )
-    release_version_no_leading_zeros = (
-        strip_leading_zeros_in_calver(current_release_version)
-        if backport_packages
-        else current_release_version
+    cross_providers_dependencies_table_rst = convert_cross_package_dependencies_to_table(
+        cross_providers_dependencies, backport_packages=backport_packages, markdown=False
     )
     context: Dict[str, Any] = {
         "ENTITY_TYPES": list(EntityType),
         "README_FILE": "BACKPORT_PROVIDER_README.md" if backport_packages else "README.md",
-        "PROVIDER_PACKAGE_ID": provider_package_id,
-        "PACKAGE_PIP_NAME": get_pip_package_name(provider_package_id, backport_packages),
-        "FULL_PACKAGE_NAME": full_package_name,
-        "PROVIDER_PATH": full_package_name.replace(".", "/"),
+        "PROVIDER_PACKAGE_ID": provider_details.provider_package_id,
+        "PACKAGE_PIP_NAME": get_pip_package_name(
+            provider_details.provider_package_id, backport_packages=backport_packages
+        ),
+        "FULL_PACKAGE_NAME": provider_details.full_package_name,
+        "PROVIDER_PATH": provider_details.full_package_name.replace(".", "/"),
         "RELEASE": current_release_version,
         "RELEASE_NO_LEADING_ZEROS": release_version_no_leading_zeros,
         "VERSION_SUFFIX": version_suffix,
-        "ADDITIONAL_INFO": get_additional_package_info(provider_package_path=source_provider_package_path),
+        "ADDITIONAL_INFO": get_additional_package_info(
+            provider_package_path=provider_details.source_provider_package_path
+        ),
+        "CHANGELOG": get_changelog_for_package(
+            provider_package_path=provider_details.source_provider_package_path
+        ),
         "CROSS_PROVIDERS_DEPENDENCIES": cross_providers_dependencies,
-        "PIP_REQUIREMENTS": PROVIDERS_REQUIREMENTS[provider_package_id],
-        "PROVIDER_TYPE": "Backport provider" if BACKPORT_PACKAGES else "Provider",
-        "PROVIDERS_FOLDER": "backport-providers" if BACKPORT_PACKAGES else "providers",
+        "PIP_REQUIREMENTS": PROVIDERS_REQUIREMENTS[provider_details.provider_package_id],
+        "PROVIDER_TYPE": "Backport provider" if backport_packages else "Provider",
+        "PROVIDERS_FOLDER": "backport-providers" if backport_packages else "providers",
         "INSTALL_REQUIREMENTS": get_install_requirements(
-            provider_package_id=provider_package_id, backport_packages=backport_packages
+            provider_package_id=provider_details.provider_package_id, backport_packages=backport_packages
         ),
         "SETUP_REQUIREMENTS": get_setup_requirements(),
         "EXTRAS_REQUIREMENTS": get_package_extras(
-            provider_package_id=provider_package_id, backport_packages=backport_packages
+            provider_package_id=provider_details.provider_package_id, backport_packages=backport_packages
         ),
-        "PROVIDER_INFO": get_provider_info_from_provider_yaml(provider_package_id),
+        "CROSS_PROVIDERS_DEPENDENCIES_TABLE": cross_providers_dependencies_table,
+        "CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST": cross_providers_dependencies_table_rst,
+        "PIP_REQUIREMENTS_TABLE": pip_requirements_table,
+        "PIP_REQUIREMENTS_TABLE_RST": pip_requirements_table_rst,
     }
-    if update_release_notes:
-        git_cmd = get_git_command(previous_release)
-        try:
-            changes = subprocess.check_output(
-                git_cmd, cwd=source_provider_package_path, universal_newlines=True
-            )
-            changes_table = convert_git_changes_to_table(
-                changes, base_url="https://github.com/apache/airflow/commit/"
-            )
-        except subprocess.CalledProcessError:
-            # TODO(potiuk) fix me for both backport/provider package check
-            changes_table = ''
-        context["CURRENT_CHANGES_TABLE"] = changes_table
-        pip_requirements_table = convert_pip_requirements_to_table(
-            PROVIDERS_REQUIREMENTS[provider_package_id]
-        )
-        cross_providers_dependencies_table = convert_cross_package_dependencies_to_table(
-            cross_providers_dependencies,
-            backport_packages=backport_packages,
-        )
-        context["CROSS_PROVIDERS_DEPENDENCIES_TABLE"] = cross_providers_dependencies_table
-        context["PIP_REQUIREMENTS_TABLE"] = pip_requirements_table
+    return context
 
-    total, bad = check_if_classes_are_properly_named(entity_summaries)
+
+def prepare_readme_file(
+    context,
+):
+    readme_content = LICENCE
+    readme_template_name = PROVIDER_TEMPLATE_PREFIX + "README"
+    readme_content += render_template(template_name=readme_template_name, context=context, extension='.md')
+    readme_file_path = os.path.join(TARGET_PROVIDER_PACKAGES_PATH, "README.md")
+    with open(readme_file_path, "wt") as readme_file:
+        readme_file.write(readme_content)
+
+
+def update_generated_files_for_regular_package(
+    provider_package_id: str,
+    version_suffix: str,
+    update_release_notes: bool,
+    update_setup: bool,
+) -> None:
+    """
+    Updates generated files (readme, changes and/or setup.cfg/setup.py/manifest.in/provider_info)
+
+    :param provider_package_id: id of the package
+    :param version_suffix: version suffix corresponding to the version in the code
+    :param update_release_notes: whether to update release notes
+    :param update_setup: whether to update setup files
+    """
+    verify_provider_package(provider_package_id)
+    provider_details = get_provider_details(provider_package_id)
+    provider_info = get_provider_info_from_provider_yaml(provider_package_id)
+    versions: List[str] = provider_info['versions']
+    current_release_version = versions[0]
+    # previous_release = versions[1] if len(versions) > 1 else None
+    # previous_release_commit_ref = f"providers-{provider_package_id.replace('.','-')}/{previous_release}"

Review comment:
       this can go I guess

##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       Oh yes, I agree with that @mik-laj -- My assumption was detailed changelog was updating guide but looks like it is just list of commits.
   
   So yeah, **Changelog** -> **Upgrading guide** and **Detailed changelog** -> **Changelog** . 

##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       Oh we should definitely have a Changelog which contains the commit log whether it is automated or done manually




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-763085083


   [The Workflow run](https://github.com/apache/airflow/actions/runs/496754528) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-764630173


   [The Workflow run](https://github.com/apache/airflow/actions/runs/501081226) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561726275



##########
File path: dev/provider_packages/BACKPORT_PROVIDER_README_TEMPLATE.md.jinja2
##########
@@ -88,6 +95,6 @@ You can install such cross-provider dependencies when installing from PyPI. For
 pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
 ```
 
-{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE }}
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE | safe }}

Review comment:
       Avoids escaping < > into &gt; &lt;




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561724951



##########
File path: airflow/providers/amazon/README.md
##########
@@ -1,483 +0,0 @@
-<!--
- 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.
- -->
-
-
-# Package apache-airflow-providers-amazon

Review comment:
       Yeah. Only during release. We do not really need them here. There is very little value in keeping them in the repo, they will be available in PyPI. But If you really think they are needed, I can add them back. I just thought - since we already have the information there in setup.py, dependencies.json , the value of having it in the repo is very little. But I can be convince otherwise..




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561723291



##########
File path: airflow/providers/README.md
##########
@@ -1,28 +0,0 @@
-<!--
- 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.
- -->
-
-# Airflow Providers

Review comment:
       Yes. All the READMEs  are not needed any more - they are dynamically generated while package is prepared and there is no point in keeping them in repo. 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-770834992


   Looks green https://github.com/potiuk/airflow/actions/runs/527627362


----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-770846938


   [The Workflow run](https://github.com/apache/airflow/actions/runs/527627243) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567772927



##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -766,40 +785,50 @@ def convert_pip_requirements_to_table(requirements: Iterable[str]) -> str:
         if found:
             package = found.group(1)
             version_required = found.group(2)
-            table_data.append((package, version_required))
+            if version_required == "":
+                version_required = ""
+            else:

Review comment:
       :D. Some left-over :). changed 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-766146487


   [The Workflow run](https://github.com/apache/airflow/actions/runs/506055069) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] mik-laj commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-766279190


   > Looking at one of the operators in particular, and I love the prerequisite tasks (Oh, are these perhaps the guides that are embedded in this?)
   
   I can see that this fragment is not displayed correctly. All guides have an identical "the prerequisite tasks" section.
   <img width="1361" alt="Screenshot 2021-01-24 at 03 30 07" src="https://user-images.githubusercontent.com/12058428/105619621-b0876a00-5df4-11eb-9ee6-2a3673294b0a.png">
   
   


----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk merged pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk merged pull request #13767:
URL: https://github.com/apache/airflow/pull/13767


   


----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-770428606


   Hey @kaxil @ash @mik-laj  - I got - I think - final version of the provider PR finally working and green. It will fail here because it needs to be merged to MASTER to succeed but I got it running and "green" in my own fork here:
   
   https://github.com/potiuk/airflow/actions/runs/525666213
   
   The output is now sooo much nicer and shows exactly what's going on so that it is easy to grasp it. The logs are nicely folded in groups and all is colored. Also errors are much better visible, red and pop-out immediately.
   
   Some screenshots are taken from the CI job, but similar output you get as release manager when you are running ./breeze commands to see how many of the packages need version bump:
   
   Building PROD image using provider packages:
   
   ![Screenshot from 2021-01-31 19-09-15](https://user-images.githubusercontent.com/595491/106393513-d5cd3700-63f7-11eb-9545-d8b64cfbe2fc.png)
   
   You can see that first all provider packages are built, then the main airflow package and the PROD image is built using those provider packages (by unfolding each package build you can see exact commands in each of the steps for each package:
   * generating setup files
   * checking if suffixes are fine
   * building the package
   
   Preparing provider documentation:
   
   You can see an example output genmerated by new neo4j proivider (fresh provider):
   
   ![Screenshot from 2021-01-31 19-12-42](https://user-images.githubusercontent.com/595491/106393597-51c77f00-63f8-11eb-8527-bb7688f851d9.png)
   
   Similar output for packages that had some changes but their version has not been updated:
   
   ![image](https://user-images.githubusercontent.com/595491/106393635-96531a80-63f8-11eb-8fb8-f2135f8054d8.png)
   
   And another one when when the version is already updatead and package is ready for release:
   
   ![Screenshot from 2021-01-31 19-17-28](https://user-images.githubusercontent.com/595491/106393726-1a0d0700-63f9-11eb-9efc-f39f140245a8.png)
   
   We also build and test ALL provider packages in the CI build separately (in PROD image build we only build a subset of those :
   
   ![image](https://user-images.githubusercontent.com/595491/106393821-6f491880-63f9-11eb-9353-6d690ad11561.png)
   
   In the "regular" provider's package. We install all the providers, see if everything can be imported and we test all `airflow providers` command to see if everything is good:
   
   ![image](https://user-images.githubusercontent.com/595491/106393885-c7801a80-63f9-11eb-8d2f-8df4f8ab1769.png)
   
   For Backport providers  we just check if the backport providers can be installed in latest 1.10.* released (1.10.14 now): 
   
   ![image](https://user-images.githubusercontent.com/595491/106393917-03b37b00-63fa-11eb-9e5a-6f4f891e4fd1.png)
   


----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-766452624






----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-770812242


   Fixed comments @turbaszek  and rebased to latest master: I also run it in my fork's master to verify no typos etc. https://github.com/potiuk/airflow/actions/runs/527627362


----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567774110



##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -830,87 +859,17 @@ def convert_cross_package_dependencies_to_table(
 )
 
 
-def strip_leading_zeros_in_calver(calver_version: str) -> str:
+def strip_leading_zeros(version: str) -> str:
     """
-    Strips leading zeros from calver version number.
+    Strips leading zeros from version number.
 
     This converts 1974.04.03 to 1974.4.3 as the format with leading month and day zeros is not accepted
     by PIP versioning.
 
-    :param calver_version: version number in calver format (potentially with leading 0s in date and month)
+    :param version: version number in calver format (potentially with leading 0s in date and month)
     :return: string with leading 0s after dot replaced.
     """
-    return calver_version.replace(".0", ".")
-
-
-def get_provider_changes_prefix(backport_packages: bool) -> str:
-    """
-    Returns prefix for provider CHANGES files.
-    """
-    if backport_packages:
-        return "BACKPORT_PROVIDER_CHANGES_"
-    else:
-        return "PROVIDER_CHANGES_"
-
-
-def get_all_releases(provider_package_path: str, backport_packages: bool) -> List[ReleaseInfo]:
-    """
-    Returns information about past releases (retrieved from *changes_*md files stored in the
-    package folder.
-    :param provider_package_path: path of the package
-    :param backport_packages: whether to prepare regular (False) or backport (True) packages
-    :return: list of releases made so far.
-    """
-    changes_file_prefix = get_provider_changes_prefix(backport_packages=backport_packages)
-    past_releases: List[ReleaseInfo] = []
-    changes_file_names = listdir(provider_package_path)
-    for file_name in sorted(changes_file_names, reverse=True):
-        if file_name.startswith(changes_file_prefix) and file_name.endswith(".md"):
-            changes_file_path = os.path.join(provider_package_path, file_name)
-            with open(changes_file_path) as changes_file:
-                content = changes_file.read()
-            found = re.search(r'/([a-z0-9]*)\)', content, flags=re.MULTILINE)
-            if not found:
-                print("[yellow]No commit found. This seems to be first time you run it[/]", file=sys.stderr)
-            else:
-                last_commit_hash = found.group(1)
-                release_version = file_name[len(changes_file_prefix) :][:-3]
-                release_version_no_leading_zeros = (
-                    strip_leading_zeros_in_calver(release_version) if backport_packages else release_version
-                )
-                past_releases.append(
-                    ReleaseInfo(
-                        release_version=release_version,
-                        release_version_no_leading_zeros=release_version_no_leading_zeros,
-                        last_commit_hash=last_commit_hash,
-                        content=content,
-                        file_name=file_name,
-                    )
-                )
-    return past_releases
-
-
-def get_latest_release(provider_package_path: str, backport_packages: bool) -> ReleaseInfo:
-    """
-    Gets information about the latest release.
-
-    :param provider_package_path: path of package
-    :param backport_packages: whether to prepare regular (False) or backport (True) packages
-    :return: latest release information
-    """
-    releases = get_all_releases(
-        provider_package_path=provider_package_path, backport_packages=backport_packages
-    )
-    if len(releases) == 0:
-        return ReleaseInfo(
-            release_version="0.0.0",
-            release_version_no_leading_zeros="0.0.0",
-            last_commit_hash="no_hash",
-            content="empty",
-            file_name="no_file",
-        )
-    else:
-        return releases[0]
+    return ".".join(str(int(i)) for i in version.split("."))

Review comment:
       Yes. This strips leading zeros in case the original string is in the form of "01" for example. See the comment few lines above.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561405492



##########
File path: BREEZE.rst
##########
@@ -2056,6 +2065,28 @@ This is the current syntax for  `./breeze <./breeze>`_:
 
   Flags:
 
+  -S, --version-suffix-for-pypi SUFFIX
+          Adds optional suffix to the version in the generated backport package. It can be used

Review comment:
       Is this only for backports or regular providers too?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-770221007


   [The Workflow run](https://github.com/apache/airflow/actions/runs/523447289) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-763039729


   [The Workflow run](https://github.com/apache/airflow/actions/runs/496721864) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561412135



##########
File path: airflow/providers/README.md
##########
@@ -1,28 +0,0 @@
-<!--
- 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.
- -->
-
-# Airflow Providers

Review comment:
       is this deletion intentional?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r560993720



##########
File path: dev/provider_packages/copy_provider_package_sources.py
##########
@@ -767,27 +781,38 @@ def do_refactor(self, in_process: bool = False) -> None:  # noqa
         self.refactor_kubernetes_pod_operator()
         # In order to debug Bowler - set in_process to True
         self.qry.execute(write=True, silent=False, interactive=False, in_process=in_process)
+        print("[green]OK[/]")
+
+
+def get_parser():
+    cli_parser = argparse.ArgumentParser(
+        description="Copies sources and optionally refactors provider code to be Airflow 1.10 compatible.",
+        formatter_class=argparse.RawTextHelpFormatter,
+    )
+    cli_parser.add_argument(
+        "--backports",
+        action='store_true',
+        help=textwrap.dedent("Includes refactoring to prepare backport packages rather than regular ones"),
+    )
+    cli_parser.add_argument(
+        "--debug",
+        action='store_true',
+        help=textwrap.dedent(
+            "Run bowler refactoring in single process. Makes it debuggable with regular"
+            " IDE debugger (much slower)"
+        ),
+    )
+    return cli_parser
 
 
 if __name__ == '__main__':
-    BACKPORT_PACKAGES = os.getenv('BACKPORT_PACKAGES') == "true"
-    in_process = False
-    if len(sys.argv) > 1:
-        if sys.argv[1] in ['--help', '-h']:
-            print()
-            print("Refactors provider packages to be Airflow 1.10 compatible.")
-            print()
-            print(f"Usage: {sys.argv[0]} [--debug] | [-h] | [--help]")
-            print()
-            print("You can use --debug flag in order to run bowler refactoring in process.")
-            print("This allows you to debug bowler process as usual using your IDE debugger")
-            print("Otherwise it heavily uses multi-processing and is next-to-impossible to debug")
-            print()
-            print("Note - Bowler is also a lot slower in this mode.")
-            print()
-            sys.exit(0)
-        if sys.argv[1] == '--debug':
-            in_process = True
-    copy_provider_sources()
-    if BACKPORT_PACKAGES:
+    parser = get_parser()
+    args = parser.parse_args()
+
+    in_process = args.debug
+    copy_provider_sources(args.backports)
+    os.chdir(get_target_providers_folder())
+    if args.backports:
+        print("\nRefactoring code to be Airflow 1.10 - compatible\n")

Review comment:
       I prefer inlining :) 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561742229



##########
File path: dev/provider_packages/copy_provider_package_sources.py
##########
@@ -767,27 +781,38 @@ def do_refactor(self, in_process: bool = False) -> None:  # noqa
         self.refactor_kubernetes_pod_operator()
         # In order to debug Bowler - set in_process to True
         self.qry.execute(write=True, silent=False, interactive=False, in_process=in_process)
+        print("[green]OK[/]")
+
+
+def get_parser():
+    cli_parser = argparse.ArgumentParser(
+        description="Copies sources and optionally refactors provider code to be Airflow 1.10 compatible.",
+        formatter_class=argparse.RawTextHelpFormatter,
+    )
+    cli_parser.add_argument(
+        "--backports",
+        action='store_true',
+        help=textwrap.dedent("Includes refactoring to prepare backport packages rather than regular ones"),
+    )
+    cli_parser.add_argument(
+        "--debug",
+        action='store_true',
+        help=textwrap.dedent(
+            "Run bowler refactoring in single process. Makes it debuggable with regular"
+            " IDE debugger (much slower)"
+        ),
+    )
+    return cli_parser
 
 
 if __name__ == '__main__':
-    BACKPORT_PACKAGES = os.getenv('BACKPORT_PACKAGES') == "true"
-    in_process = False
-    if len(sys.argv) > 1:
-        if sys.argv[1] in ['--help', '-h']:
-            print()
-            print("Refactors provider packages to be Airflow 1.10 compatible.")
-            print()
-            print(f"Usage: {sys.argv[0]} [--debug] | [-h] | [--help]")
-            print()
-            print("You can use --debug flag in order to run bowler refactoring in process.")
-            print("This allows you to debug bowler process as usual using your IDE debugger")
-            print("Otherwise it heavily uses multi-processing and is next-to-impossible to debug")
-            print()
-            print("Note - Bowler is also a lot slower in this mode.")
-            print()
-            sys.exit(0)
-        if sys.argv[1] == '--debug':
-            in_process = True
-    copy_provider_sources()
-    if BACKPORT_PACKAGES:
+    parser = get_parser()
+    args = parser.parse_args()
+
+    in_process = args.debug
+    copy_provider_sources(args.backports)
+    os.chdir(get_target_providers_folder())
+    if args.backports:
+        print("\nRefactoring code to be Airflow 1.10 - compatible\n")
         RefactorBackportPackages().do_refactor(in_process=in_process)
+        print("\n[green]Refactored code successfully[/]\n")

Review comment:
       :D




----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561405889



##########
File path: BREEZE.rst
##########
@@ -2056,6 +2065,28 @@ This is the current syntax for  `./breeze <./breeze>`_:
 
   Flags:
 
+  -S, --version-suffix-for-pypi SUFFIX
+          Adds optional suffix to the version in the generated backport package. It can be used
+          to generate rc1/rc2 ... versions of the packages to be uploaded to PyPI.
+
+  -N, --version-suffix-for-svn SUFFIX
+          Adds optional suffix to the generated names of package. It can be used to generate
+          rc1/rc2 ... versions of the packages to be uploaded to SVN.
+
+  --package-format PACKAGE_FORMAT
+
+          Chooses format of packages to prepare.
+
+          One of:
+
+                 wheel,sdist,both
+
+          Default: 

Review comment:
       is there NO default?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567697537



##########
File path: dev/provider_packages/README_BACKPORT_PACKAGES.md
##########
@@ -0,0 +1,260 @@
+<!--
+ 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.
+-->
+
+<!-- START doctoc generated TOC please keep comment here to allow auto update -->
+<!-- DON'T EDIT THIS SECTION, INSTEAD RE-RUN doctoc TO UPDATE -->
+**Table of Contents**  *generated with [DocToc](https://github.com/thlorenz/doctoc)*
+
+- [Backport packages](#backport-packages)
+- [What the backport packages are](#what-the-backport-packages-are)
+- [Content of the release notes](#content-of-the-release-notes)
+  - [Generating release notes](#generating-release-notes)
+  - [Preparing backport packages](#preparing-backport-packages)
+- [Debugging the backport provider packages](#debugging-the-backport-provider-packages)
+- [Testing backport provider packages](#testing-backport-provider-packages)
+
+<!-- END doctoc generated TOC please keep comment here to allow auto update -->
+
+# Backport packages
+
+# What the backport packages are
+
+The Backport Provider packages are packages (per provider) that make it possible to easily use Hooks,
+Operators, Sensors, and Secrets from the 2.0 version of Airflow in the 1.10.* series.
+
+The release manager prepares backport packages separately from the main Airflow Release, using
+`breeze` commands and accompanying scripts. This document provides an overview of the command line tools
+needed to prepare backport packages.
+
+# Content of the release notes
+
+Each of the backport packages contains Release notes in the form of the README.md file that is
+automatically generated from history of the changes and code of the provider.
+
+The script generates all the necessary information:
+
+* summary of requirements for each backport package
+* list of dependencies (including extras to install them) when package
+  depends on other providers packages
+* table of new hooks/operators/sensors/protocols/secrets
+* table of moved hooks/operators/sensors/protocols/secrets with the
+  information where they were moved from
+* changelog of all the changes to the provider package. This will be
+  automatically updated with an incremental changelog whenever we decide to
+  release separate packages.
+
+The script generates two types of files:
+
+* BACKPORT_PROVIDERS_CHANGES_YYYY.MM.DD.md which keeps information about changes (commits) in a particular
+  version of the provider package. The file for latest release gets updated when you iterate with
+  the same new date/version, but it never changes automatically for already released packages.
+  This way - just before the final release, you can manually correct the changes file if you
+  want to remove some changes from the file.
+
+* README.md which is regenerated every time you run the script (unless there are no changes since
+  the last time you generated the release notes
+
+Note that our CI system builds the release notes for backport packages automatically with every build and
+current date - this way you might be sure the automated generation of the release notes continues to
+work. You can also preview the generated readme files (by downloading artifacts from GitHub Actions).
+The script does not modify the README and CHANGES files if there is no change in the repo for that provider.
+
+
+## Generating release notes
+
+When you want to prepare release notes for a package, you need to run:
+
+```
+./breeze --backports prepare-provider-documentation [YYYY.MM.DD] <PACKAGE_ID> ...
+```
+
+
+* YYYY.MM.DD - is the CALVER version of the package to prepare. Note that this date cannot be earlier
+  than the already released version (the script will fail if it will be). It can be set in the future
+  anticipating the future release date. If you do not specify date, the date will be taken from the last
+  generated readme - the last generated CHANGES file will be updated.
+
+* <PACKAGE_ID> is usually directory in the `airflow/providers` folder (for example `google` but in several
+  cases, it might be one level deeper separated with `.` for example `apache.hive`
+
+You can run the script with multiple package names if you want to prepare several packages at the same time.
+Before you specify a new version, the last released version is update in case you have any bug fixes
+merged in the master recently, they will be automatically taken into account.
+
+Typically, the first time you run release before release, you run it with target release.date:

Review comment:
       ```suggestion
   Typically, the first time you run he script before release, you run it with target release date:
   ```
   ?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-763040319


   [The Workflow run](https://github.com/apache/airflow/actions/runs/496588104) is cancelling this PR. Building image for the PR has been cancelled


----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-770386623


   [The Workflow run](https://github.com/apache/airflow/actions/runs/525316601) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561398276



##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       I like what you have suggested here Jarek.
   
   - Concise README
   - Link to commit changelog
   - detailed changelog is the Updating guide right?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567774609



##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -977,70 +933,94 @@ def get_cross_provider_dependent_packages(provider_package_id: str) -> List[str]
 
 def make_sure_remote_apache_exists_and_fetch():
     """
-    Make sure that apache remote exist in git. We need to take a log from the master of apache
-    repository - not locally - because when we commit this change and run it, our log will include the
-    current commit - which is going to have different commit id once we merge. So it is a bit
-    catch-22.
+    Make sure that apache remote exist in git. We need to take a log from the apache
+    repository - not locally.
+
+    Also the the local repo might be shallow so we need to unshallow it.
+
+    This will:
+    * check if the remote exists and add if it does not
+    * check if the local repo is shallow, markit to be unshallowed in this case
+    * fetch from the remote including all tags and overriding local taags in case they are set differently
 
-    :return:
     """
     try:
+        check_remote_command = ["git", "remote", "get-url", HTTPS_REMOTE]
         subprocess.check_call(
-            ["git", "remote", "add", "apache-https-for-providers", "https://github.com/apache/airflow.git"],
+            check_remote_command,
             stdout=subprocess.DEVNULL,
             stderr=subprocess.DEVNULL,
         )
-    except subprocess.CalledProcessError as e:
-        if e.returncode == 128:
-            print(
-                "[yellow]The remote `apache-https-for-providers` already exists. If you have trouble running "
-                "git log delete the remote[/]",
-                file=sys.stderr,
-            )
+        print(f"Running command: '{' '.join(check_remote_command)}'")

Review comment:
       Ah yeah. Fixed :)




----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561417616



##########
File path: dev/provider_packages/BACKPORT_PROVIDER_README_TEMPLATE.md.jinja2
##########
@@ -88,6 +95,6 @@ You can install such cross-provider dependencies when installing from PyPI. For
 pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
 ```
 
-{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE }}
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE | safe }}

Review comment:
       what's the reason for `| safe`




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-766461430


   [The Workflow run](https://github.com/apache/airflow/actions/runs/508053358) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567855387



##########
File path: airflow/providers/res.txt
##########
@@ -0,0 +1,61 @@
+./salesforce
+./celery
+./amazon

Review comment:
       This file needs to be removed right?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567697537



##########
File path: dev/provider_packages/README_BACKPORT_PACKAGES.md
##########
@@ -0,0 +1,260 @@
+<!--
+ 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.
+-->
+
+<!-- START doctoc generated TOC please keep comment here to allow auto update -->
+<!-- DON'T EDIT THIS SECTION, INSTEAD RE-RUN doctoc TO UPDATE -->
+**Table of Contents**  *generated with [DocToc](https://github.com/thlorenz/doctoc)*
+
+- [Backport packages](#backport-packages)
+- [What the backport packages are](#what-the-backport-packages-are)
+- [Content of the release notes](#content-of-the-release-notes)
+  - [Generating release notes](#generating-release-notes)
+  - [Preparing backport packages](#preparing-backport-packages)
+- [Debugging the backport provider packages](#debugging-the-backport-provider-packages)
+- [Testing backport provider packages](#testing-backport-provider-packages)
+
+<!-- END doctoc generated TOC please keep comment here to allow auto update -->
+
+# Backport packages
+
+# What the backport packages are
+
+The Backport Provider packages are packages (per provider) that make it possible to easily use Hooks,
+Operators, Sensors, and Secrets from the 2.0 version of Airflow in the 1.10.* series.
+
+The release manager prepares backport packages separately from the main Airflow Release, using
+`breeze` commands and accompanying scripts. This document provides an overview of the command line tools
+needed to prepare backport packages.
+
+# Content of the release notes
+
+Each of the backport packages contains Release notes in the form of the README.md file that is
+automatically generated from history of the changes and code of the provider.
+
+The script generates all the necessary information:
+
+* summary of requirements for each backport package
+* list of dependencies (including extras to install them) when package
+  depends on other providers packages
+* table of new hooks/operators/sensors/protocols/secrets
+* table of moved hooks/operators/sensors/protocols/secrets with the
+  information where they were moved from
+* changelog of all the changes to the provider package. This will be
+  automatically updated with an incremental changelog whenever we decide to
+  release separate packages.
+
+The script generates two types of files:
+
+* BACKPORT_PROVIDERS_CHANGES_YYYY.MM.DD.md which keeps information about changes (commits) in a particular
+  version of the provider package. The file for latest release gets updated when you iterate with
+  the same new date/version, but it never changes automatically for already released packages.
+  This way - just before the final release, you can manually correct the changes file if you
+  want to remove some changes from the file.
+
+* README.md which is regenerated every time you run the script (unless there are no changes since
+  the last time you generated the release notes
+
+Note that our CI system builds the release notes for backport packages automatically with every build and
+current date - this way you might be sure the automated generation of the release notes continues to
+work. You can also preview the generated readme files (by downloading artifacts from GitHub Actions).
+The script does not modify the README and CHANGES files if there is no change in the repo for that provider.
+
+
+## Generating release notes
+
+When you want to prepare release notes for a package, you need to run:
+
+```
+./breeze --backports prepare-provider-documentation [YYYY.MM.DD] <PACKAGE_ID> ...
+```
+
+
+* YYYY.MM.DD - is the CALVER version of the package to prepare. Note that this date cannot be earlier
+  than the already released version (the script will fail if it will be). It can be set in the future
+  anticipating the future release date. If you do not specify date, the date will be taken from the last
+  generated readme - the last generated CHANGES file will be updated.
+
+* <PACKAGE_ID> is usually directory in the `airflow/providers` folder (for example `google` but in several
+  cases, it might be one level deeper separated with `.` for example `apache.hive`
+
+You can run the script with multiple package names if you want to prepare several packages at the same time.
+Before you specify a new version, the last released version is update in case you have any bug fixes
+merged in the master recently, they will be automatically taken into account.
+
+Typically, the first time you run release before release, you run it with target release.date:

Review comment:
       ```suggestion
   Typically, the first time you run the script before release, you run it with target release date:
   ```
   ?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561411664



##########
File path: airflow/providers/CHANGELOG.rst
##########
@@ -0,0 +1,25 @@
+ .. 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.
+
+
+Changelog
+---------
+
+1.0.0
+.....
+
+Initial version of the provider.

Review comment:
       Don't  think this is intended -- we don't want to have Changelog.rst in `airflow/providers`




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561722086



##########
File path: BREEZE.rst
##########
@@ -2056,6 +2065,28 @@ This is the current syntax for  `./breeze <./breeze>`_:
 
   Flags:
 
+  -S, --version-suffix-for-pypi SUFFIX
+          Adds optional suffix to the version in the generated backport package. It can be used
+          to generate rc1/rc2 ... versions of the packages to be uploaded to PyPI.
+
+  -N, --version-suffix-for-svn SUFFIX
+          Adds optional suffix to the generated names of package. It can be used to generate
+          rc1/rc2 ... versions of the packages to be uploaded to SVN.
+
+  --package-format PACKAGE_FORMAT
+
+          Chooses format of packages to prepare.
+
+          One of:
+
+                 wheel,sdist,both
+
+          Default: 

Review comment:
       There is - wheel (though I might want to change it to both). I will see why :)




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-770252604


   [The Workflow run](https://github.com/apache/airflow/actions/runs/523736793) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561710341



##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       > > feat!:, or fix!:, refactor!:, etc., which represent a breaking change (indicated by the !) and will result in a SemVer major.
   
   I think automatically generating nice changelog is something we can think about in the future. And if you look at the commit changes now, there are many we would have to exclude because they are just refactors/global updates. For now I thin we should generate changelog manually (and every time someone makes a new feature or breaking change - during review, we should ask the person to add entry to the changelog. I will update the changelogs manually with the first release, but from the 1.0.1 onwards, this is just a matter that all committers will pay attention to it and make sure that Changelog is updated.
   
   As the next step we could introduce the semantic commits and automated changelog generation. But this would require discussion, voting and likely some automation to make sure we follow it. I would like to decouple this from this change.
   




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561742545



##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -1225,184 +1195,272 @@ def get_provider_info_from_provider_yaml(provider_package_id: str) -> Dict[str,
     return provider_info
 
 
-def update_generated_files_for_package(
+def get_backport_current_changes_table(previous_release_commit_ref: str, source_provider_package_path: str):
+    git_cmd = get_git_log_command(previous_release_commit_ref)
+    try:
+        changes = subprocess.check_output(git_cmd, cwd=source_provider_package_path, universal_newlines=True)
+        changes_table = convert_git_changes_to_table(
+            previous_release_commit_ref, changes, base_url="https://github.com/apache/airflow/commit/"
+        )
+    except subprocess.CalledProcessError:
+        changes_table = ''
+    return changes_table
+
+
+def get_version_tag(version: str, provider_package_id: str, version_suffix: str):
+    return f"providers-{provider_package_id.replace('.','-')}/{version}{version_suffix}"
+
+
+def print_changes_table(changes_table):
+    syntax = Syntax(changes_table, "rst", theme="light")
+    console = Console(width=200)
+    console.print(syntax)
+
+
+def get_all_changes_for_regular_packages(
+    versions: List[str],
     provider_package_id: str,
-    current_release_version: str,
+    source_provider_package_path: str,
     version_suffix: str,
-    imported_classes: List[str],
-    backport_packages: bool,
-    update_release_notes: bool,
-    update_setup: bool,
-) -> Tuple[int, int]:
-    """
-    Updates release notes (BACKPORT_PROVIDER_README.md/README.md) for the package.
-    Returns Tuple of total number of entities and badly named entities.
+) -> Tuple[bool, str]:
+    current_version = versions[0]
+    current_tag = get_version_tag(current_version, provider_package_id, version_suffix)
+    print(f"Checking if tag `{current_tag}` exists.")
+    if not subprocess.call(
+        get_git_tag_check_command(current_tag),
+        cwd=source_provider_package_path,
+        stderr=subprocess.DEVNULL,
+    ):
+        print(f"The tag {current_tag} exists.")
+        # The tag already exists
+        changes = subprocess.check_output(
+            get_git_log_command(current_tag, 'HEAD'),
+            cwd=source_provider_package_path,
+            universal_newlines=True,
+        )
+        if len(changes) > 0:
+            print(
+                f"[yellow]The provider {provider_package_id} has changes"
+                f" since last release but version is not updated[/]"
+            )
+            print()
+            print(
+                f'Please update version in `airflow/providers/{provider_package_id.replace("-","/")}/'
+                'provider.yaml` to prepare release.\n'
+            )
+            changes_table = convert_git_changes_to_table(
+                "UNKNOWN", changes, base_url="https://github.com/apache/airflow/commit/", markdown=False
+            )
+            print_changes_table(changes_table)
+            return False, changes_table
+        else:
+            print(f"No changes for {provider_package_id}")
+            return False, ""
+    print("The tag does not exist. ")
+    if len(versions) == 1:
+        print("This provider has never been released!\n")
+    else:
+        print(f"The {provider_package_id} is ready to be released!\n")
+    current_version = 'HEAD'
+    changes_table = ''
+    print_version = versions[0]
+    for version in versions[1:]:
+        version_tag = get_version_tag(version, provider_package_id, version_suffix)
+        changes = subprocess.check_output(
+            get_git_log_command(version_tag, current_version),
+            cwd=source_provider_package_path,
+            universal_newlines=True,
+        )
+        changes_table += convert_git_changes_to_table(
+            print_version, changes, base_url="https://github.com/apache/airflow/commit/", markdown=False
+        )
+        current_version = version
+        print_version = version
+    print_changes_table(changes_table)
+    return True, changes_table
+
+
+def get_provider_details(provider_package_id: str) -> ProviderPackageDetails:
+    return ProviderPackageDetails(
+        provider_package_id=provider_package_id,
+        full_package_name=f"airflow.providers.{provider_package_id}",
+        source_provider_package_path=get_source_package_path(provider_package_id),
+        documentation_provider_package_path=get_documentation_package_path(provider_package_id),
+    )
 
-    :param provider_package_id: id of the package
-    :param current_release_version: release version:
-    :param version_suffix: version suffix corresponding to the version in the code
-    :param imported_classes - entities that have been imported from providers
-    :param backport_packages: whether to prepare regular (False) or backport (True) packages
-    :param update_release_notes: whether to update release notes
-    :param update_setup: whether to update setup files
 
-    :return: Tuple of total/bad number of entities
-    """
-    verify_provider_package(provider_package_id)
-    full_package_name = f"airflow.providers.{provider_package_id}"
-    source_provider_package_path = get_source_package_path(provider_package_id)
-    entity_summaries = get_package_class_summary(full_package_name, imported_classes)
-    past_releases = get_all_releases(
-        provider_package_path=source_provider_package_path, backport_packages=backport_packages
+def get_provider_jinja_context(
+    provider_details: ProviderPackageDetails,
+    current_release_version: str,
+    version_suffix: str,
+    backport_packages: bool,
+):
+    verify_provider_package(provider_details.provider_package_id)
+    cross_providers_dependencies = get_cross_provider_dependent_packages(
+        provider_package_id=provider_details.provider_package_id
     )
-    current_release_version, previous_release = check_if_release_version_ok(
-        past_releases, current_release_version, backport_packages
+    release_version_no_leading_zeros = strip_leading_zeros(current_release_version)
+    pip_requirements_table = convert_pip_requirements_to_table(
+        PROVIDERS_REQUIREMENTS[provider_details.provider_package_id]
     )
-    cross_providers_dependencies = get_cross_provider_dependent_packages(
-        provider_package_id=provider_package_id
+    pip_requirements_table_rst = convert_pip_requirements_to_table(
+        PROVIDERS_REQUIREMENTS[provider_details.provider_package_id], markdown=False
     )
-    previous_release = get_previous_release_info(
-        previous_release_version=previous_release,
-        past_releases=past_releases,
-        current_release_version=current_release_version,
+    cross_providers_dependencies_table = convert_cross_package_dependencies_to_table(
+        cross_providers_dependencies,
+        backport_packages=backport_packages,
     )
-    release_version_no_leading_zeros = (
-        strip_leading_zeros_in_calver(current_release_version)
-        if backport_packages
-        else current_release_version
+    cross_providers_dependencies_table_rst = convert_cross_package_dependencies_to_table(
+        cross_providers_dependencies, backport_packages=backport_packages, markdown=False
     )
     context: Dict[str, Any] = {
         "ENTITY_TYPES": list(EntityType),
         "README_FILE": "BACKPORT_PROVIDER_README.md" if backport_packages else "README.md",
-        "PROVIDER_PACKAGE_ID": provider_package_id,
-        "PACKAGE_PIP_NAME": get_pip_package_name(provider_package_id, backport_packages),
-        "FULL_PACKAGE_NAME": full_package_name,
-        "PROVIDER_PATH": full_package_name.replace(".", "/"),
+        "PROVIDER_PACKAGE_ID": provider_details.provider_package_id,
+        "PACKAGE_PIP_NAME": get_pip_package_name(
+            provider_details.provider_package_id, backport_packages=backport_packages
+        ),
+        "FULL_PACKAGE_NAME": provider_details.full_package_name,
+        "PROVIDER_PATH": provider_details.full_package_name.replace(".", "/"),
         "RELEASE": current_release_version,
         "RELEASE_NO_LEADING_ZEROS": release_version_no_leading_zeros,
         "VERSION_SUFFIX": version_suffix,
-        "ADDITIONAL_INFO": get_additional_package_info(provider_package_path=source_provider_package_path),
+        "ADDITIONAL_INFO": get_additional_package_info(
+            provider_package_path=provider_details.source_provider_package_path
+        ),
+        "CHANGELOG": get_changelog_for_package(
+            provider_package_path=provider_details.source_provider_package_path
+        ),
         "CROSS_PROVIDERS_DEPENDENCIES": cross_providers_dependencies,
-        "PIP_REQUIREMENTS": PROVIDERS_REQUIREMENTS[provider_package_id],
-        "PROVIDER_TYPE": "Backport provider" if BACKPORT_PACKAGES else "Provider",
-        "PROVIDERS_FOLDER": "backport-providers" if BACKPORT_PACKAGES else "providers",
+        "PIP_REQUIREMENTS": PROVIDERS_REQUIREMENTS[provider_details.provider_package_id],
+        "PROVIDER_TYPE": "Backport provider" if backport_packages else "Provider",
+        "PROVIDERS_FOLDER": "backport-providers" if backport_packages else "providers",
         "INSTALL_REQUIREMENTS": get_install_requirements(
-            provider_package_id=provider_package_id, backport_packages=backport_packages
+            provider_package_id=provider_details.provider_package_id, backport_packages=backport_packages
         ),
         "SETUP_REQUIREMENTS": get_setup_requirements(),
         "EXTRAS_REQUIREMENTS": get_package_extras(
-            provider_package_id=provider_package_id, backport_packages=backport_packages
+            provider_package_id=provider_details.provider_package_id, backport_packages=backport_packages
         ),
-        "PROVIDER_INFO": get_provider_info_from_provider_yaml(provider_package_id),
+        "CROSS_PROVIDERS_DEPENDENCIES_TABLE": cross_providers_dependencies_table,
+        "CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST": cross_providers_dependencies_table_rst,
+        "PIP_REQUIREMENTS_TABLE": pip_requirements_table,
+        "PIP_REQUIREMENTS_TABLE_RST": pip_requirements_table_rst,
     }
-    if update_release_notes:
-        git_cmd = get_git_command(previous_release)
-        try:
-            changes = subprocess.check_output(
-                git_cmd, cwd=source_provider_package_path, universal_newlines=True
-            )
-            changes_table = convert_git_changes_to_table(
-                changes, base_url="https://github.com/apache/airflow/commit/"
-            )
-        except subprocess.CalledProcessError:
-            # TODO(potiuk) fix me for both backport/provider package check
-            changes_table = ''
-        context["CURRENT_CHANGES_TABLE"] = changes_table
-        pip_requirements_table = convert_pip_requirements_to_table(
-            PROVIDERS_REQUIREMENTS[provider_package_id]
-        )
-        cross_providers_dependencies_table = convert_cross_package_dependencies_to_table(
-            cross_providers_dependencies,
-            backport_packages=backport_packages,
-        )
-        context["CROSS_PROVIDERS_DEPENDENCIES_TABLE"] = cross_providers_dependencies_table
-        context["PIP_REQUIREMENTS_TABLE"] = pip_requirements_table
+    return context
 
-    total, bad = check_if_classes_are_properly_named(entity_summaries)
+
+def prepare_readme_file(
+    context,
+):
+    readme_content = LICENCE
+    readme_template_name = PROVIDER_TEMPLATE_PREFIX + "README"
+    readme_content += render_template(template_name=readme_template_name, context=context, extension='.md')
+    readme_file_path = os.path.join(TARGET_PROVIDER_PACKAGES_PATH, "README.md")
+    with open(readme_file_path, "wt") as readme_file:
+        readme_file.write(readme_content)
+
+
+def update_generated_files_for_regular_package(
+    provider_package_id: str,
+    version_suffix: str,
+    update_release_notes: bool,
+    update_setup: bool,
+) -> None:
+    """
+    Updates generated files (readme, changes and/or setup.cfg/setup.py/manifest.in/provider_info)
+
+    :param provider_package_id: id of the package
+    :param version_suffix: version suffix corresponding to the version in the code
+    :param update_release_notes: whether to update release notes
+    :param update_setup: whether to update setup files
+    """
+    verify_provider_package(provider_package_id)
+    provider_details = get_provider_details(provider_package_id)
+    provider_info = get_provider_info_from_provider_yaml(provider_package_id)
+    versions: List[str] = provider_info['versions']
+    current_release_version = versions[0]
+    # previous_release = versions[1] if len(versions) > 1 else None
+    # previous_release_commit_ref = f"providers-{provider_package_id.replace('.','-')}/{previous_release}"

Review comment:
       Yep




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567693445



##########
File path: breeze
##########
@@ -1809,19 +1836,22 @@ ${CMDNAME} prepare-provider-packages [FLAGS] [PACKAGE_ID ...]
       prepared there so make sure you run prepare-provider-packages first,
       and prepare-airflow-packages second.
 
+      You can also specify --backport flag to prepare backport providers or --package-format to
+      prepare one or both types of format supported.

Review comment:
       ```suggestion
         prepare one or both types of supported formats.
   ```




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567772471



##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -747,15 +750,31 @@ def convert_git_changes_to_table(changes: str, base_url: str) -> str:
         if line == "":
             continue
         full_hash, short_hash, date, message = line.split(" ", maxsplit=3)
-        table_data.append((f"[{short_hash}]({base_url}{full_hash})", date, message))
-    return tabulate(table_data, headers=headers, tablefmt="pipe")
+        message_without_backticks = message.replace("`", "'")
+        table_data.append(
+            (
+                f"[{short_hash}]({base_url}{full_hash})"
+                if markdown
+                else f"`{short_hash} <{base_url}{full_hash}>`_",
+                date,
+                f"`{message_without_backticks}`" if markdown else f"``{message_without_backticks}``",
+            )
+        )
+    table = tabulate(table_data, headers=headers, tablefmt="pipe" if markdown else "rst")
+    header = ""
+    if not markdown:
+        header += f"\n\n{print_version}\n" + "." * len(print_version) + "\n\n"
+        release_date = table_data[0][1]
+        header += f"Latest change: {release_date}\n\n"
+    return header + table

Review comment:
       This is a table that lands in commit.rst and I think it's really eeasy to follow (those are just lists of commits). They are not supposed to be updated/modified etc - they are always automatically generted  I do not think we can make them more readable to be honest :). 
   
   See https://user-images.githubusercontent.com/595491/106393635-96531a80-63f8-11eb-8fb8-f2135f8054d8.png :) 
   
   
   
   




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-766470238


   [The Workflow run](https://github.com/apache/airflow/actions/runs/508110819) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r560629925



##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       In my opinion, this page has too much content and some are missing (but the missing ones do not need to be added now).  I like the README files. which is available in the documentation of Google libraries. https://googleapis.dev/python/firestore/latest/index.html
   It is standardized too, but contains all the information you need.
   
   1. Starting at the top. So I think it's worth adding a description of the package itself.  This information is available in the `provider.yaml` file, although so far the descriptions are not very. precise.h ttps://github.com/apache/airflow/blob/1602ec97c8d5bc7a7a8b42e850ac6c7a7030e47d/airflow/provider.yaml.schema.json#L13-L16
   I hope that we will be able to prepare descriptions in the future gradually, but so far I have made a minimum step and added a link to the product page for each provider.
   
   2. The next step is to link to the documentation, if we want to use the same file in setup.py and in the documentation. This will make it easier to find the documentation when you are in pypi.
   
   We have these links in the side menu, but this is quite a new feature and not everyone uses this menu. Especially on mobile devices, I start reading from the package description, not the side menu.
   ![Screenshot 2021-01-20 at 03 10 13](https://user-images.githubusercontent.com/12058428/105117662-2924b880-5acd-11eb-987b-445ed4334f9d.png)
   
   3.  The division into a detailed changelog and a normal changelog is strange for me. As for me, the contents of the ADDITIONAL_INFO.rst file should be a normal documentation page and have the title "Migration guide" because this is exactly what this file contains. As this file is fully human written, I don't think other scripts need to process it as well.
   
   4.  I think the detailed changelog and migration guides worth moving to new subpages, because it is not important for the new user.  This then will look similar to the Google documentation
   ![Screenshot 2021-01-20 at 03 16 53](https://user-images.githubusercontent.com/12058428/105118117-1959a400-5ace-11eb-8d74-1cdc3afe3680.png).
   When we move the guides to new pages, we will also be able to easily prepare the migration guide from Airflow 1.10, because we already have a table that facilitates these migrations. 
   
   ![Screenshot 2021-01-20 at 03 21 33](https://user-images.githubusercontent.com/12058428/105118493-d5b36a00-5ace-11eb-9346-10337d43b906.png)
   
   5. The links in the table use markdown syntax instead of rst.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-764630173






----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567715844



##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -747,15 +750,31 @@ def convert_git_changes_to_table(changes: str, base_url: str) -> str:
         if line == "":
             continue
         full_hash, short_hash, date, message = line.split(" ", maxsplit=3)
-        table_data.append((f"[{short_hash}]({base_url}{full_hash})", date, message))
-    return tabulate(table_data, headers=headers, tablefmt="pipe")
+        message_without_backticks = message.replace("`", "'")
+        table_data.append(
+            (
+                f"[{short_hash}]({base_url}{full_hash})"
+                if markdown
+                else f"`{short_hash} <{base_url}{full_hash}>`_",
+                date,
+                f"`{message_without_backticks}`" if markdown else f"``{message_without_backticks}``",
+            )
+        )
+    table = tabulate(table_data, headers=headers, tablefmt="pipe" if markdown else "rst")
+    header = ""
+    if not markdown:
+        header += f"\n\n{print_version}\n" + "." * len(print_version) + "\n\n"
+        release_date = table_data[0][1]
+        header += f"Latest change: {release_date}\n\n"
+    return header + table

Review comment:
       Have you considered using rich for rendering the table instead of using tabulate + custom headers? I think using rich table make take less code and be easier to understand 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561395893



##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       > human description on what changed in each version. Which I think CHANGELOG is a great name for.
   
   I am afraid nobody will write it by hand. Google automatically generates CHANGELOG.md files. Everyone has similar references to Github. 
   https://github.com/googleapis/python-bigquery/blob/master/CHANGELOG.md
   https://github.com/googleapis/python-bigquery/commit/b0e074f7522710886be1da2f117ea22de411b408
   To generate the changelog, they use the[googleapis/release-please](https://github.com/googleapis/python-bigquery/commit/b0e074f7522710886be1da2f117ea22de411b408) tool, which analyzes the message and generates a changelog based on it.
   
   > Release Please assumes you are using Conventional Commit messages.
   > 
   > The most important prefixes you should have in mind are:
   > 
   > fix: which represents bug fixes, and correlates to a SemVer patch.
   > feat: which represents a new feature, and correlates to a SemVer minor.
   > feat!:, or fix!:, refactor!:, etc., which represent a breaking change (indicated by the !) and will result in a SemVer major.
   
   In the first version, I think that we don't need the changes to be divided into several categories, although that would be super functional, we can limit ourselves to listing the changes. If we publish these packages regularly, each release will not have a very long list of changes.

##########
File path: airflow/providers/res.txt
##########
@@ -0,0 +1,61 @@
+./salesforce

Review comment:
       Is it needed?

##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       @kaxil For now, I can see that we have two changelogs, one with only a list of breaking changes and the detailed changelog with a list of all commits.
   http://gabby-cough.surge.sh/#changelog
   We don't have any updating guide yet.
   
   I only suggest changing the name from "Changelog" to "Upgrading guide" and from "Detailed changelog" to "Changelog" because I am concerned that we will be able to keep a changelog written by hand if we have a lot of these contributions. I only suggest changing the name from "Changelog" to "Upgrading guide" and from "Detailed changelog" to "Changelog", because I am concerned that we will be able to maintain the changelog if we have as many contributions as we do now. Over time, it will just copy selected changes that are repeated. in detailed changelog.

##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       @kaxil For now, I can see that we have two changelogs, one with only a list of breaking changes and the detailed changelog with a list of all commits.
   http://gabby-cough.surge.sh/#changelog
   We don't have any updating guide yet.
   
   I only suggest changing the name from "Changelog" to "Upgrading guide" and from "Detailed changelog" to "Changelog", because I am concerned that we will be able to maintain the changelog if we have as many contributions as we do now. After time, I suspect we'll be creating a handwritten changelog by copying only the titles of a few changes. 

##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       > (happy to name it differently if someone has a good proposal).
   
   Alternatively, we can call it "commits"

##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       > (happy to name it differently if someone has a good proposal).
   
   Alternatively, we can call it "Commits"




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-770356233


   [The Workflow run](https://github.com/apache/airflow/actions/runs/524998771) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] mik-laj commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-766278663


   > This might be out of scope for this change, but in general, when you go to Operators, then click on an operator and hit "view source" the source code is embedded in the page. Should we instead consider redirecting folks to GitHub, or adding a "View on GitHub" option? (Often I want to look at the code history, but I recognize I'm just one data point)
   
   It is intentional for the link and example to always work. Very often these examples can vary drastically from one version to another, so linking to Github can be problematic as this changes frequently.  Besides, it can be problematic because of Apache releasing policy.
   
   > Projects MUST direct outsiders towards official releases rather than `raw source repositories`, nightly builds, snapshots, release candidates, or any other similar packages. 
   
   There is one more benefit to source code in the documentation. In some cases, we have added additional annotations to make it easier to go to a specific part of the documentation. See at "[docs]" link.
   
   <img width="444" alt="Screenshot 2021-01-24 at 03 24 11" src="https://user-images.githubusercontent.com/12058428/105619525-ab75eb00-5df3-11eb-95a0-21565b7af921.png">
   


----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567720979



##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -1225,184 +1269,308 @@ def get_provider_info_from_provider_yaml(provider_package_id: str) -> Dict[str,
     return provider_info
 
 
-def update_generated_files_for_package(
+def get_backport_current_changes_table(previous_release_commit_ref: str, source_provider_package_path: str):
+    git_cmd = get_git_log_command(HEAD_OF_HTTPS_REMOTE, previous_release_commit_ref)
+    try:
+        changes = subprocess.check_output(git_cmd, cwd=source_provider_package_path, universal_newlines=True)
+        changes_table = convert_git_changes_to_table(
+            previous_release_commit_ref, changes, base_url="https://github.com/apache/airflow/commit/"
+        )
+    except subprocess.CalledProcessError:
+        changes_table = ''
+    return changes_table
+
+
+def get_version_tag(version: str, provider_package_id: str, version_suffix: str = ''):
+    return f"providers-{provider_package_id.replace('.','-')}/{version}{version_suffix}"
+
+
+def print_changes_table(changes_table):
+    syntax = Syntax(changes_table, "rst", theme="ansi_dark")
+    console = Console(width=200)
+    console.print(syntax)
+
+
+def get_all_changes_for_regular_packages(
+    versions: List[str],
     provider_package_id: str,
-    current_release_version: str,
+    source_provider_package_path: str,
     version_suffix: str,
-    imported_classes: List[str],
-    backport_packages: bool,
-    update_release_notes: bool,
-    update_setup: bool,
-) -> Tuple[int, int]:
-    """
-    Updates release notes (BACKPORT_PROVIDER_README.md/README.md) for the package.
-    Returns Tuple of total number of entities and badly named entities.
+) -> Tuple[bool, str]:
+    current_version = versions[0]
+    current_tag_no_suffix = get_version_tag(current_version, provider_package_id)
+    print(f"Checking if tag '{current_tag_no_suffix}' exist.")
+    if not subprocess.call(
+        get_git_tag_check_command(current_tag_no_suffix),
+        cwd=source_provider_package_path,
+        stderr=subprocess.DEVNULL,
+    ):
+        print(f"The tag {current_tag_no_suffix} exists.")
+        # The tag already exists
+        changes = subprocess.check_output(
+            get_git_log_command(HEAD_OF_HTTPS_REMOTE, current_tag_no_suffix),
+            cwd=source_provider_package_path,
+            universal_newlines=True,
+        )
+        if len(changes) > 0:

Review comment:
       ```suggestion
           if changes:
   ```




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567692400



##########
File path: BREEZE.rst
##########
@@ -2103,19 +2134,22 @@ This is the current syntax for  `./breeze <./breeze>`_:
         prepared there so make sure you run prepare-provider-packages first,
         and prepare-airflow-packages second.
 
+        You can also specify --backport flag to prepare backport providers or --package-format to
+        prepare one or both types of format supported.

Review comment:
       ```suggestion
           prepare one or both types of supported formats.
   ```




----------------------------------------------------------------
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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561410905



##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       @kaxil For now, I can see that we have two changelogs, one with only a list of breaking changes and the detailed changelog with a list of all commits.
   http://gabby-cough.surge.sh/#changelog
   We don't have any updating guide yet.
   
   I only suggest changing the name from "Changelog" to "Upgrading guide" and from "Detailed changelog" to "Changelog", because I am concerned that we will be able to maintain the changelog if we have as many contributions as we do now. After time, I suspect we'll be creating a handwritten changelog by copying only the titles of a few changes. 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-763025828


   Hey - especially @kaxil and @ashb - you've been waiting for this one and I think it is finally ready for review.
   
   I've refactored the original code of 'prepare_provider_packages.py" to simplify it a bit and to prepare for removal of backports in a few month's time. It should still work fine for backports (we will see it soon in CI) - but I mainly focused on getting the workflow agreed before in https://github.com/apache/airflow/issues/11425#issuecomment-749647496
   
   I separated it out into two ommits:
   * First commit smply reshufles the generated documentation - it removes the old one generated and replaces them with just CHANGELOG.rst that will be kept per-provider. 
   * The second one implements the logic that allows to prepare and release the providers in bulk.
   
   I described all the process in the dev/readme\s as usual, but this time iI also explained how you can easily debug the process step by step by manually running all the scripts. You do not have to use `breeze` to prepare provider's documentation you can simply follow the "debugging" steps explained in dev/README.md:
   
   1) import check - to check if all provider classes can be imported
   2) verification if all provider classes are properly named
   3) preparing the documentation - either for selected or for all providers that are ready for it
   3) generating setup/manifest/provider_info fikes - either for selected or for all providers that are ready for it
   4) preparing the packages
   
   Breeze encapsulates all of that in two commands, and has support for "bulk" operations on multiple providers. So for the "release" process `breeze` should be rather useful.
   
   More about how the process looks like from the release manager perspective: 
   
   1)  Preparing the release documentation and reviewing providers to release
   
   First, you can quickly see which packages should be released. When running `./breeze prepare-provider-documentation` without any parameters, it will loop through all providers and prints changes vs. the last release. Additio
   naly, if already there is a new version set for a package in 'provider.yaml' it will update the documentation for that provider and update the index.rst of the provider with the appropriate information:
   
   This is what you will see for packages that do not have a new version set in `provider.yaml` :
   
   ![Screenshot from 2021-01-19 18-35-49](https://user-images.githubusercontent.com/595491/105071656-2bfbbb00-5a85-11eb-896d-5f914b779068.png)
   
   This way you will see the summary of all those changes and providers that might be candidates for releases. The summary of changes has all the links to commits - you can click through from the terminal directly.
   
   Additionally, you will see information if all the provider classes are properly named (the step will fail here and tell you if they are not):
   
   ![Screenshot from 2021-01-19 19-02-49](https://user-images.githubusercontent.com/595491/105074804-26a06f80-5a89-11eb-8d79-151abf7e2d71.png)
   
   For packages that already have a new version in `provider.yaml` (in our case I set 2.0.0 for the upcoming release of Google provider) you will see the information that the index has been updated and "diff" of the index.rst vs. the last time it was generated:
   
   ![Screenshot from 2021-01-19 18-36-59](https://user-images.githubusercontent.com/595491/105071991-96146000-5a85-11eb-87a2-06a89e651340.png)
   
   As a release manager, you can re-run this after merging new commits and it will be correctly updated (and you will see the incremental diffs every time you update). it's safe to commit those updated index.rst files.
   
   At the end of such run, you will see a summary showing for which packages documentation was generated and for which it was skipped (or errors). Skipped is when no newer version was added to `provider.yaml` vs. the already released and tagged one:
   
   ![Screenshot from 2021-01-19 18-43-16](https://user-images.githubusercontent.com/595491/105072536-35395780-5a86-11eb-85ca-a93b52596dca.png)
   
   The documentation generated is added to index.rst in the "doc" folder, so when we release the docs, we can simply point to it from the README (rather than add the changelog to the package README). The documentation in the provider's index contains both - High Level CHANGELOG.rst content and detailed per-commit changelog. 
   
   You can see example Google Provider 2.0 docs generated this way: http://gabby-cough.surge.sh/
   
   Once you (as release manager) update versions of the providers in other provider.yaml files this 
   `./breeze prepare-provider-documentation` command can be rerun and documentation for all the providers ready to
   released will be updated automatically in-bulk (and only for those). All the providers that have the same version as already released tags, will be skipped.
   
   2) Generating the providers
   
   Similar to generating documentation, provider packages can be generated in-bulk:
   
   `./breeze  prepare-provider-packages --package-format both`
   
   The above will go through all providers and generate the providers which are ready to be released (i.e those that have updated version in provider.yaml (and hopefully documentation generated and committed in the previous step). You will see the information about skipped packages (and why they are skipped) and green confirmation for those packages that are generated:
   
   Google one:
   ![Screenshot from 2021-01-19 19-05-37](https://user-images.githubusercontent.com/595491/105074977-6bc4a180-5a89-11eb-8e29-cc1e525a09d2.png)
   
   and neo4j (it has never been generated so this is the first time it is generated):
   
   ![Screenshot from 2021-01-19 19-08-57](https://user-images.githubusercontent.com/595491/105075250-cb22b180-5a89-11eb-8187-5e7aaaad8942.png)
   
   At the end you will see the summary showing the generated/skipped/errored packages:
   
   ![Screenshot from 2021-01-19 19-09-19](https://user-images.githubusercontent.com/595491/105075289-d8d83700-5a89-11eb-9d59-6fe59bbae687.png)
   
   I think it is quite robust already, and the same code is run in the CI, so I hope it will be green soon, in the meantime - that's quite a lot of code, so probably good time to start reviewing.
   
   Unfortunately the change is significant and it's hard to split it , the best split I could do is to separate out the "docs" from the code (which I did - in separate commits). But I am also open if you have other suggestions on how to split it (but i think it ain't easy).
   
   
   
   
   
   
   


----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-767154886


   Yeah. I have to test it in "master' of my fork as it requires changes in "buid image workflow" . Once I get it finallzed (I hope tomorrow) I will link to successful build there :(


----------------------------------------------------------------
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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561397211



##########
File path: airflow/providers/res.txt
##########
@@ -0,0 +1,61 @@
+./salesforce

Review comment:
       Is it needed?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561712252



##########
File path: airflow/providers/res.txt
##########
@@ -0,0 +1,61 @@
+./salesforce

Review comment:
       Now. That's added by accident :)

##########
File path: airflow/providers/res.txt
##########
@@ -0,0 +1,61 @@
+./salesforce

Review comment:
       No. That's added by accident :)




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-763032089


   Also - here is the content of README.md file generated for Google 2.0.0 package (this is what will be available in PyPI).
   
   As discussed in #11425 the content is much shorter - it contains only the most important information and link to detailed release notes (basically to this page: http://gabby-cough.surge.sh/) . Note that those pages eventually will contain changelogss for all the previous versions as well (we will incrementally add new versions to the changelo - latest at the top).
   
   ```
   # Package apache-airflow-providers-google
   
   Release: 2.0.0
   
   **Table of contents**
   
   - [Provider package](#provider-package)
   - [Installation](#installation)
   - [PIP requirements](#pip-requirements)
   - [Cross provider package dependencies](#cross-provider-package-dependencies)
   
   ## Provider package
   
   This is a provider package for `google` provider. All classes for this provider package
   are in `airflow.providers.google` python package.
   
   You can find detailed changelog for the provider
   in the [documentation](https://airflow.apache.org/docs/apache-airflow-providers-google/stable/index.html)
   
   ## Installation
   
   NOTE!
   
   On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
   does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
   of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
   `pip install --upgrade pip==20.2.4` or, in case you use Pip 20.3, you need to add option
   `--use-deprecated legacy-resolver` to your pip install command.
   
   You can install this package on top of an existing airflow 2.* installation via
   `pip install apache-airflow-providers-google`
   
   ## PIP requirements
   
   | PIP package                          | Version required   |
   |:-------------------------------------|:-------------------|
   | `PyOpenSSL`                          |                    |
   | `google-ads`                         | `>=4.0.0,<8.0.0`   |
   | `google-api-python-client`           | `>=1.6.0,<2.0.0`   |
   | `google-auth-httplib2`               | `>=0.0.1`          |
   | `google-auth`                        | `>=1.0.0,<2.0.0`   |
   | `google-cloud-automl`                | `>=2.1.0,<3.0.0`   |
   | `google-cloud-bigquery-datatransfer` | `>=3.0.0,<4.0.0`   |
   | `google-cloud-bigtable`              | `>=1.0.0,<2.0.0`   |
   | `google-cloud-container`             | `>=0.1.1,<2.0.0`   |
   | `google-cloud-datacatalog`           | `>=3.0.0,<4.0.0`   |
   | `google-cloud-dataproc`              | `>=2.2.0,<3.0.0`   |
   | `google-cloud-dlp`                   | `>=0.11.0,<2.0.0`  |
   | `google-cloud-kms`                   | `>=2.0.0,<3.0.0`   |
   | `google-cloud-language`              | `>=1.1.1,<2.0.0`   |
   | `google-cloud-logging`               | `>=1.14.0,<2.0.0`  |
   | `google-cloud-memcache`              | `>=0.2.0`          |
   | `google-cloud-monitoring`            | `>=0.34.0,<2.0.0`  |
   | `google-cloud-os-login`              | `>=2.0.0,<3.0.0`   |
   | `google-cloud-pubsub`                | `>=2.0.0,<3.0.0`   |
   | `google-cloud-redis`                 | `>=2.0.0,<3.0.0`   |
   | `google-cloud-secret-manager`        | `>=0.2.0,<2.0.0`   |
   | `google-cloud-spanner`               | `>=1.10.0,<2.0.0`  |
   | `google-cloud-speech`                | `>=0.36.3,<2.0.0`  |
   | `google-cloud-storage`               | `>=1.30,<2.0.0`    |
   | `google-cloud-tasks`                 | `>=2.0.0,<3.0.0`   |
   | `google-cloud-texttospeech`          | `>=0.4.0,<2.0.0`   |
   | `google-cloud-translate`             | `>=1.5.0,<2.0.0`   |
   | `google-cloud-videointelligence`     | `>=1.7.0,<2.0.0`   |
   | `google-cloud-vision`                | `>=0.35.2,<2.0.0`  |
   | `grpcio-gcp`                         | `>=0.2.2`          |
   | `json-merge-patch`                   | `~=0.2`            |
   | `pandas-gbq`                         |                    |
   
   ## Cross provider package dependencies
   
   Those are dependencies that might be needed in order to use all the features of the package.
   You need to install the specified backport providers package in order to use them.
   
   You can install such cross-provider dependencies when installing from PyPI. For example:
   
   pip install apache-airflow-providers-google[amazon]
   
   
   | Dependent package                                                                             | Extra              |
   |:----------------------------------------------------------------------------------------------|:-------------------|
   | [apache-airflow-providers-amazon](https://airflow.apache.org/docs/amazon)                     | `amazon`           |
   | [apache-airflow-providers-apache-cassandra](https://airflow.apache.org/docs/apache-cassandra) | `apache.cassandra` |
   | [apache-airflow-providers-cncf-kubernetes](https://airflow.apache.org/docs/cncf-kubernetes)   | `cncf.kubernetes`  |
   | [apache-airflow-providers-facebook](https://airflow.apache.org/docs/facebook)                 | `facebook`         |
   | [apache-airflow-providers-microsoft-azure](https://airflow.apache.org/docs/microsoft-azure)   | `microsoft.azure`  |
   | [apache-airflow-providers-microsoft-mssql](https://airflow.apache.org/docs/microsoft-mssql)   | `microsoft.mssql`  |
   | [apache-airflow-providers-mysql](https://airflow.apache.org/docs/mysql)                       | `mysql`            |
   | [apache-airflow-providers-oracle](https://airflow.apache.org/docs/oracle)                     | `oracle`           |
   | [apache-airflow-providers-postgres](https://airflow.apache.org/docs/postgres)                 | `postgres`         |
   | [apache-airflow-providers-presto](https://airflow.apache.org/docs/presto)                     | `presto`           |
   | [apache-airflow-providers-salesforce](https://airflow.apache.org/docs/salesforce)             | `salesforce`       |
   | [apache-airflow-providers-sftp](https://airflow.apache.org/docs/sftp)                         | `sftp`             |
   | [apache-airflow-providers-ssh](https://airflow.apache.org/docs/ssh)                           | `ssh`              |
   ```
   


----------------------------------------------------------------
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.

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



[GitHub] [airflow] leahecole commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
leahecole commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-765732982


   Hey friends! @potiuk asked me to look at [http://gabby-cough.surge.sh/](http://gabby-cough.surge.sh/) and to share some thoughts. With the sheer volume of files in this PR, I had trouble figuring out what source files were mapping to the site, so I'm sharing my thoughts in this comment in bullet point form. 
   
   - Right now, the "Content" section at the top is a duplication of what's on the left sidebar. I'd vote for its removal, especially because I feel like Installation and requirements should be at the very top. I like what's in the sidebar though.
   - Are there any prerequisites a user must fulfill before installing this package? 
   - There may be general ones that we cover on a page for all provider packages, but is there anything that a user has to do before installing the google ones? 
   - In the "Provider package" section it says "This is a provider package for `google`" provider - there should be a "the" before the word Google. Additionally, what do y'all think about linking to the source code on the words "`google` provider"? 
   - How are we linking to other guides? (Thinking that things like this https://github.com/apache/airflow/pull/9798 belong linked)
   - Looking at one of the operators in particular, and I love the prerequisite tasks (Oh, are these perhaps the guides that are embedded in this?)
   - This might be out of scope for this change, but in general, when you go to Operators, then click on an operator and hit "view source" the source code is embedded in the page. Should we instead consider redirecting folks to GitHub, or adding a "View on GitHub" option? (Often I want to look at the code history, but I recognize I'm just one data point)
   ![4qaKTNCeTSRHBW3](https://user-images.githubusercontent.com/6719667/105557969-5a1a0e80-5cc2-11eb-99eb-40f7d25e3d96.png)
   
   - I really love the detailed changelog and that it's there. Not sure that the links are rendering correctly though. What is the difference between the changelog and the detailed changelog though? Is the changelog breaking changes only? Or just major ones? As an end user, I dont' understand why both are there.
   


----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561720939



##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       Just one comment here @kaxil @mik-laj about the naming/change process.
   
   I am not totally against naming it UPDATING.md, but I think we are confusing a bit airflow and providers release cycle.  I think we should get the naming right and content of this "manually written" parts.
   
   I believe  most of provider releases will be incremental updates - new hooks/operators added basically or new (backwards-compatible) changes in the operators. The changes that we have now in Google Providers are exception rather than rule. We only need to add the "upgrading" guide because we make backwards-incompatible changes due to 2.0 APIs. In vast majority of cases people should be able to do 'pip install --upgrade `apache-airflow-providers-<NN>` and there should be no need to follow any "updating" guide. This is the main premise of semver, which communicates breaking changes. The only time where we need to add "updating" information is when we go 1.0.0 -> 2.0.0 etc.
   
   
   Now - this means that vast majority of changes will be simply "new operator "xxx" added". That's why I think CHANGELOG.rst is a good name, and this is also a good name for the Heading in the docs.
   
   While idally indeed we should be able to automaticaly get it from semantic commits (and I am more and more convinced that we should switch to those), this is a decision that will time to discuss and vote and apply and it will take some time for people to adjust to, and we can do it as a follow-up. For now I think this will be very little overhead if we just agree between committers, that every time a new stuff is added to  provider, we ask the PR author to add entry in the CHANELOG.md and provider.yaml:
   
   1) Add new version in provider.yaml patchelevel number (for bugfixes) or minor version (for features)
   2) Add a line in CHANGELOG.txt to describe the change added.
   
   I think this makes it nicely distributed among the PR authors, and decreases a lot the work needed by release manager (@kaxil :) - then PR authors and commiters will take care about updating the versions and the role of release manager will be just to review whether everything was included. Remember that release manager will have to sometimes release 10-20 providers at a time and mostly providers that the release manager knows very little about, so the less work is put on the shoulders of that person, the better. 
   
   Let me know WDYT @kaxil @mik-laj and @ashb (I am mentioning @ashb here for the release management part mainly).
   
   




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r562094509



##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       I will make it separate page then next to index.rst and link to it.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-770266877


   [The Workflow run](https://github.com/apache/airflow/actions/runs/523850363) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561415429



##########
File path: airflow/providers/res.txt
##########
@@ -0,0 +1,61 @@
+./salesforce
+./celery
+./amazon

Review comment:
       yeah not sure what this file is about




----------------------------------------------------------------
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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r567716932



##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -766,40 +785,50 @@ def convert_pip_requirements_to_table(requirements: Iterable[str]) -> str:
         if found:
             package = found.group(1)
             version_required = found.group(2)
-            table_data.append((package, version_required))
+            if version_required == "":
+                version_required = ""
+            else:

Review comment:
       Can you explain this? Should `if version_required != ""` yield the same result?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-770402030


   [The Workflow run](https://github.com/apache/airflow/actions/runs/525472603) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561710341



##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       > > feat!:, or fix!:, refactor!:, etc., which represent a breaking change (indicated by the !) and will result in a SemVer major.
   
   I think automatically generating nice changelog is something we can think about in the future. And if you look at the commit changes now, there are many we would have to exclude because they are just refactors/global updates. For now I thin we should generate changelog manually (and every time someone makes a new feature or breaking change - during review, we should ask the person to add entry to the changelog. I will update the changelogs manually with the first release, but from the 1.0.1 onwards, this is just a matter that all committers will pay attention to it and make sure that Changelog is updated.
   
   As the next step we could introduce the semantic commits and automated changelog generation. But this would require discussion, voting and likely some automation to make sure we follow it. I would like to decouple this from this change.
   

##########
File path: airflow/providers/res.txt
##########
@@ -0,0 +1,61 @@
+./salesforce

Review comment:
       Now. That's added by accident :)

##########
File path: airflow/providers/res.txt
##########
@@ -0,0 +1,61 @@
+./salesforce

Review comment:
       No. That's added by accident :)

##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       Just one comment here @kaxil @mik-laj about the naming/change process.
   
   I am not totally against naming it UPDATING.md, but I think we are confusing a bit airflow and providers release cycle.  I think we should get the naming right and content of this "manually written" parts.
   
   I believe  most of provider releases will be incremental updates - new hooks/operators added basically or new (backwards-compatible) changes in the operators. The changes that we have now in Google Providers are exception rather than rule. We only need to add the "upgrading" guide because we make backwards-incompatible changes due to 2.0 APIs. In vast majority of cases people should be able to do 'pip install --upgrade `apache-airflow-providers-<NN>` and there should be no need to follow any "updating" guide. This is the main premise of semver, which communicates breaking changes. The only time where we need to add "updating" information is when we go 1.0.0 -> 2.0.0 etc.
   
   
   Now - this means that vast majority of changes will be simply "new operator "xxx" added". That's why I think CHANGELOG.rst is a good name, and this is also a good name for the Heading in the docs.
   
   While idally indeed we should be able to automaticaly get it from semantic commits (and I am more and more convinced that we should switch to those), this is a decision that will time to discuss and vote and apply and it will take some time for people to adjust to, and we can do it as a follow-up. For now I think this will be very little overhead if we just agree between committers, that every time a new stuff is added to  provider, we ask the PR author to add entry in the CHANELOG.md and provider.yaml:
   
   1) Add new version in provider.yaml patchelevel number (for bugfixes) or minor version (for features)
   2) Add a line in CHANGELOG.txt to describe the change added.
   
   I think this makes it nicely distributed among the PR authors, and decreases a lot the work needed by release manager (@kaxil :) - then PR authors and commiters will take care about updating the versions and the role of release manager will be just to review whether everything was included. Remember that release manager will have to sometimes release 10-20 providers at a time and mostly providers that the release manager knows very little about, so the less work is put on the shoulders of that person, the better. 
   
   Let me know WDYT @kaxil @mik-laj and @ashb (I am mentioning @ashb here for the release management part mainly).
   
   

##########
File path: BREEZE.rst
##########
@@ -2056,6 +2065,28 @@ This is the current syntax for  `./breeze <./breeze>`_:
 
   Flags:
 
+  -S, --version-suffix-for-pypi SUFFIX
+          Adds optional suffix to the version in the generated backport package. It can be used

Review comment:
       For regular providers too

##########
File path: BREEZE.rst
##########
@@ -2056,6 +2065,28 @@ This is the current syntax for  `./breeze <./breeze>`_:
 
   Flags:
 
+  -S, --version-suffix-for-pypi SUFFIX
+          Adds optional suffix to the version in the generated backport package. It can be used
+          to generate rc1/rc2 ... versions of the packages to be uploaded to PyPI.
+
+  -N, --version-suffix-for-svn SUFFIX
+          Adds optional suffix to the generated names of package. It can be used to generate
+          rc1/rc2 ... versions of the packages to be uploaded to SVN.
+
+  --package-format PACKAGE_FORMAT
+
+          Chooses format of packages to prepare.
+
+          One of:
+
+                 wheel,sdist,both
+
+          Default: 

Review comment:
       There is - wheel (though I might want to change it to both). I will see why :)

##########
File path: airflow/providers/CHANGELOG.rst
##########
@@ -0,0 +1,25 @@
+ .. 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.
+
+
+Changelog
+---------
+
+1.0.0
+.....
+
+Initial version of the provider.

Review comment:
       Hmmmmm . Strange indeed :)

##########
File path: airflow/providers/README.md
##########
@@ -1,28 +0,0 @@
-<!--
- 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.
- -->
-
-# Airflow Providers

Review comment:
       Yes. All the READMEs  are not needed any more - they are dynamically generated while package is prepared and there is no point in keeping them in repo. 

##########
File path: airflow/providers/amazon/README.md
##########
@@ -1,483 +0,0 @@
-<!--
- 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.
- -->
-
-
-# Package apache-airflow-providers-amazon

Review comment:
       Yeah. Only during release. We do not really need them here. There is very little value in keeping them in the repo, they will be available in PyPI. But If you really think they are needed, I can add them back. I just thought - since we already have the information there in setup.py, dependencies.json , the value of having it in the repo is very little. But I can be convince otherwise..

##########
File path: airflow/providers/res.txt
##########
@@ -0,0 +1,61 @@
+./salesforce
+./celery
+./amazon

Review comment:
       Debug file :)

##########
File path: dev/provider_packages/BACKPORT_PROVIDER_README_TEMPLATE.md.jinja2
##########
@@ -88,6 +95,6 @@ You can install such cross-provider dependencies when installing from PyPI. For
 pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
 ```
 
-{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE }}
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE | safe }}

Review comment:
       Avoids escaping < > into &gt; &lt;

##########
File path: dev/provider_packages/BACKPORT_PROVIDER_README_TEMPLATE.md.jinja2
##########
@@ -88,6 +95,6 @@ You can install such cross-provider dependencies when installing from PyPI. For
 pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
 ```
 
-{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE }}
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE | safe }}

Review comment:
       Avoids escaping < > into `&gt;` `&lt;`

##########
File path: dev/provider_packages/copy_provider_package_sources.py
##########
@@ -767,27 +781,38 @@ def do_refactor(self, in_process: bool = False) -> None:  # noqa
         self.refactor_kubernetes_pod_operator()
         # In order to debug Bowler - set in_process to True
         self.qry.execute(write=True, silent=False, interactive=False, in_process=in_process)
+        print("[green]OK[/]")
+
+
+def get_parser():
+    cli_parser = argparse.ArgumentParser(
+        description="Copies sources and optionally refactors provider code to be Airflow 1.10 compatible.",
+        formatter_class=argparse.RawTextHelpFormatter,
+    )
+    cli_parser.add_argument(
+        "--backports",
+        action='store_true',
+        help=textwrap.dedent("Includes refactoring to prepare backport packages rather than regular ones"),
+    )
+    cli_parser.add_argument(
+        "--debug",
+        action='store_true',
+        help=textwrap.dedent(
+            "Run bowler refactoring in single process. Makes it debuggable with regular"
+            " IDE debugger (much slower)"
+        ),
+    )
+    return cli_parser
 
 
 if __name__ == '__main__':
-    BACKPORT_PACKAGES = os.getenv('BACKPORT_PACKAGES') == "true"
-    in_process = False
-    if len(sys.argv) > 1:
-        if sys.argv[1] in ['--help', '-h']:
-            print()
-            print("Refactors provider packages to be Airflow 1.10 compatible.")
-            print()
-            print(f"Usage: {sys.argv[0]} [--debug] | [-h] | [--help]")
-            print()
-            print("You can use --debug flag in order to run bowler refactoring in process.")
-            print("This allows you to debug bowler process as usual using your IDE debugger")
-            print("Otherwise it heavily uses multi-processing and is next-to-impossible to debug")
-            print()
-            print("Note - Bowler is also a lot slower in this mode.")
-            print()
-            sys.exit(0)
-        if sys.argv[1] == '--debug':
-            in_process = True
-    copy_provider_sources()
-    if BACKPORT_PACKAGES:
+    parser = get_parser()
+    args = parser.parse_args()
+
+    in_process = args.debug
+    copy_provider_sources(args.backports)
+    os.chdir(get_target_providers_folder())
+    if args.backports:
+        print("\nRefactoring code to be Airflow 1.10 - compatible\n")
         RefactorBackportPackages().do_refactor(in_process=in_process)
+        print("\n[green]Refactored code successfully[/]\n")

Review comment:
       :D

##########
File path: dev/provider_packages/prepare_provider_packages.py
##########
@@ -1225,184 +1195,272 @@ def get_provider_info_from_provider_yaml(provider_package_id: str) -> Dict[str,
     return provider_info
 
 
-def update_generated_files_for_package(
+def get_backport_current_changes_table(previous_release_commit_ref: str, source_provider_package_path: str):
+    git_cmd = get_git_log_command(previous_release_commit_ref)
+    try:
+        changes = subprocess.check_output(git_cmd, cwd=source_provider_package_path, universal_newlines=True)
+        changes_table = convert_git_changes_to_table(
+            previous_release_commit_ref, changes, base_url="https://github.com/apache/airflow/commit/"
+        )
+    except subprocess.CalledProcessError:
+        changes_table = ''
+    return changes_table
+
+
+def get_version_tag(version: str, provider_package_id: str, version_suffix: str):
+    return f"providers-{provider_package_id.replace('.','-')}/{version}{version_suffix}"
+
+
+def print_changes_table(changes_table):
+    syntax = Syntax(changes_table, "rst", theme="light")
+    console = Console(width=200)
+    console.print(syntax)
+
+
+def get_all_changes_for_regular_packages(
+    versions: List[str],
     provider_package_id: str,
-    current_release_version: str,
+    source_provider_package_path: str,
     version_suffix: str,
-    imported_classes: List[str],
-    backport_packages: bool,
-    update_release_notes: bool,
-    update_setup: bool,
-) -> Tuple[int, int]:
-    """
-    Updates release notes (BACKPORT_PROVIDER_README.md/README.md) for the package.
-    Returns Tuple of total number of entities and badly named entities.
+) -> Tuple[bool, str]:
+    current_version = versions[0]
+    current_tag = get_version_tag(current_version, provider_package_id, version_suffix)
+    print(f"Checking if tag `{current_tag}` exists.")
+    if not subprocess.call(
+        get_git_tag_check_command(current_tag),
+        cwd=source_provider_package_path,
+        stderr=subprocess.DEVNULL,
+    ):
+        print(f"The tag {current_tag} exists.")
+        # The tag already exists
+        changes = subprocess.check_output(
+            get_git_log_command(current_tag, 'HEAD'),
+            cwd=source_provider_package_path,
+            universal_newlines=True,
+        )
+        if len(changes) > 0:
+            print(
+                f"[yellow]The provider {provider_package_id} has changes"
+                f" since last release but version is not updated[/]"
+            )
+            print()
+            print(
+                f'Please update version in `airflow/providers/{provider_package_id.replace("-","/")}/'
+                'provider.yaml` to prepare release.\n'
+            )
+            changes_table = convert_git_changes_to_table(
+                "UNKNOWN", changes, base_url="https://github.com/apache/airflow/commit/", markdown=False
+            )
+            print_changes_table(changes_table)
+            return False, changes_table
+        else:
+            print(f"No changes for {provider_package_id}")
+            return False, ""
+    print("The tag does not exist. ")
+    if len(versions) == 1:
+        print("This provider has never been released!\n")
+    else:
+        print(f"The {provider_package_id} is ready to be released!\n")
+    current_version = 'HEAD'
+    changes_table = ''
+    print_version = versions[0]
+    for version in versions[1:]:
+        version_tag = get_version_tag(version, provider_package_id, version_suffix)
+        changes = subprocess.check_output(
+            get_git_log_command(version_tag, current_version),
+            cwd=source_provider_package_path,
+            universal_newlines=True,
+        )
+        changes_table += convert_git_changes_to_table(
+            print_version, changes, base_url="https://github.com/apache/airflow/commit/", markdown=False
+        )
+        current_version = version
+        print_version = version
+    print_changes_table(changes_table)
+    return True, changes_table
+
+
+def get_provider_details(provider_package_id: str) -> ProviderPackageDetails:
+    return ProviderPackageDetails(
+        provider_package_id=provider_package_id,
+        full_package_name=f"airflow.providers.{provider_package_id}",
+        source_provider_package_path=get_source_package_path(provider_package_id),
+        documentation_provider_package_path=get_documentation_package_path(provider_package_id),
+    )
 
-    :param provider_package_id: id of the package
-    :param current_release_version: release version:
-    :param version_suffix: version suffix corresponding to the version in the code
-    :param imported_classes - entities that have been imported from providers
-    :param backport_packages: whether to prepare regular (False) or backport (True) packages
-    :param update_release_notes: whether to update release notes
-    :param update_setup: whether to update setup files
 
-    :return: Tuple of total/bad number of entities
-    """
-    verify_provider_package(provider_package_id)
-    full_package_name = f"airflow.providers.{provider_package_id}"
-    source_provider_package_path = get_source_package_path(provider_package_id)
-    entity_summaries = get_package_class_summary(full_package_name, imported_classes)
-    past_releases = get_all_releases(
-        provider_package_path=source_provider_package_path, backport_packages=backport_packages
+def get_provider_jinja_context(
+    provider_details: ProviderPackageDetails,
+    current_release_version: str,
+    version_suffix: str,
+    backport_packages: bool,
+):
+    verify_provider_package(provider_details.provider_package_id)
+    cross_providers_dependencies = get_cross_provider_dependent_packages(
+        provider_package_id=provider_details.provider_package_id
     )
-    current_release_version, previous_release = check_if_release_version_ok(
-        past_releases, current_release_version, backport_packages
+    release_version_no_leading_zeros = strip_leading_zeros(current_release_version)
+    pip_requirements_table = convert_pip_requirements_to_table(
+        PROVIDERS_REQUIREMENTS[provider_details.provider_package_id]
     )
-    cross_providers_dependencies = get_cross_provider_dependent_packages(
-        provider_package_id=provider_package_id
+    pip_requirements_table_rst = convert_pip_requirements_to_table(
+        PROVIDERS_REQUIREMENTS[provider_details.provider_package_id], markdown=False
     )
-    previous_release = get_previous_release_info(
-        previous_release_version=previous_release,
-        past_releases=past_releases,
-        current_release_version=current_release_version,
+    cross_providers_dependencies_table = convert_cross_package_dependencies_to_table(
+        cross_providers_dependencies,
+        backport_packages=backport_packages,
     )
-    release_version_no_leading_zeros = (
-        strip_leading_zeros_in_calver(current_release_version)
-        if backport_packages
-        else current_release_version
+    cross_providers_dependencies_table_rst = convert_cross_package_dependencies_to_table(
+        cross_providers_dependencies, backport_packages=backport_packages, markdown=False
     )
     context: Dict[str, Any] = {
         "ENTITY_TYPES": list(EntityType),
         "README_FILE": "BACKPORT_PROVIDER_README.md" if backport_packages else "README.md",
-        "PROVIDER_PACKAGE_ID": provider_package_id,
-        "PACKAGE_PIP_NAME": get_pip_package_name(provider_package_id, backport_packages),
-        "FULL_PACKAGE_NAME": full_package_name,
-        "PROVIDER_PATH": full_package_name.replace(".", "/"),
+        "PROVIDER_PACKAGE_ID": provider_details.provider_package_id,
+        "PACKAGE_PIP_NAME": get_pip_package_name(
+            provider_details.provider_package_id, backport_packages=backport_packages
+        ),
+        "FULL_PACKAGE_NAME": provider_details.full_package_name,
+        "PROVIDER_PATH": provider_details.full_package_name.replace(".", "/"),
         "RELEASE": current_release_version,
         "RELEASE_NO_LEADING_ZEROS": release_version_no_leading_zeros,
         "VERSION_SUFFIX": version_suffix,
-        "ADDITIONAL_INFO": get_additional_package_info(provider_package_path=source_provider_package_path),
+        "ADDITIONAL_INFO": get_additional_package_info(
+            provider_package_path=provider_details.source_provider_package_path
+        ),
+        "CHANGELOG": get_changelog_for_package(
+            provider_package_path=provider_details.source_provider_package_path
+        ),
         "CROSS_PROVIDERS_DEPENDENCIES": cross_providers_dependencies,
-        "PIP_REQUIREMENTS": PROVIDERS_REQUIREMENTS[provider_package_id],
-        "PROVIDER_TYPE": "Backport provider" if BACKPORT_PACKAGES else "Provider",
-        "PROVIDERS_FOLDER": "backport-providers" if BACKPORT_PACKAGES else "providers",
+        "PIP_REQUIREMENTS": PROVIDERS_REQUIREMENTS[provider_details.provider_package_id],
+        "PROVIDER_TYPE": "Backport provider" if backport_packages else "Provider",
+        "PROVIDERS_FOLDER": "backport-providers" if backport_packages else "providers",
         "INSTALL_REQUIREMENTS": get_install_requirements(
-            provider_package_id=provider_package_id, backport_packages=backport_packages
+            provider_package_id=provider_details.provider_package_id, backport_packages=backport_packages
         ),
         "SETUP_REQUIREMENTS": get_setup_requirements(),
         "EXTRAS_REQUIREMENTS": get_package_extras(
-            provider_package_id=provider_package_id, backport_packages=backport_packages
+            provider_package_id=provider_details.provider_package_id, backport_packages=backport_packages
         ),
-        "PROVIDER_INFO": get_provider_info_from_provider_yaml(provider_package_id),
+        "CROSS_PROVIDERS_DEPENDENCIES_TABLE": cross_providers_dependencies_table,
+        "CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST": cross_providers_dependencies_table_rst,
+        "PIP_REQUIREMENTS_TABLE": pip_requirements_table,
+        "PIP_REQUIREMENTS_TABLE_RST": pip_requirements_table_rst,
     }
-    if update_release_notes:
-        git_cmd = get_git_command(previous_release)
-        try:
-            changes = subprocess.check_output(
-                git_cmd, cwd=source_provider_package_path, universal_newlines=True
-            )
-            changes_table = convert_git_changes_to_table(
-                changes, base_url="https://github.com/apache/airflow/commit/"
-            )
-        except subprocess.CalledProcessError:
-            # TODO(potiuk) fix me for both backport/provider package check
-            changes_table = ''
-        context["CURRENT_CHANGES_TABLE"] = changes_table
-        pip_requirements_table = convert_pip_requirements_to_table(
-            PROVIDERS_REQUIREMENTS[provider_package_id]
-        )
-        cross_providers_dependencies_table = convert_cross_package_dependencies_to_table(
-            cross_providers_dependencies,
-            backport_packages=backport_packages,
-        )
-        context["CROSS_PROVIDERS_DEPENDENCIES_TABLE"] = cross_providers_dependencies_table
-        context["PIP_REQUIREMENTS_TABLE"] = pip_requirements_table
+    return context
 
-    total, bad = check_if_classes_are_properly_named(entity_summaries)
+
+def prepare_readme_file(
+    context,
+):
+    readme_content = LICENCE
+    readme_template_name = PROVIDER_TEMPLATE_PREFIX + "README"
+    readme_content += render_template(template_name=readme_template_name, context=context, extension='.md')
+    readme_file_path = os.path.join(TARGET_PROVIDER_PACKAGES_PATH, "README.md")
+    with open(readme_file_path, "wt") as readme_file:
+        readme_file.write(readme_content)
+
+
+def update_generated_files_for_regular_package(
+    provider_package_id: str,
+    version_suffix: str,
+    update_release_notes: bool,
+    update_setup: bool,
+) -> None:
+    """
+    Updates generated files (readme, changes and/or setup.cfg/setup.py/manifest.in/provider_info)
+
+    :param provider_package_id: id of the package
+    :param version_suffix: version suffix corresponding to the version in the code
+    :param update_release_notes: whether to update release notes
+    :param update_setup: whether to update setup files
+    """
+    verify_provider_package(provider_package_id)
+    provider_details = get_provider_details(provider_package_id)
+    provider_info = get_provider_info_from_provider_yaml(provider_package_id)
+    versions: List[str] = provider_info['versions']
+    current_release_version = versions[0]
+    # previous_release = versions[1] if len(versions) > 1 else None
+    # previous_release_commit_ref = f"providers-{provider_package_id.replace('.','-')}/{previous_release}"

Review comment:
       Yep

##########
File path: BREEZE.rst
##########
@@ -2056,6 +2065,28 @@ This is the current syntax for  `./breeze <./breeze>`_:
 
   Flags:
 
+  -S, --version-suffix-for-pypi SUFFIX
+          Adds optional suffix to the version in the generated backport package. It can be used
+          to generate rc1/rc2 ... versions of the packages to be uploaded to PyPI.
+
+  -N, --version-suffix-for-svn SUFFIX
+          Adds optional suffix to the generated names of package. It can be used to generate
+          rc1/rc2 ... versions of the packages to be uploaded to SVN.
+
+  --package-format PACKAGE_FORMAT
+
+          Chooses format of packages to prepare.
+
+          One of:
+
+                 wheel,sdist,both
+
+          Default: 

Review comment:
       Changed to both. The problem was with extra 's' in variable name

##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       Also one small comment here.
   
   Looking at https://apache-airflow.slack.com/archives/CCR6P6JRL/p1611225089006700 - this is why I think we should have the detailed commit log (as a separate document linked from the index). Users are sometimes aware of the PRs they are interested in and want to know in which version they were included. And it's not very easy to find it out. We will have a separate set of tags for each provider, so for some people, it would be great to have it in this nicely formatted document rather than having to run some complex git commands to find out. 
   
   

##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       I will make it separate page then next to index.rst and link to it.

##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       I will also add a few hyperlinks here and there, so that we will be able to freely jump between those different pages - 
   * README in PyPI, 
   * index.rst in docs
   * commit changes in docs
   
   
   Those will be interlinked with version numbers so that we will be able to jump from PYPI 2.0.0 package straight to detailed commit log for 2.0.0 and 2.0.0 Changelog (and all the cross-links between those).
   
   




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#issuecomment-763123970


   [The Workflow run](https://github.com/apache/airflow/actions/runs/496899311) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561420778



##########
File path: dev/provider_packages/copy_provider_package_sources.py
##########
@@ -767,27 +781,38 @@ def do_refactor(self, in_process: bool = False) -> None:  # noqa
         self.refactor_kubernetes_pod_operator()
         # In order to debug Bowler - set in_process to True
         self.qry.execute(write=True, silent=False, interactive=False, in_process=in_process)
+        print("[green]OK[/]")
+
+
+def get_parser():
+    cli_parser = argparse.ArgumentParser(
+        description="Copies sources and optionally refactors provider code to be Airflow 1.10 compatible.",
+        formatter_class=argparse.RawTextHelpFormatter,
+    )
+    cli_parser.add_argument(
+        "--backports",
+        action='store_true',
+        help=textwrap.dedent("Includes refactoring to prepare backport packages rather than regular ones"),
+    )
+    cli_parser.add_argument(
+        "--debug",
+        action='store_true',
+        help=textwrap.dedent(
+            "Run bowler refactoring in single process. Makes it debuggable with regular"
+            " IDE debugger (much slower)"
+        ),
+    )
+    return cli_parser
 
 
 if __name__ == '__main__':
-    BACKPORT_PACKAGES = os.getenv('BACKPORT_PACKAGES') == "true"
-    in_process = False
-    if len(sys.argv) > 1:
-        if sys.argv[1] in ['--help', '-h']:
-            print()
-            print("Refactors provider packages to be Airflow 1.10 compatible.")
-            print()
-            print(f"Usage: {sys.argv[0]} [--debug] | [-h] | [--help]")
-            print()
-            print("You can use --debug flag in order to run bowler refactoring in process.")
-            print("This allows you to debug bowler process as usual using your IDE debugger")
-            print("Otherwise it heavily uses multi-processing and is next-to-impossible to debug")
-            print()
-            print("Note - Bowler is also a lot slower in this mode.")
-            print()
-            sys.exit(0)
-        if sys.argv[1] == '--debug':
-            in_process = True
-    copy_provider_sources()
-    if BACKPORT_PACKAGES:
+    parser = get_parser()
+    args = parser.parse_args()
+
+    in_process = args.debug
+    copy_provider_sources(args.backports)
+    os.chdir(get_target_providers_folder())
+    if args.backports:
+        print("\nRefactoring code to be Airflow 1.10 - compatible\n")
         RefactorBackportPackages().do_refactor(in_process=in_process)
+        print("\n[green]Refactored code successfully[/]\n")

Review comment:
       ```suggestion
       copy_provider_sources(args.backports)
       os.chdir(get_target_providers_folder())
       if args.backports:
           print("\nRefactoring code to be Airflow 1.10 - compatible\n")
           RefactorBackportPackages().do_refactor(in_process=args.debug)
           print("\n[green]Refactored code successfully[/]\n")
   ```




----------------------------------------------------------------
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.

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



[GitHub] [airflow] potiuk commented on a change in pull request #13767: Implement provider versioning tools

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561753209



##########
File path: BREEZE.rst
##########
@@ -2056,6 +2065,28 @@ This is the current syntax for  `./breeze <./breeze>`_:
 
   Flags:
 
+  -S, --version-suffix-for-pypi SUFFIX
+          Adds optional suffix to the version in the generated backport package. It can be used
+          to generate rc1/rc2 ... versions of the packages to be uploaded to PyPI.
+
+  -N, --version-suffix-for-svn SUFFIX
+          Adds optional suffix to the generated names of package. It can be used to generate
+          rc1/rc2 ... versions of the packages to be uploaded to SVN.
+
+  --package-format PACKAGE_FORMAT
+
+          Chooses format of packages to prepare.
+
+          One of:
+
+                 wheel,sdist,both
+
+          Default: 

Review comment:
       Changed to both. The problem was with extra 's' in variable name




----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #13767: Implement target provider versioning tools

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #13767:
URL: https://github.com/apache/airflow/pull/13767#discussion_r561431446



##########
File path: dev/provider_packages/PROVIDER_INDEX_TEMPLATE.rst.jinja2
##########
@@ -0,0 +1,84 @@
+{#
+ 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 WHEN PREPARING PACKAGES.
+
+ IF YOU WANT TO MODIFY IT, YOU SHOULD MODIFY THE TEMPLATE
+ `PROVIDER_INDEX_TEMPLATE.rst.jinja2` IN the `provider_packages` DIRECTORY
+
+#}
+
+Package {{ PACKAGE_PIP_NAME }}
+------------------------------------------------------
+
+Release: {{ RELEASE }}{{ VERSION_SUFFIX }}
+
+Provider package
+----------------
+
+This is a provider package for ``{{PROVIDER_PACKAGE_ID}}`` provider. All classes for this provider package
+are in ``{{FULL_PACKAGE_NAME}}`` python package.
+
+Installation
+------------
+
+.. note::
+
+    On November 2020, new version of PIP (20.3) has been released with a new, 2020 resolver. This resolver
+    does not yet work with Apache Airflow and might lead to errors in installation - depends on your choice
+    of extras. In order to install Airflow you need to either downgrade pip to version 20.2.4
+    ``pip install --upgrade pip==20.2.4`` or, in case you use Pip 20.3, you need to add option
+    ``--use-deprecated legacy-resolver`` to your pip install command.
+
+
+You can install this package on top of an existing airflow 2.* installation via
+``pip install {{PACKAGE_PIP_NAME}}``
+{%- if PIP_REQUIREMENTS %}
+
+PIP requirements
+----------------
+
+{{  PIP_REQUIREMENTS_TABLE_RST | safe}}
+{%- endif %}
+{%- if CROSS_PROVIDERS_DEPENDENCIES %}
+
+Cross provider package dependencies
+-----------------------------------
+
+Those are dependencies that might be needed in order to use all the features of the package.
+You need to install the specified backport providers package in order to use them.
+
+You can install such cross-provider dependencies when installing from PyPI. For example:
+
+.. code-block:: bash
+
+    pip install {{ PACKAGE_PIP_NAME }}[{{ CROSS_PROVIDERS_DEPENDENCIES[0] }}]
+
+
+{{ CROSS_PROVIDERS_DEPENDENCIES_TABLE_RST | safe }}
+
+{%- endif %}

Review comment:
       Oh yes, I agree with that @mik-laj -- My assumption was detailed changelog was updating guide but looks like it is just list of commits.
   
   So yeah, **Changelog** -> **Upgrading guide** and **Detailed changelog** -> **Changelog** . 




----------------------------------------------------------------
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.

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