You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by ep...@apache.org on 2023/03/07 20:46:13 UTC

[airflow] branch v2-5-test updated (8ab8ac84b5 -> 45caa5ec14)

This is an automated email from the ASF dual-hosted git repository.

ephraimanierobi pushed a change to branch v2-5-test
in repository https://gitbox.apache.org/repos/asf/airflow.git


    from 8ab8ac84b5 Make skipping provider tests part of selective-checks (#29216)
     new 5bf2a4a829 Fix circular imports when airflow starts (#29494)
     new dd1673d9be Fix import cycle starting from airflow/__main__.py (#29523)
     new 83779d8e5e Revert "Google have re-worked their docs site ane appear not publish objects.inv (#28834)" (#29543)
     new 45caa5ec14 Aggressively cache entry points in process (#29625)

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 airflow/__main__.py                             | 13 ++++++--
 airflow/models/xcom.py                          |  5 ++-
 airflow/providers_manager.py                    |  6 ++--
 airflow/settings.py                             |  5 ++-
 airflow/utils/entry_points.py                   | 44 ++++++++++++++-----------
 airflow/utils/state.py                          |  3 --
 airflow/utils/timezone.py                       | 12 +++++--
 docs/conf.py                                    | 35 ++++++++++++++++++++
 docs/exts/docs_build/third_party_inventories.py | 24 ++++++++++++++
 docs/spelling_wordlist.txt                      |  8 -----
 tests/cli/commands/test_db_command.py           |  4 +--
 tests/conftest.py                               |  7 ++++
 tests/models/test_taskinstance.py               |  5 +--
 tests/utils/test_entry_points.py                |  6 ++--
 14 files changed, 132 insertions(+), 45 deletions(-)


[airflow] 01/04: Fix circular imports when airflow starts (#29494)

Posted by ep...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ephraimanierobi pushed a commit to branch v2-5-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit 5bf2a4a829e9505120a964087f4224cc66fef206
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Tue Feb 14 00:51:20 2023 +0100

    Fix circular imports when airflow starts (#29494)
    
    * Fix circular imports when airflow starts from scratch
    
    When airflow CLI is started without generated config file the CLI
    fails with circular imports. Some of the util classes imported
    during such start were importing "constants" (in fact variables)
    from the setttings and this happened before settings were fully
    initialized.
    
    This PR moves all relevant top-level imports to be local imports
    and moves initialization of the State class with settings-defined
    colors to initialization() method in order to avoid those circular
    imports.
    
    * Update airflow/models/xcom.py
    
    Co-authored-by: Tzu-ping Chung <ur...@gmail.com>
    
    ---------
    
    Co-authored-by: Tzu-ping Chung <ur...@gmail.com>
    (cherry picked from commit 47b67f13da667de4eb69233404d4d5a33574dd58)
---
 airflow/models/xcom.py                |  5 ++++-
 airflow/settings.py                   |  5 ++++-
 airflow/utils/state.py                |  3 ---
 airflow/utils/timezone.py             | 12 ++++++++++--
 tests/cli/commands/test_db_command.py |  4 ++--
 tests/models/test_taskinstance.py     |  5 +++--
 6 files changed, 23 insertions(+), 11 deletions(-)

diff --git a/airflow/models/xcom.py b/airflow/models/xcom.py
index 6294fa3d7f..831f05e6c9 100644
--- a/airflow/models/xcom.py
+++ b/airflow/models/xcom.py
@@ -769,7 +769,10 @@ class LazyXComAccess(collections.abc.Sequence):
             yield self._query
             return
 
-        session = settings.Session()
+        Session = getattr(settings, "Session", None)
+        if Session is None:
+            raise RuntimeError("Session must be set before!")
+        session = Session()
         try:
             yield self._query.with_session(session)
         finally:
diff --git a/airflow/settings.py b/airflow/settings.py
index 537c49141a..dcf0513d56 100644
--- a/airflow/settings.py
+++ b/airflow/settings.py
@@ -39,6 +39,7 @@ from airflow.exceptions import RemovedInAirflow3Warning
 from airflow.executors import executor_constants
 from airflow.logging_config import configure_logging
 from airflow.utils.orm_event_handlers import setup_event_handlers
+from airflow.utils.state import State
 
 if TYPE_CHECKING:
     from airflow.www.utils import UIAlert
@@ -408,7 +409,7 @@ def dispose_orm():
     global engine
     global Session
 
-    if Session:
+    if Session is not None:  # type: ignore[truthy-function]
         Session.remove()
         Session = None
     if engine:
@@ -567,6 +568,8 @@ def initialize():
     import_local_settings()
     global LOGGING_CLASS_PATH
     LOGGING_CLASS_PATH = configure_logging()
+    State.state_color.update(STATE_COLORS)
+
     configure_adapters()
     # The webservers import this file from models.py with the default settings.
     configure_orm()
diff --git a/airflow/utils/state.py b/airflow/utils/state.py
index 6558af9f2a..6f89174bd6 100644
--- a/airflow/utils/state.py
+++ b/airflow/utils/state.py
@@ -19,8 +19,6 @@ from __future__ import annotations
 
 from enum import Enum
 
-from airflow.settings import STATE_COLORS
-
 
 class TaskInstanceState(str, Enum):
     """
@@ -122,7 +120,6 @@ class State:
         TaskInstanceState.SCHEDULED: "tan",
         TaskInstanceState.DEFERRED: "mediumpurple",
     }
-    state_color.update(STATE_COLORS)  # type: ignore
 
     @classmethod
     def color(cls, state):
diff --git a/airflow/utils/timezone.py b/airflow/utils/timezone.py
index 062e7ad955..bbe38aba2e 100644
--- a/airflow/utils/timezone.py
+++ b/airflow/utils/timezone.py
@@ -24,8 +24,6 @@ import pendulum
 from dateutil.relativedelta import relativedelta
 from pendulum.datetime import DateTime
 
-from airflow.settings import TIMEZONE
-
 # UTC time zone as a tzinfo instance.
 utc = pendulum.tz.timezone("UTC")
 
@@ -104,6 +102,8 @@ def convert_to_utc(value: dt.datetime | None) -> DateTime | None:
         return value
 
     if not is_localized(value):
+        from airflow.settings import TIMEZONE
+
         value = pendulum.instance(value, TIMEZONE)
 
     return pendulum.instance(value.astimezone(utc))
@@ -133,6 +133,8 @@ def make_aware(value: dt.datetime | None, timezone: dt.tzinfo | None = None) ->
     :return: localized datetime in settings.TIMEZONE or timezone
     """
     if timezone is None:
+        from airflow.settings import TIMEZONE
+
         timezone = TIMEZONE
 
     if not value:
@@ -168,6 +170,8 @@ def make_naive(value, timezone=None):
     :return: naive datetime
     """
     if timezone is None:
+        from airflow.settings import TIMEZONE
+
         timezone = TIMEZONE
 
     # Emulate the behavior of astimezone() on Python < 3.6.
@@ -191,6 +195,8 @@ def datetime(*args, **kwargs):
     :return: datetime.datetime
     """
     if "tzinfo" not in kwargs:
+        from airflow.settings import TIMEZONE
+
         kwargs["tzinfo"] = TIMEZONE
 
     return dt.datetime(*args, **kwargs)
@@ -203,6 +209,8 @@ def parse(string: str, timezone=None) -> DateTime:
     :param string: time string
     :param timezone: the timezone
     """
+    from airflow.settings import TIMEZONE
+
     return pendulum.parse(string, tz=timezone or TIMEZONE, strict=False)  # type: ignore
 
 
diff --git a/tests/cli/commands/test_db_command.py b/tests/cli/commands/test_db_command.py
index 53c62e84cf..17d7f49da4 100644
--- a/tests/cli/commands/test_db_command.py
+++ b/tests/cli/commands/test_db_command.py
@@ -285,7 +285,7 @@ class TestCLIDBClean:
         coerced to tz-aware with default timezone
         """
         timestamp = "2021-01-01 00:00:00"
-        with patch("airflow.utils.timezone.TIMEZONE", pendulum.timezone(timezone)):
+        with patch("airflow.settings.TIMEZONE", pendulum.timezone(timezone)):
             args = self.parser.parse_args(["db", "clean", "--clean-before-timestamp", f"{timestamp}", "-y"])
             db_command.cleanup_tables(args)
         run_cleanup_mock.assert_called_once_with(
@@ -304,7 +304,7 @@ class TestCLIDBClean:
         When tz included in the string then default timezone should not be used.
         """
         timestamp = "2021-01-01 00:00:00+03:00"
-        with patch("airflow.utils.timezone.TIMEZONE", pendulum.timezone(timezone)):
+        with patch("airflow.settings.TIMEZONE", pendulum.timezone(timezone)):
             args = self.parser.parse_args(["db", "clean", "--clean-before-timestamp", f"{timestamp}", "-y"])
             db_command.cleanup_tables(args)
 
diff --git a/tests/models/test_taskinstance.py b/tests/models/test_taskinstance.py
index bb2b676a59..9b126c68c9 100644
--- a/tests/models/test_taskinstance.py
+++ b/tests/models/test_taskinstance.py
@@ -70,6 +70,7 @@ from airflow.operators.python import PythonOperator
 from airflow.sensors.base import BaseSensorOperator
 from airflow.sensors.python import PythonSensor
 from airflow.serialization.serialized_objects import SerializedBaseOperator
+from airflow.settings import TIMEZONE
 from airflow.stats import Stats
 from airflow.ti_deps.dep_context import DepContext
 from airflow.ti_deps.dependencies_deps import REQUEUEABLE_DEPS, RUNNING_DEPS
@@ -2307,13 +2308,13 @@ class TestTaskInstance:
         )
         context = ti.get_template_context()
         with pytest.deprecated_call():
-            assert context["execution_date"] == pendulum.DateTime(2021, 9, 6, tzinfo=timezone.TIMEZONE)
+            assert context["execution_date"] == pendulum.DateTime(2021, 9, 6, tzinfo=TIMEZONE)
         with pytest.deprecated_call():
             assert context["next_ds"] == "2021-09-07"
         with pytest.deprecated_call():
             assert context["next_ds_nodash"] == "20210907"
         with pytest.deprecated_call():
-            assert context["next_execution_date"] == pendulum.DateTime(2021, 9, 7, tzinfo=timezone.TIMEZONE)
+            assert context["next_execution_date"] == pendulum.DateTime(2021, 9, 7, tzinfo=TIMEZONE)
         with pytest.deprecated_call():
             assert context["prev_ds"] is None, "Does not make sense for custom timetable"
         with pytest.deprecated_call():


[airflow] 02/04: Fix import cycle starting from airflow/__main__.py (#29523)

Posted by ep...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ephraimanierobi pushed a commit to branch v2-5-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit dd1673d9be635cda2b3f39a0ffe2ef6a06b8b54f
Author: Niko Oliveira <on...@amazon.com>
AuthorDate: Tue Feb 14 13:24:15 2023 -0800

    Fix import cycle starting from airflow/__main__.py (#29523)
    
    (cherry picked from commit 81f07274b9cd9369a1024eb8b0ad5ee6058202f0)
---
 airflow/__main__.py | 13 +++++++++++--
 1 file changed, 11 insertions(+), 2 deletions(-)

diff --git a/airflow/__main__.py b/airflow/__main__.py
index 6114534e1c..b355a6a52d 100644
--- a/airflow/__main__.py
+++ b/airflow/__main__.py
@@ -24,12 +24,21 @@ import os
 
 import argcomplete
 
+# The configuration module initializes and validates the conf object as a side effect the first
+# time it is imported. If it is not imported before importing the settings module, the conf
+# object will then be initted/validated as a side effect of it being imported in settings,
+# however this can cause issues since those modules are very tightly coupled and can
+# very easily cause import cycles in the conf init/validate code (since downstream code from
+# those functions likely import settings).
+# Therefore importing configuration early (as the first airflow import) avoids
+# any possible import cycles with settings downstream.
+from airflow import configuration
 from airflow.cli import cli_parser
-from airflow.configuration import conf
 
 
 def main():
-    """Main executable function"""
+    """Main executable function."""
+    conf = configuration.conf
     if conf.get("core", "security") == "kerberos":
         os.environ["KRB5CCNAME"] = conf.get("kerberos", "ccache")
         os.environ["KRB5_KTNAME"] = conf.get("kerberos", "keytab")


[airflow] 04/04: Aggressively cache entry points in process (#29625)

Posted by ep...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ephraimanierobi pushed a commit to branch v2-5-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit 45caa5ec146760c273fe96b05135b8f2e786bee8
Author: Tzu-ping Chung <ur...@gmail.com>
AuthorDate: Sat Feb 25 13:36:14 2023 +0800

    Aggressively cache entry points in process (#29625)
    
    (cherry picked from commit 9f51845fdc305e2f5847584e984278c906f9f293)
---
 airflow/providers_manager.py     |  6 ++++--
 airflow/utils/entry_points.py    | 44 +++++++++++++++++++++++-----------------
 tests/conftest.py                |  7 +++++++
 tests/utils/test_entry_points.py |  6 +++---
 4 files changed, 39 insertions(+), 24 deletions(-)

diff --git a/airflow/providers_manager.py b/airflow/providers_manager.py
index 6088e3b373..a0fe51510e 100644
--- a/airflow/providers_manager.py
+++ b/airflow/providers_manager.py
@@ -31,6 +31,8 @@ from functools import wraps
 from time import perf_counter
 from typing import TYPE_CHECKING, Any, Callable, MutableMapping, NamedTuple, TypeVar, cast
 
+from packaging.utils import canonicalize_name
+
 from airflow.exceptions import AirflowOptionalProviderFeatureException
 from airflow.typing_compat import Literal
 from airflow.utils import yaml
@@ -454,8 +456,8 @@ class ProvidersManager(LoggingMixin):
         and verifies only the subset of fields that are needed at runtime.
         """
         for entry_point, dist in entry_points_with_dist("apache_airflow_provider"):
-            package_name = dist.metadata["name"]
-            if self._provider_dict.get(package_name) is not None:
+            package_name = canonicalize_name(dist.metadata["name"])
+            if package_name in self._provider_dict:
                 continue
             log.debug("Loading %s from package %s", entry_point, package_name)
             version = dist.version
diff --git a/airflow/utils/entry_points.py b/airflow/utils/entry_points.py
index 41ea38845f..b3f145110f 100644
--- a/airflow/utils/entry_points.py
+++ b/airflow/utils/entry_points.py
@@ -16,10 +16,10 @@
 # under the License.
 from __future__ import annotations
 
+import collections
+import functools
 import logging
-from typing import Iterator
-
-from packaging.utils import canonicalize_name
+from typing import Iterator, Tuple
 
 try:
     import importlib_metadata as metadata
@@ -28,26 +28,32 @@ except ImportError:
 
 log = logging.getLogger(__name__)
 
+EPnD = Tuple[metadata.EntryPoint, metadata.Distribution]
 
-def entry_points_with_dist(group: str) -> Iterator[tuple[metadata.EntryPoint, metadata.Distribution]]:
-    """Retrieve entry points of the given group.
-
-    This is like the ``entry_points()`` function from importlib.metadata,
-    except it also returns the distribution the entry_point was loaded from.
 
-    :param group: Filter results to only this entrypoint group
-    :return: Generator of (EntryPoint, Distribution) objects for the specified groups
-    """
-    loaded: set[str] = set()
+@functools.lru_cache(maxsize=None)
+def _get_grouped_entry_points() -> dict[str, list[EPnD]]:
+    mapping: dict[str, list[EPnD]] = collections.defaultdict(list)
     for dist in metadata.distributions():
         try:
-            key = canonicalize_name(dist.metadata["Name"])
-            if key in loaded:
-                continue
-            loaded.add(key)
             for e in dist.entry_points:
-                if e.group != group:
-                    continue
-                yield e, dist
+                mapping[e.group].append((e, dist))
         except Exception as e:
             log.warning("Error when retrieving package metadata (skipping it): %s, %s", dist, e)
+    return mapping
+
+
+def entry_points_with_dist(group: str) -> Iterator[EPnD]:
+    """Retrieve entry points of the given group.
+
+    This is like the ``entry_points()`` function from ``importlib.metadata``,
+    except it also returns the distribution the entry point was loaded from.
+
+    Note that this may return multiple distributions to the same package if they
+    are loaded from different ``sys.path`` entries. The caller site should
+    implement appropriate deduplication logic if needed.
+
+    :param group: Filter results to only this entrypoint group
+    :return: Generator of (EntryPoint, Distribution) objects for the specified groups
+    """
+    return iter(_get_grouped_entry_points()[group])
diff --git a/tests/conftest.py b/tests/conftest.py
index 945ece6e67..bdaec7da0f 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -856,3 +856,10 @@ def reset_logging_config():
 
     logging_config = import_string(settings.LOGGING_CLASS_PATH)
     logging.config.dictConfig(logging_config)
+
+
+@pytest.fixture(autouse=True)
+def _clear_entry_point_cache():
+    from airflow.utils.entry_points import _get_grouped_entry_points
+
+    _get_grouped_entry_points.cache_clear()
diff --git a/tests/utils/test_entry_points.py b/tests/utils/test_entry_points.py
index de4843dbaa..22537245fc 100644
--- a/tests/utils/test_entry_points.py
+++ b/tests/utils/test_entry_points.py
@@ -45,6 +45,6 @@ class MockMetadata:
 def test_entry_points_with_dist():
     entries = list(entry_points_with_dist("group_x"))
 
-    # The second "dist2" is ignored. Only "group_x" entries are loaded.
-    assert [dist.metadata["Name"] for _, dist in entries] == ["dist1", "Dist2"]
-    assert [ep.name for ep, _ in entries] == ["a", "e"]
+    # Only "group_x" entries are loaded. Distributions are not deduplicated.
+    assert [dist.metadata["Name"] for _, dist in entries] == ["dist1", "Dist2", "dist2"]
+    assert [ep.name for ep, _ in entries] == ["a", "e", "g"]


[airflow] 03/04: Revert "Google have re-worked their docs site ane appear not publish objects.inv (#28834)" (#29543)

Posted by ep...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ephraimanierobi pushed a commit to branch v2-5-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit 83779d8e5ef1db447f9db26221d8276a69bb1777
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Tue Feb 14 21:15:11 2023 +0100

    Revert "Google have re-worked their docs site ane appear not publish objects.inv (#28834)" (#29543)
    
    This reverts commit 44d93bea3adbe9429b6fdee14de31f80ac51c3bb.
    
    Google re-published the .inv files after deleting them in January.
    
    Details in https://github.com/googleapis/google-cloud-python/issues/10814
    
    (cherry picked from commit c2c27c34fc76db7bface94affacc82b0d1a34d92)
---
 docs/conf.py                                    | 35 +++++++++++++++++++++++++
 docs/exts/docs_build/third_party_inventories.py | 24 +++++++++++++++++
 docs/spelling_wordlist.txt                      |  8 ------
 3 files changed, 59 insertions(+), 8 deletions(-)

diff --git a/docs/conf.py b/docs/conf.py
index 5f5e4c3a5f..f55b954d5c 100644
--- a/docs/conf.py
+++ b/docs/conf.py
@@ -634,6 +634,41 @@ intersphinx_mapping = {
         "sqlalchemy",
     ]
 }
+if PACKAGE_NAME in ("apache-airflow-providers-google", "apache-airflow"):
+    intersphinx_mapping.update(
+        {
+            pkg_name: (
+                f"{THIRD_PARTY_INDEXES[pkg_name]}/",
+                (f"{INVENTORY_CACHE_DIR}/{pkg_name}/objects.inv",),
+            )
+            for pkg_name in [
+                "google-api-core",
+                "google-cloud-automl",
+                "google-cloud-bigquery",
+                "google-cloud-bigquery-datatransfer",
+                "google-cloud-bigquery-storage",
+                "google-cloud-bigtable",
+                "google-cloud-container",
+                "google-cloud-core",
+                "google-cloud-datacatalog",
+                "google-cloud-datastore",
+                "google-cloud-dlp",
+                "google-cloud-kms",
+                "google-cloud-language",
+                "google-cloud-monitoring",
+                "google-cloud-pubsub",
+                "google-cloud-redis",
+                "google-cloud-spanner",
+                "google-cloud-speech",
+                "google-cloud-storage",
+                "google-cloud-tasks",
+                "google-cloud-texttospeech",
+                "google-cloud-translate",
+                "google-cloud-videointelligence",
+                "google-cloud-vision",
+            ]
+        }
+    )
 
 # -- Options for sphinx.ext.viewcode -------------------------------------------
 # See: https://www.sphinx-doc.org/es/master/usage/extensions/viewcode.html
diff --git a/docs/exts/docs_build/third_party_inventories.py b/docs/exts/docs_build/third_party_inventories.py
index 720af261b5..7ac33a5333 100644
--- a/docs/exts/docs_build/third_party_inventories.py
+++ b/docs/exts/docs_build/third_party_inventories.py
@@ -27,4 +27,28 @@ THIRD_PARTY_INDEXES = {
     "python": "https://docs.python.org/3",
     "requests": "https://requests.readthedocs.io/en/stable",
     "sqlalchemy": "https://docs.sqlalchemy.org/en/latest",
+    "google-api-core": "https://googleapis.dev/python/google-api-core/latest",
+    "google-cloud-automl": "https://googleapis.dev/python/automl/latest",
+    "google-cloud-bigquery": "https://googleapis.dev/python/bigquery/latest",
+    "google-cloud-bigquery-datatransfer": "https://googleapis.dev/python/bigquerydatatransfer/latest",
+    "google-cloud-bigquery-storage": "https://googleapis.dev/python/bigquerystorage/latest",
+    "google-cloud-bigtable": "https://googleapis.dev/python/bigtable/latest",
+    "google-cloud-container": "https://googleapis.dev/python/container/latest",
+    "google-cloud-core": "https://googleapis.dev/python/google-cloud-core/latest",
+    "google-cloud-datacatalog": "https://googleapis.dev/python/datacatalog/latest",
+    "google-cloud-datastore": "https://googleapis.dev/python/datastore/latest",
+    "google-cloud-dlp": "https://googleapis.dev/python/dlp/latest",
+    "google-cloud-kms": "https://googleapis.dev/python/cloudkms/latest",
+    "google-cloud-language": "https://googleapis.dev/python/language/latest",
+    "google-cloud-monitoring": "https://googleapis.dev/python/monitoring/latest",
+    "google-cloud-pubsub": "https://googleapis.dev/python/pubsub/latest",
+    "google-cloud-redis": "https://googleapis.dev/python/redis/latest",
+    "google-cloud-spanner": "https://googleapis.dev/python/spanner/latest",
+    "google-cloud-speech": "https://googleapis.dev/python/speech/latest",
+    "google-cloud-storage": "https://googleapis.dev/python/storage/latest",
+    "google-cloud-tasks": "https://googleapis.dev/python/cloudtasks/latest",
+    "google-cloud-texttospeech": "https://googleapis.dev/python/texttospeech/latest",
+    "google-cloud-translate": "https://googleapis.dev/python/translation/latest",
+    "google-cloud-videointelligence": "https://googleapis.dev/python/videointelligence/latest",
+    "google-cloud-vision": "https://googleapis.dev/python/vision/latest",
 }
diff --git a/docs/spelling_wordlist.txt b/docs/spelling_wordlist.txt
index 5a62b70939..3c3b855133 100644
--- a/docs/spelling_wordlist.txt
+++ b/docs/spelling_wordlist.txt
@@ -850,11 +850,8 @@ LineItem
 lineterminator
 linter
 linux
-ListColumnSpecsPager
-ListDatasetsPager
 ListGenerator
 ListInfoTypesResponse
-ListTableSpecsPager
 Liveness
 liveness
 livy
@@ -1352,8 +1349,6 @@ ssm
 Stackdriver
 stackdriver
 stacktrace
-Standarization
-StartManualTransferRunsResponse
 starttls
 StatefulSet
 StatefulSets
@@ -1420,7 +1415,6 @@ symlink
 symlinking
 symlinks
 sync'ed
-SynthesizeSpeechResponse
 sys
 syspath
 Systemd
@@ -1492,8 +1486,6 @@ tooltip
 tooltips
 traceback
 tracebacks
-TransferConfig
-TransferRun
 travis
 triage
 triaging