Skip to content

Commit

Permalink
Automatically use utf8mb3_general_ci collation for mysql (#17729)
Browse files Browse the repository at this point in the history
The index size is too big in case utf8mb4 is used as encoding
for MySQL database. We already had `sql_engine_collation_for_ids`
configuration to allow the id fields to use different collation,
but the user had to set it up manually in case of a failure to
create a db and it was not obvious, not discoverable and rather
clumsy.

Since this is really only a problem with MySQL the easy solution
is to force this parameter to utf8mb3_general_ci for all mysql
databases. It has no negative consequences, really as all
relevant IDs are ASCII anyway.

Related: #17603
  • Loading branch information
potiuk authored Aug 22, 2021
1 parent 6511ce7 commit ccaff5a
Show file tree
Hide file tree
Showing 7 changed files with 68 additions and 12 deletions.
7 changes: 4 additions & 3 deletions airflow/config_templates/config.yml
Original file line number Diff line number Diff line change
Expand Up @@ -81,9 +81,10 @@
- name: sql_engine_collation_for_ids
description: |
Collation for ``dag_id``, ``task_id``, ``key`` columns in case they have different encoding.
This is particularly useful in case of mysql with utf8mb4 encoding because
primary keys for XCom table has too big size and ``sql_engine_collation_for_ids`` should
be set to ``utf8mb3_general_ci``.
By default this collation is the same as the database collation, however for ``mysql`` and ``mariadb``
the default is ``utf8mb3_general_ci`` so that the index sizes of our index keys will not exceed
the maximum size of allowed index when collation is set to ``utf8mb4`` variant
(see https://github.com/apache/airflow/pull/17603#issuecomment-901121618).
version_added: 2.0.0
type: string
example: ~
Expand Down
7 changes: 4 additions & 3 deletions airflow/config_templates/default_airflow.cfg
Original file line number Diff line number Diff line change
Expand Up @@ -63,9 +63,10 @@ sql_alchemy_conn = sqlite:///{AIRFLOW_HOME}/airflow.db
sql_engine_encoding = utf-8

# Collation for ``dag_id``, ``task_id``, ``key`` columns in case they have different encoding.
# This is particularly useful in case of mysql with utf8mb4 encoding because
# primary keys for XCom table has too big size and ``sql_engine_collation_for_ids`` should
# be set to ``utf8mb3_general_ci``.
# By default this collation is the same as the database collation, however for ``mysql`` and ``mariadb``
# the default is ``utf8mb3_general_ci`` so that the index sizes of our index keys will not exceed
# the maximum size of allowed index when collation is set to ``utf8mb4`` variant
# (see https://github.com/apache/airflow/pull/17603#issuecomment-901121618).
# sql_engine_collation_for_ids =

# If SqlAlchemy should pool database connections.
Expand Down
13 changes: 13 additions & 0 deletions airflow/models/base.py
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,19 @@ def get_id_collation_args():
if collation:
return {'collation': collation}
else:
# Automatically use utf8mb3_general_ci collation for mysql
# This is backwards-compatible. All our IDS are ASCII anyway so even if
# we migrate from previously installed database with different collation and we end up mixture of
# COLLATIONS, it's not a problem whatsoever (and we keep it small enough so that our indexes
# for MYSQL will not exceed the maximum index size.
#
# See https://github.com/apache/airflow/pull/17603#issuecomment-901121618.
#
# We cannot use session/dialect as at this point we are trying to determine the right connection
# parameters, so we use the connection
conn = conf.get('core', 'sql_alchemy_conn', fallback='')
if conn.startswith('mysql') or conn.startswith("mariadb"):
return {'collation': 'utf8mb3_general_ci'}
return {}


Expand Down
8 changes: 6 additions & 2 deletions docs/apache-airflow/howto/set-up-database.rst
Original file line number Diff line number Diff line change
Expand Up @@ -154,14 +154,18 @@ In the example below, a database ``airflow_db`` and user with username ``airflo

.. code-block:: sql
CREATE DATABASE airflow_db CHARACTER SET utf8 COLLATE utf8_general_ci;
CREATE DATABASE airflow_db CHARACTER SET utf8 COLLATE utf8mb4_unicode_ci;
CREATE USER 'airflow_user' IDENTIFIED BY 'airflow_pass';
GRANT ALL PRIVILEGES ON airflow_db.* TO 'airflow_user';
.. note::

The database must use a UTF-8 character set
The database must use a UTF-8 character set. A small caveat that you must be aware of is that utf8 in newer versions of MySQL is really utf8mb4 which
causes Airflow indexes to grow too large (see https://github.com/apache/airflow/pull/17603#issuecomment-901121618). Therefore as of Airflow 2.2
all MySQL databases have ``sql_engine_collation_for_ids`` set automatically to ``utf8mb3_general_ci`` (unless you override it). This might
lead to a mixture of collation ids for id fields in Airflow Database, but it has no negative consequences since all relevant IDs in Airflow use
ASCII characters only.

We rely on more strict ANSI SQL settings for MySQL in order to have sane defaults.
Make sure to have specified ``explicit_defaults_for_timestamp=1`` option under ``[mysqld]`` section
Expand Down
1 change: 0 additions & 1 deletion scripts/ci/docker-compose/backend-mysql.yml
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ services:
environment:
- BACKEND=mysql
- AIRFLOW__CORE__SQL_ALCHEMY_CONN=mysql://root@mysql/airflow?charset=utf8mb4
- AIRFLOW__CORE__SQL_ENGINE_COLLATION_FOR_IDS=utf8mb3_general_ci
- AIRFLOW__CELERY__RESULT_BACKEND=db+mysql://root@mysql/airflow?charset=utf8mb4
- AIRFLOW__CORE__EXECUTOR=LocalExecutor
depends_on:
Expand Down
3 changes: 0 additions & 3 deletions scripts/in_container/entrypoint_ci.sh
Original file line number Diff line number Diff line change
Expand Up @@ -39,9 +39,6 @@ echo
echo "Airflow home: ${AIRFLOW_HOME}"
echo "Airflow sources: ${AIRFLOW_SOURCES}"
echo "Airflow core SQL connection: ${AIRFLOW__CORE__SQL_ALCHEMY_CONN:=}"
if [[ -n "${AIRFLOW__CORE__SQL_ENGINE_COLLATION_FOR_IDS=}" ]]; then
echo "Airflow collation for IDs: ${AIRFLOW__CORE__SQL_ENGINE_COLLATION_FOR_IDS}"
fi

echo

Expand Down
41 changes: 41 additions & 0 deletions tests/sensors/test_base.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,15 @@

import unittest
from datetime import timedelta
from unittest import mock
from unittest.mock import Mock, patch

import pytest
from freezegun import freeze_time

from airflow.exceptions import AirflowException, AirflowRescheduleException, AirflowSensorTimeout
from airflow.models import DagBag, TaskInstance, TaskReschedule
from airflow.models.base import get_id_collation_args
from airflow.models.dag import DAG
from airflow.operators.dummy import DummyOperator
from airflow.sensors.base import BaseSensorOperator, poke_mode_only
Expand Down Expand Up @@ -655,3 +657,42 @@ def test_poke_mode_only_bad_poke(self):
sensor = DummyPokeOnlySensor(task_id='foo', mode='poke', poke_changes_mode=True, dag=self.dag)
with pytest.raises(ValueError):
sensor.poke({})


class TestCollation(unittest.TestCase):
@mock.patch.dict(
'os.environ',
AIRFLOW__CORE__SQL_ALCHEMY_CONN='postgres://host/the_database',
)
def test_collation_empty_on_non_mysql(self):
assert {} == get_id_collation_args()

@mock.patch.dict(
'os.environ',
AIRFLOW__CORE__SQL_ALCHEMY_CONN='mysql://host/the_database',
)
def test_collation_set_on_mysql(self):
assert {"collation": "utf8mb3_general_ci"} == get_id_collation_args()

@mock.patch.dict(
'os.environ',
AIRFLOW__CORE__SQL_ALCHEMY_CONN='mysql+pymsql://host/the_database',
)
def test_collation_set_on_mysql_with_pymsql(self):
assert {"collation": "utf8mb3_general_ci"} == get_id_collation_args()

@mock.patch.dict(
'os.environ',
AIRFLOW__CORE__SQL_ALCHEMY_CONN='mysql://host/the_database',
AIRFLOW__CORE__SQL_ENGINE_COLLATION_FOR_IDS='ascii',
)
def test_collation_override_on_non_mysql(self):
assert {"collation": "ascii"} == get_id_collation_args()

@mock.patch.dict(
'os.environ',
AIRFLOW__CORE__SQL_ALCHEMY_CONN='postgres://host/the_database',
AIRFLOW__CORE__SQL_ENGINE_COLLATION_FOR_IDS='ascii',
)
def test_collation_override_on_mysql(self):
assert {"collation": "ascii"} == get_id_collation_args()

0 comments on commit ccaff5a

Please sign in to comment.