You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2022/06/17 13:59:29 UTC

[GitHub] [airflow] flrn77 opened a new issue, #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on upgrade migration-job

flrn77 opened a new issue, #24526:
URL: https://github.com/apache/airflow/issues/24526

   ### Apache Airflow version
   
   2.3.2 (latest released)
   
   ### What happened
   
   Upgrade Airflow 2.2.3 or 2.2.5 -> 2.3.2 fails on migration-job.
   
   **first time upgrade execution:**
   ```
   Referencing column 'task_id' and referenced column 'task_id' in foreign key constraint 'task_map_task_instance_fkey' are incompatible.")
   [SQL: 
   CREATE TABLE task_map (
       dag_id VARCHAR(250) COLLATE utf8mb3_bin NOT NULL, 
       task_id VARCHAR(250) COLLATE utf8mb3_bin NOT NULL, 
       run_id VARCHAR(250) COLLATE utf8mb3_bin NOT NULL, 
       map_index INTEGER NOT NULL, 
       length INTEGER NOT NULL, 
       `keys` JSON, 
       PRIMARY KEY (dag_id, task_id, run_id, map_index), 
       CONSTRAINT task_map_length_not_negative CHECK (length >= 0), 
       CONSTRAINT task_map_task_instance_fkey FOREIGN KEY(dag_id, task_id, run_id, map_index) REFERENCES task_instance (dag_id, task_id, run_id, map_index) ON DELETE CASCADE
   )
   ]
   ```
   
   **after the first failed execution (should be due to the first failed execution):**
   ```
   Can't DROP 'task_reschedule_ti_fkey'; check that column/key exists")
   [SQL: ALTER TABLE task_reschedule DROP FOREIGN KEY task_reschedule_ti_fkey[]
   ```
   
   ### What you think should happen instead
   
   The migration-job shouldn't fail ;)
   
   ### How to reproduce
   
   Everytime in my environment just need to create a snapshot from last working DB-Snapshot (Airflow Version 2.2.3)
   and then deploy Airflow 2.3.2.
   I can update in between to 2.2.5 but ran into the same issue by update to 2.3.2.
   
   ### Operating System
   
   Debian GNU/Linux 10 (buster) - apache/airflow:2.3.2-python3.8 (hub.docker.com)
   
   ### Versions of Apache Airflow Providers
   
   apache-airflow-providers-amazon==2.4.0
   apache-airflow-providers-celery==2.1.0
   apache-airflow-providers-cncf-kubernetes==2.2.0
   apache-airflow-providers-docker==2.3.0
   apache-airflow-providers-elasticsearch==2.1.0
   apache-airflow-providers-ftp==2.0.1
   apache-airflow-providers-google==6.2.0
   apache-airflow-providers-grpc==2.0.1
   apache-airflow-providers-hashicorp==2.1.1
   apache-airflow-providers-http==2.0.1
   apache-airflow-providers-imap==2.0.1
   apache-airflow-providers-microsoft-azure==3.4.0
   apache-airflow-providers-mysql==2.1.1
   apache-airflow-providers-odbc==2.0.1
   apache-airflow-providers-postgres==2.4.0
   apache-airflow-providers-redis==2.0.1
   apache-airflow-providers-sendgrid==2.0.1
   apache-airflow-providers-sftp==2.3.0
   apache-airflow-providers-slack==4.1.0
   apache-airflow-providers-sqlite==2.0.1
   apache-airflow-providers-ssh==2.3.0
   apache-airflow-providers-tableau==2.1.4
   
   ### Deployment
   
   Official Apache Airflow Helm Chart
   
   ### Deployment details
   
   - K8s Rev: v1.21.12-eks-a64ea69
   - helm chart version: 1.6.0
   - Database: AWS RDS MySQL 8.0.28
   
   ### Anything else
   
   Full error Log **first** execution:
   ```
   /home/airflow/.local/lib/python3.8/site-packages/airflow/configuration.py:529: DeprecationWarning: The auth_backend option in [api[] has been renamed to auth_backends - the old setting has been used, but please update your config.
     option = self._get_option_from_config_file(deprecated_key, deprecated_section, key, kwargs, section)
   /home/airflow/.local/lib/python3.8/site-packages/airflow/configuration.py:356: FutureWarning: The auth_backends setting in [api[] has had airflow.api.auth.backend.session added in the running config, which is needed by the UI. Please update your config before Apache Airflow 3.0.
     warnings.warn(
   DB: mysql+mysqldb://airflow:***@test-airflow2-db-blue.fsgfsdcfds76.eu-central-1.rds.amazonaws.com:3306/airflow
   Performing upgrade with database mysql+mysqldb://airflow:***@test-airflow2-db-blue.fsgfsdcfds76.eu-central-1.rds.amazonaws.com:3306/airflow
   [2022-06-17 12:19:59,724[] {db.py:920} WARNING - Found 33 duplicates in table task_fail.  Will attempt to move them.
   [2022-06-17 12:36:18,813[] {db.py:1448} INFO - Creating tables
   INFO  [alembic.runtime.migration[] Context impl MySQLImpl.
   INFO  [alembic.runtime.migration[] Will assume non-transactional DDL.
   INFO  [alembic.runtime.migration[] Running upgrade be2bfac3da23 -> c381b21cb7e4, Create a ``session`` table to store web session data
   INFO  [alembic.runtime.migration[] Running upgrade c381b21cb7e4 -> 587bdf053233, Add index for ``dag_id`` column in ``job`` table.
   INFO  [alembic.runtime.migration[] Running upgrade 587bdf053233 -> 5e3ec427fdd3, Increase length of email and username in ``ab_user`` and ``ab_register_user`` table to ``256`` characters
   INFO  [alembic.runtime.migration[] Running upgrade 5e3ec427fdd3 -> 786e3737b18f, Add ``timetable_description`` column to DagModel for UI.
   INFO  [alembic.runtime.migration[] Running upgrade 786e3737b18f -> f9da662e7089, Add ``LogTemplate`` table to track changes to config values ``log_filename_template``
   INFO  [alembic.runtime.migration[] Running upgrade f9da662e7089 -> e655c0453f75, Add ``map_index`` column to TaskInstance to identify task-mapping,
   and a ``task_map`` table to track mapping values from XCom.
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/engine/base.py", line 1705, in _execute_context
       self.dialect.do_execute(
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/engine/default.py", line 716, in do_execute
       cursor.execute(statement, parameters)
     File "/home/airflow/.local/lib/python3.8/site-packages/MySQLdb/cursors.py", line 206, in execute
       res = self._query(query)
     File "/home/airflow/.local/lib/python3.8/site-packages/MySQLdb/cursors.py", line 319, in _query
       db.query(q)
     File "/home/airflow/.local/lib/python3.8/site-packages/MySQLdb/connections.py", line 254, in query
       _mysql.connection.query(self, query)
   MySQLdb._exceptions.OperationalError: (3780, "Referencing column 'task_id' and referenced column 'task_id' in foreign key constraint 'task_map_task_instance_fkey' are incompatible.")
   The above exception was the direct cause of the following exception:
   Traceback (most recent call last):
     File "/home/airflow/.local/bin/airflow", line 8, in <module>
       sys.exit(main())
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/__main__.py", line 38, in main
       args.func(args)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/cli/cli_parser.py", line 51, in command
       return func(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/cli.py", line 99, in wrapper
       return f(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/cli/commands/db_command.py", line 82, in upgradedb
       db.upgradedb(to_revision=to_revision, from_revision=from_revision, show_sql_only=args.show_sql_only)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/session.py", line 71, in wrapper
       return func(*args, session=session, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/db.py", line 1449, in upgradedb
       command.upgrade(config, revision=to_revision or 'heads')
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/command.py", line 322, in upgrade
       script.run_env()
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/script/base.py", line 569, in run_env
       util.load_python_file(self.dir, "env.py")
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/util/pyfiles.py", line 94, in load_python_file
       module = load_module_py(module_id, path)
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/util/pyfiles.py", line 110, in load_module_py
       spec.loader.exec_module(module)  # type: ignore
     File "<frozen importlib._bootstrap_external>", line 843, in exec_module
     File "<frozen importlib._bootstrap>", line 219, in _call_with_frames_removed
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/migrations/env.py", line 107, in <module>
       run_migrations_online()
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/migrations/env.py", line 101, in run_migrations_online
       context.run_migrations()
     File "<string>", line 8, in run_migrations
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/runtime/environment.py", line 853, in run_migrations
       self.get_context().run_migrations(**kw)
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/runtime/migration.py", line 623, in run_migrations
       step.migration_fn(**kw)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/migrations/versions/0100_2_3_0_add_taskmap_and_map_id_on_taskinstance.py", line 75, in upgrade
       op.create_table(
     File "<string>", line 8, in create_table
     File "<string>", line 3, in create_table
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/operations/ops.py", line 1254, in create_table
       return operations.invoke(op)
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/operations/base.py", line 394, in invoke
       return fn(self, operation)
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/operations/toimpl.py", line 114, in create_table
       operations.impl.create_table(table)
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/ddl/impl.py", line 354, in create_table
       self._exec(schema.CreateTable(table))
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/ddl/impl.py", line 195, in _exec
       return conn.execute(construct, multiparams)
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/engine/base.py", line 1200, in execute
       return meth(self, multiparams, params, _EMPTY_EXECUTION_OPTS)
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/sql/ddl.py", line 77, in _execute_on_connection
       return connection._execute_ddl(
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/engine/base.py", line 1290, in _execute_ddl
       ret = self._execute_context(
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/engine/base.py", line 1748, in _execute_context
       self._handle_dbapi_exception(
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/engine/base.py", line 1929, in _handle_dbapi_exception
       util.raise_(
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/util/compat.py", line 211, in raise_
       raise exception
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/engine/base.py", line 1705, in _execute_context
       self.dialect.do_execute(
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/engine/default.py", line 716, in do_execute
       cursor.execute(statement, parameters)
     File "/home/airflow/.local/lib/python3.8/site-packages/MySQLdb/cursors.py", line 206, in execute
       res = self._query(query)
     File "/home/airflow/.local/lib/python3.8/site-packages/MySQLdb/cursors.py", line 319, in _query
       db.query(q)
     File "/home/airflow/.local/lib/python3.8/site-packages/MySQLdb/connections.py", line 254, in query
       _mysql.connection.query(self, query)
   sqlalchemy.exc.OperationalError: (MySQLdb._exceptions.OperationalError) (3780, "Referencing column 'task_id' and referenced column 'task_id' in foreign key constraint 'task_map_task_instance_fkey' are incompatible.")
   [SQL: 
   CREATE TABLE task_map (
       dag_id VARCHAR(250) COLLATE utf8mb3_bin NOT NULL, 
       task_id VARCHAR(250) COLLATE utf8mb3_bin NOT NULL, 
       run_id VARCHAR(250) COLLATE utf8mb3_bin NOT NULL, 
       map_index INTEGER NOT NULL, 
       length INTEGER NOT NULL, 
       `keys` JSON, 
       PRIMARY KEY (dag_id, task_id, run_id, map_index), 
       CONSTRAINT task_map_length_not_negative CHECK (length >= 0), 
       CONSTRAINT task_map_task_instance_fkey FOREIGN KEY(dag_id, task_id, run_id, map_index) REFERENCES task_instance (dag_id, task_id, run_id, map_index) ON DELETE CASCADE
   )
   ]
   (Background on this error at: http://sqlalche.me/e/14/e3q8)
   ```
   
   Full error Log **after** first execution (should caused by first execution):
   ```
   
   /home/airflow/.local/lib/python3.8/site-packages/airflow/configuration.py:529: DeprecationWarning: The auth_backend option in [api[] has been renamed to auth_backends - the old setting has been used, but please update your config.
     option = self._get_option_from_config_file(deprecated_key, deprecated_section, key, kwargs, section)
   /home/airflow/.local/lib/python3.8/site-packages/airflow/configuration.py:356: FutureWarning: The auth_backends setting in [api[] has had airflow.api.auth.backend.session added in the running config, which is needed by the UI. Please update your config before Apache Airflow 3.0.
     warnings.warn(
   DB: mysql+mysqldb://airflow:***@test-airflow2-db-blue.cndbtlpttl69.eu-central-1.rds.amazonaws.com:3306/airflow
   Performing upgrade with database mysql+mysqldb://airflow:***@test-airflow2-db-blue.cndbtlpttl69.eu-central-1.rds.amazonaws.com:3306/airflow
   [2022-06-17 12:41:53,882[] {db.py:1448} INFO - Creating tables
   INFO  [alembic.runtime.migration[] Context impl MySQLImpl.
   INFO  [alembic.runtime.migration[] Will assume non-transactional DDL.
   INFO  [alembic.runtime.migration[] Running upgrade f9da662e7089 -> e655c0453f75, Add ``map_index`` column to TaskInstance to identify task-mapping,
   and a ``task_map`` table to track mapping values from XCom.
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/engine/base.py", line 1705, in _execute_context
       self.dialect.do_execute(
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/engine/default.py", line 716, in do_execute
       cursor.execute(statement, parameters)
     File "/home/airflow/.local/lib/python3.8/site-packages/MySQLdb/cursors.py", line 206, in execute
       res = self._query(query)
     File "/home/airflow/.local/lib/python3.8/site-packages/MySQLdb/cursors.py", line 319, in _query
       db.query(q)
     File "/home/airflow/.local/lib/python3.8/site-packages/MySQLdb/connections.py", line 254, in query
       _mysql.connection.query(self, query)
   MySQLdb._exceptions.OperationalError: (1091, "Can't DROP 'task_reschedule_ti_fkey'; check that column/key exists")
   The above exception was the direct cause of the following exception:
   Traceback (most recent call last):
     File "/home/airflow/.local/bin/airflow", line 8, in <module>
       sys.exit(main())
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/__main__.py", line 38, in main
       args.func(args)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/cli/cli_parser.py", line 51, in command
       return func(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/cli.py", line 99, in wrapper
       return f(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/cli/commands/db_command.py", line 82, in upgradedb
       db.upgradedb(to_revision=to_revision, from_revision=from_revision, show_sql_only=args.show_sql_only)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/session.py", line 71, in wrapper
       return func(*args, session=session, **kwargs)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/utils/db.py", line 1449, in upgradedb
       command.upgrade(config, revision=to_revision or 'heads')
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/command.py", line 322, in upgrade
       script.run_env()
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/script/base.py", line 569, in run_env
       util.load_python_file(self.dir, "env.py")
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/util/pyfiles.py", line 94, in load_python_file
       module = load_module_py(module_id, path)
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/util/pyfiles.py", line 110, in load_module_py
       spec.loader.exec_module(module)  # type: ignore
     File "<frozen importlib._bootstrap_external>", line 843, in exec_module
     File "<frozen importlib._bootstrap>", line 219, in _call_with_frames_removed
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/migrations/env.py", line 107, in <module>
       run_migrations_online()
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/migrations/env.py", line 101, in run_migrations_online
       context.run_migrations()
     File "<string>", line 8, in run_migrations
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/runtime/environment.py", line 853, in run_migrations
       self.get_context().run_migrations(**kw)
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/runtime/migration.py", line 623, in run_migrations
       step.migration_fn(**kw)
     File "/home/airflow/.local/lib/python3.8/site-packages/airflow/migrations/versions/0100_2_3_0_add_taskmap_and_map_id_on_taskinstance.py", line 49, in upgrade
       batch_op.drop_index("idx_task_reschedule_dag_task_run")
     File "/usr/local/lib/python3.8/contextlib.py", line 120, in __exit__
       next(self.gen)
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/operations/base.py", line 376, in batch_alter_table
       impl.flush()
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/operations/batch.py", line 111, in flush
       fn(*arg, **kw)
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/ddl/mysql.py", line 155, in drop_constraint
       super(MySQLImpl, self).drop_constraint(const)
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/ddl/impl.py", line 338, in drop_constraint
       self._exec(schema.DropConstraint(const))
     File "/home/airflow/.local/lib/python3.8/site-packages/alembic/ddl/impl.py", line 195, in _exec
       return conn.execute(construct, multiparams)
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/engine/base.py", line 1200, in execute
       return meth(self, multiparams, params, _EMPTY_EXECUTION_OPTS)
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/sql/ddl.py", line 77, in _execute_on_connection
       return connection._execute_ddl(
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/engine/base.py", line 1290, in _execute_ddl
       ret = self._execute_context(
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/engine/base.py", line 1748, in _execute_context
       self._handle_dbapi_exception(
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/engine/base.py", line 1929, in _handle_dbapi_exception
       util.raise_(
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/util/compat.py", line 211, in raise_
       raise exception
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/engine/base.py", line 1705, in _execute_context
       self.dialect.do_execute(
     File "/home/airflow/.local/lib/python3.8/site-packages/sqlalchemy/engine/default.py", line 716, in do_execute
       cursor.execute(statement, parameters)
     File "/home/airflow/.local/lib/python3.8/site-packages/MySQLdb/cursors.py", line 206, in execute
       res = self._query(query)
     File "/home/airflow/.local/lib/python3.8/site-packages/MySQLdb/cursors.py", line 319, in _query
       db.query(q)
     File "/home/airflow/.local/lib/python3.8/site-packages/MySQLdb/connections.py", line 254, in query
       _mysql.connection.query(self, query)
   sqlalchemy.exc.OperationalError: (MySQLdb._exceptions.OperationalError) (1091, "Can't DROP 'task_reschedule_ti_fkey'; check that column/key exists")
   [SQL: ALTER TABLE task_reschedule DROP FOREIGN KEY task_reschedule_ti_fkey[]
   (Background on this error at: http://sqlalche.me/e/14/e3q8)
   ```
   
   ### Are you willing to submit PR?
   
   - [ ] Yes I am willing to submit a PR!
   
   ### Code of Conduct
   
   - [X] I agree to follow this project's [Code of Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org.apache.org

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


[GitHub] [airflow] michaelosthege commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
michaelosthege commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1190560680

   Thanks, @potiuk from [the documentation on `sql_engine_collation_for_ids`](https://airflow.apache.org/docs/apache-airflow/stable/configurations-ref.html#sql-engine-collation-for-ids) I kew about `dag_id`, `task_id` and `key`, but it did not mention `external_executor_id` ⚠
   
   Also thank you for clarifying the recommended CHARSET/COLLATION:
   👉 `DEFAULT CHARSET=utf8 COLLATE=utf8_unicode_ci` (as described by the [database setup docs](https://airflow.apache.org/docs/apache-airflow/stable/howto/set-up-database.html#setting-up-a-mysql-database))
   👉 `CHARACTER SET utf8mb3 COLLATE utf8mb3_bin` for `dag_id`, `task_id`, `key`, `external_executor_id` fields
   
   I found the easiest way to fix this is to fix it right in my `backup.sql` file.
   
   When applying the modified `backup.sql` some `CREATE TABLE` failed because dependent tables only got updated later. I believe this is because `mysqldump` doesn't `DROP TABLE` everything at the beginning, but only one by one. Manually doing `DROP TABLE` for these few tables works.
   
   > My recommendation is as usual here: Switch to Postgres.
   
   First things first.. If there's a way to migrate the data I'll consider it once I recovered from todays incident..


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] esthomw commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
esthomw commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1191480379

   @potiuk would you like me to update my previous suggestion with
   > CHARSET/COLLATION for the "id" fields should be explicitly `utf8mb3' and collation 'utf8mb3_bin' NOT 'utf8', 'utf8_bin'
   ?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] esthomw commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
esthomw commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1173582891

   @itayB I think you had a different issue, the log you provided says `'Lost connection to MySQL server during query'`, however, other issues in this thread are related to CHARSET configuration. I haven't been able to solve this after a few hours of debugging and testing multiple CHARSET configuration options.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] dstandish commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
dstandish commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1162075878

   @ephraimbuddy @uranusjr seems like maybe when we changed to use `utf8mb3_bin` we needed to either (a) add logic to see how the existing task_id column was encoded and just stick with that  or (b) alter the column to use utf8mb3_bin?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] flrn77 commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
flrn77 commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1179043246

   Hello,
   Since I've better understanding on migration procees by had a deeper look into it ;) I could fix the migration manually.
   @dstandish It was all about the charset and at one point I had remove duplicates from xcom. Don't know, guess due to failed migration run. @potiuk I've taken your offer and put into what I think could help user on it as pull to the docs.
   
   @miimsam have a look at my modified sql dry-run code and into my [pull request 24926](https://github.com/apache/airflow/pull/24926) you should ba able fix it manual.
   
   ```sql
   UPDATE alembic_version SET version_num='e655c0453f75' WHERE alembic_version.version_num = 'f9da662e7089';
   
   -- Running upgrade e655c0453f75 -> a3bcd0914482
   
   ALTER TABLE serialized_dag MODIFY data JSON NULL;
   
   ALTER TABLE serialized_dag ADD COLUMN data_compressed BLOB;
   
   UPDATE alembic_version SET version_num='a3bcd0914482' WHERE alembic_version.version_num = 'e655c0453f75';
   
   -- Running upgrade a3bcd0914482 -> c306b5b5ae4a
   
   CREATE TABLE __airflow_tmp_xcom (
       dag_run_id INTEGER NOT NULL,
       task_id VARCHAR(250) COLLATE utf8mb3_bin NOT NULL,
       `key` VARCHAR(512) COLLATE utf8mb3_bin NOT NULL,
       value BLOB,
       timestamp TIMESTAMP(6) NOT NULL,
       dag_id VARCHAR(250) COLLATE utf8mb3_bin NOT NULL,
       run_id VARCHAR(250) COLLATE utf8mb3_bin NOT NULL,
       map_index INTEGER NOT NULL DEFAULT '-1'
   );
   
   /* it took 5min */
   INSERT INTO __airflow_tmp_xcom SELECT dag_run.id, xcom.task_id, xcom.`key`, xcom.value, xcom.timestamp, xcom.dag_id, dag_run.run_id, -1
   FROM xcom INNER JOIN dag_run ON xcom.dag_id = dag_run.dag_id AND xcom.execution_date = dag_run.execution_date;
   
   DROP TABLE xcom;
   
   ALTER TABLE __airflow_tmp_xcom RENAME TO xcom;
   
   /* don't know why but had to delete duplicates */
   create table tmp_xcom as select * from xcom limit 1;
   truncate table tmp_xcom;
   ALTER TABLE tmp_xcom ADD CONSTRAINT xcom_pkey PRIMARY KEY (dag_run_id, task_id, map_index, `key`);
   INSERT IGNORE INTO tmp_xcom SELECT * FROM xcom;
   DROP TABLE xcom;
   RENAME TABLE tmp_xcom TO xcom;
   
   /* done on tmp_xcom before renaming */
   -- ALTER TABLE xcom ADD CONSTRAINT xcom_pkey PRIMARY KEY (dag_run_id, task_id, map_index, `key`);
   
   CREATE INDEX idx_xcom_key ON xcom (`key`);
   
   ALTER TABLE xcom ADD CONSTRAINT xcom_task_instance_fkey FOREIGN KEY(dag_id, task_id, run_id, map_index) REFERENCES task_instance (dag_id, task_id, run_id, map_index) ON DELETE CASCADE;
   
   UPDATE alembic_version SET version_num='c306b5b5ae4a' WHERE alembic_version.version_num = 'a3bcd0914482';
   
   -- Running upgrade c306b5b5ae4a -> c97c2ab6aa23
   
   CREATE TABLE callback_request (
       id INTEGER NOT NULL AUTO_INCREMENT,
       created_at TIMESTAMP(6) NOT NULL,
       priority_weight INTEGER NOT NULL,
       callback_data JSON NOT NULL,
       callback_type VARCHAR(20) NOT NULL,
       dag_directory VARCHAR(1000),
       PRIMARY KEY (id)
   );
   
   UPDATE alembic_version SET version_num='c97c2ab6aa23' WHERE alembic_version.version_num = 'c306b5b5ae4a';
   
   -- Running upgrade c97c2ab6aa23 -> 4eaab2fe6582
   
   ALTER TABLE rendered_task_instance_fields ADD COLUMN map_index INTEGER NOT NULL DEFAULT '-1';
   
   ALTER TABLE rendered_task_instance_fields ADD COLUMN run_id VARCHAR(250) COLLATE utf8mb3_bin;
   
   UPDATE rendered_task_instance_fields, dag_run SET rendered_task_instance_fields.run_id=dag_run.run_id WHERE dag_run.dag_id = rendered_task_instance_fields.dag_id AND dag_run.execution_date = rendered_task_instance_fields.execution_date;
   
   ALTER TABLE rendered_task_instance_fields DROP PRIMARY KEY;
   
   ALTER TABLE rendered_task_instance_fields MODIFY run_id VARCHAR(250) COLLATE utf8mb3_bin NOT NULL;
   
   ALTER TABLE rendered_task_instance_fields DROP COLUMN execution_date;
   
   ALTER TABLE rendered_task_instance_fields ADD CONSTRAINT rendered_task_instance_fields_pkey PRIMARY KEY (dag_id, task_id, run_id, map_index);
   
   
   
   /* change cahrset caused by following error*/
   ALTER TABLE rendered_task_instance_fields MODIFY task_id VARCHAR(255) CHARACTER SET utf8 COLLATE utf8mb3_bin;
   ALTER TABLE rendered_task_instance_fields MODIFY dag_id VARCHAR(255) CHARACTER SET utf8 COLLATE utf8mb3_bin;
   /* [Code: 3780, SQL State: HY000]  Referencing column 'dag_id' and referenced column 'dag_id' in foreign key constraint 'rtif_ti_fkey' are incompatible. */
   ALTER TABLE rendered_task_instance_fields ADD CONSTRAINT rtif_ti_fkey FOREIGN KEY(dag_id, task_id, run_id, map_index) REFERENCES task_instance (dag_id, task_id, run_id, map_index) ON DELETE CASCADE;
   
   UPDATE alembic_version SET version_num='4eaab2fe6582' WHERE alembic_version.version_num = 'c97c2ab6aa23';
   
   -- Running upgrade 4eaab2fe6582 -> 48925b2719cb
   
   DROP INDEX idx_task_fail_dag_task_date ON task_fail;
   
   ALTER TABLE task_fail ADD COLUMN map_index INTEGER NOT NULL DEFAULT '-1';
   
   ALTER TABLE task_fail ADD COLUMN run_id VARCHAR(250) COLLATE utf8mb3_bin;
   
   UPDATE task_fail, dag_run SET task_fail.run_id=dag_run.run_id WHERE dag_run.dag_id = task_fail.dag_id AND dag_run.execution_date = task_fail.execution_date;
   
   ALTER TABLE task_fail MODIFY run_id VARCHAR(250) COLLATE utf8mb3_bin NOT NULL;
   
   ALTER TABLE task_fail DROP COLUMN execution_date;
   
   /* change cahrset caused by following error*/
   ALTER TABLE task_fail MODIFY task_id VARCHAR(255) CHARACTER SET utf8 COLLATE utf8mb3_bin;
   ALTER TABLE task_fail MODIFY dag_id VARCHAR(255) CHARACTER SET utf8 COLLATE utf8mb3_bin;
   
   ALTER TABLE task_fail ADD CONSTRAINT task_fail_ti_fkey FOREIGN KEY(dag_id, task_id, run_id, map_index) REFERENCES task_instance (dag_id, task_id, run_id, map_index) ON DELETE CASCADE;
   
   UPDATE alembic_version SET version_num='48925b2719cb' WHERE alembic_version.version_num = '4eaab2fe6582';
   
   -- Running upgrade 48925b2719cb -> 909884dea523
   
   ALTER TABLE ab_register_user MODIFY username VARCHAR(256) NOT NULL;
   
   ALTER TABLE ab_register_user MODIFY email VARCHAR(256) NOT NULL;
   
   ALTER TABLE ab_user MODIFY username VARCHAR(256) NOT NULL;
   
   ALTER TABLE ab_user MODIFY email VARCHAR(256) NOT NULL;
   
   UPDATE alembic_version SET version_num='909884dea523' WHERE alembic_version.version_num = '48925b2719cb';
   
   -- Running upgrade 909884dea523 -> 75d5ed6c2b43
   
   ALTER TABLE log ADD COLUMN map_index INTEGER;
   
   UPDATE alembic_version SET version_num='75d5ed6c2b43' WHERE alembic_version.version_num = '909884dea523';
   
   -- Running upgrade 75d5ed6c2b43 -> b1b348e02d07
   
   UPDATE dag SET default_view='grid' WHERE dag.default_view = 'tree';
   
   UPDATE alembic_version SET version_num='b1b348e02d07' WHERE alembic_version.version_num = '75d5ed6c2b43';
   
   -- Running upgrade b1b348e02d07 -> 1de7bc13c950
   
   CREATE INDEX idx_log_event ON log (event);
   
   UPDATE alembic_version SET version_num='1de7bc13c950' WHERE alembic_version.version_num = 'b1b348e02d07';
   
   -- Running upgrade 1de7bc13c950 -> 3c94c427fdf6
   
   ALTER TABLE dag_tag DROP FOREIGN KEY dag_tag_ibfk_1;
   
   ALTER TABLE dag_tag ADD CONSTRAINT dag_tag_dag_id_fkey FOREIGN KEY(dag_id) REFERENCES dag (dag_id) ON DELETE CASCADE;
   
   UPDATE alembic_version SET version_num='3c94c427fdf6' WHERE alembic_version.version_num = '1de7bc13c950';
   ```
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] flrn77 commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
flrn77 commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1170817271

   Hi @miimsam,
   thx for the Issue description. That's a good hint ;)
   I have `CHARACTER SET utf8mb4 COLLATE utf8mb4_0900_ai_ci` in `task_instance.task_id`.
   
   yesterday I tested migration with changed  `CHARACTER SET utf8 COLLATE utf8_bin` like it is when I create DB from scratch.
   That step works. But caused by the first failed migration I have to do manuell steps to bring it back to work.
   
   ```sql
   -- 1.  change charset and collate
   ALTER TABLE task_instance MODIFY task_id VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_bin;
   
   -- 2. drop table session
   drop table session;
   
   -- 3. drop index
   drop index idx_job_dag_id on job;
   
   -- 4. drop column
   ALTER TABLE dag drop COLUMN timetable_description;
   
   -- 5. drop table log_template
   ALTER TABLE dag_run DROP FOREIGN KEY task_instance_log_template_id_fkey;
   DROP TABLE log_template;
   
   ...
   ```
   
   I also have an older Backup before the the failed migration Job, maybe I should try this if the list of manual steps growth.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk closed issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
potiuk closed issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job
URL: https://github.com/apache/airflow/issues/24526


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] esthomw commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
esthomw commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1181627863

   Hi,
   
   I managed to run migrations with some small manual changes to database before running `airflow db upgrade`. I was upgrading from 2.2.3 to 2.3.2. The main culprit in that situation was misaligned CHARSET/COLLATE for task_id and dag_id column between multiple tables. Here is the script I used to align those columns:
   
   ```sql
   -- remove foreign key temporarily as it is used by one of modified columns
   ALTER TABLE task_reschedule DROP FOREIGN KEY task_reschedule_ti_fkey;
   
   ALTER TABLE task_instance MODIFY task_id VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_bin;
   ALTER TABLE task_reschedule MODIFY task_id VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_bin;
   ALTER TABLE rendered_task_instance_fields MODIFY task_id VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_bin;
   ALTER TABLE rendered_task_instance_fields MODIFY dag_id VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_bin;
   ALTER TABLE task_fail MODIFY task_id VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_bin;
   ALTER TABLE task_fail MODIFY dag_id VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_bin;
   
   -- reintroduce the key (otherwise migrations should fail)
   ALTER TABLE task_reschedule ADD CONSTRAINT task_reschedule_ti_fkey FOREIGN KEY (dag_id, task_id, run_id) REFERENCES task_instance (dag_id, task_id, run_id);
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1170895808

   Hey Good people here @milimsam @flrn77 ..... Is this possible to make a PR to https://airflow.apache.org/docs/apache-airflow/stable/installation/upgrading.html#migration-best-practices and describe those issues in a new "Troubleshooting MySQL Exceptions at upgrade" chapter (for example)?  It would be a great contribution and giving back to the community - I think together with @dstandish we will be super happy to help with review of that and revise it and guide in the process, but it would be super, super helpful if we can have such a documentation.
   
   And starting it is easy - you just click "Suggest a change on this page" and it will open a Pull Request, where you are able to add such documentation via GitHub UI. 
   
   I think, personally it would be best if you - as users who experienced it - describe it in the way that you would find helpful if you looked for a solution to a similar problem. Simply we - as maintainers - know the system too well and we have many assumptions in our head that our users don't - and maybe we don't even know where the users like would look for similar problem/issue and how phrase the "solution" steps to make it easy to follow.
   
   Would you like to become one of more than 2100 contributors to Airlfow ? This is an easy (ans super-helpful to others) way!
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] michaelosthege commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
michaelosthege commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1190450784

   Upgrading from 2.2.5 to 2.3.3 I'm running into these issues too.
   Thanks to the posts above I'm slowly making progress, and I think I have something useful to add 👇
   
   First I'm trying to fix the the `CHARACTER SET` and `COLLATE` of all tables and columns as described by [this guide](https://confluence.atlassian.com/kb/how-to-fix-the-collation-and-character-set-of-a-mysql-database-manually-744326173.html).
   * Using `CHARACTER SET utf8mb4 COLLATE utf8mb4_unicode_ci` as described by https://airflow.apache.org/docs/apache-airflow/stable/howto/set-up-database.html#setting-up-a-mysql-database
   * ⚠ Except for `task_id` and `dag_id` columns [as described by the comment above](https://github.com/apache/airflow/issues/24526#issuecomment-1181627863)
   
   ❓ __Am I doing the right thing with the character set and collation?__
   
   ❓ __If anybody has a more convenient script/SQL command set to fix the collections that'd be awesome!__
   
   -------
   
   After that I'm trying to do the `airflow db upgrade` and now it looks like I'm running into something similar to #24247 (I'm using MySQL not MariaDB).
   
   ```
   INFO  [alembic.runtime.migration] Running upgrade 5e3ec427fdd3 -> 786e3737b18f, Add ``timetable_description`` column to DagModel for UI.
   ...
   sqlalchemy.exc.ProgrammingError: (MySQLdb.ProgrammingError) (1146, "Table 'airflow.dag' doesn't exist")
   [SQL: ALTER TABLE dag ADD COLUMN timetable_description VARCHAR(1000)]
   ```
   
   The [fix to that might be to run `airflow db init`](https://github.com/apache/airflow/issues/24247#issuecomment-1147509478) under 2.2.5 before upgrading. I'll try that and update this comment ⏳


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] j-martin commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
j-martin commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1344890185

   For posterity and in case it helps somebody else we fix our schema with the queries below. Note that it assumes that the migrations from 2.2 to 2.3 have not been executed and failed. Since DDL changes are not transaction in MySQL (or at least the way Alembic is configured), the state of the database will be different after a failed migration.
   
   ```sql
   ALTER DATABASE airflow CHARACTER SET utf8mb4 COLLATE utf8mb4_unicode_ci; 
   ALTER TABLE dag_tag DROP FOREIGN KEY dag_tag_ibfk_1;
   ALTER TABLE dag_tag MODIFY dag_id VARCHAR(250) CHARACTER SET utf8mb3 COLLATE utf8mb3_bin;
   ALTER TABLE dag MODIFY dag_id VARCHAR(250) CHARACTER SET utf8mb3 COLLATE utf8mb3_bin;
   ALTER TABLE dag_tag ADD CONSTRAINT dag_tag_ibfk_1 FOREIGN KEY (dag_id) REFERENCES dag (dag_id) ON DELETE CASCADE;
   
   ALTER TABLE task_fail MODIFY dag_id VARCHAR(250) CHARACTER SET utf8mb3 COLLATE utf8mb3_bin;
   
   ALTER TABLE job MODIFY dag_id VARCHAR(250) CHARACTER SET utf8mb3 COLLATE utf8mb3_bin;
   ALTER TABLE log MODIFY dag_id VARCHAR(250) CHARACTER SET utf8mb3 COLLATE utf8mb3_bin;
   ALTER TABLE sensor_instance MODIFY dag_id VARCHAR(250) CHARACTER SET utf8mb3 COLLATE utf8mb3_bin;
   ALTER TABLE serialized_dag MODIFY dag_id VARCHAR(250) CHARACTER SET utf8mb3 COLLATE utf8mb3_bin;
   ALTER TABLE sla_miss MODIFY dag_id VARCHAR(250) CHARACTER SET utf8mb3 COLLATE utf8mb3_bin;
   
   alter table task_instance drop foreign key task_instance_dag_run_fkey;
   ALTER TABLE task_reschedule DROP FOREIGN KEY task_reschedule_ti_fkey;
   ALTER TABLE task_reschedule MODIFY task_id VARCHAR(255) CHARACTER SET utf8mb3 COLLATE utf8mb3_bin;
   
   ALTER TABLE task_instance MODIFY dag_id VARCHAR(250) CHARACTER SET utf8mb3 COLLATE utf8mb3_bin;
   alter table task_instance add constraint task_instance_dag_run_fkey foreign key (dag_id, run_id) references dag_run (dag_id, run_id) on delete cascade;
   
   ALTER TABLE task_reschedule MODIFY dag_id VARCHAR(250) CHARACTER SET utf8mb3 COLLATE utf8mb3_bin;
   ALTER TABLE xcom MODIFY dag_id VARCHAR(250) CHARACTER SET utf8mb3 COLLATE utf8mb3_bin;
   
   ALTER TABLE task_instance MODIFY task_id VARCHAR(255) CHARACTER SET utf8mb3 COLLATE utf8mb3_bin;
   
   ALTER TABLE task_reschedule ADD CONSTRAINT task_reschedule_ti_fkey FOREIGN KEY (dag_id, task_id, run_id) REFERENCES task_instance (dag_id, task_id, run_id);
   
   ALTER TABLE rendered_task_instance_fields MODIFY dag_id VARCHAR(255) CHARACTER SET utf8mb3 COLLATE utf8mb3_bin;
   ALTER TABLE rendered_task_instance_fields MODIFY task_id VARCHAR(255) CHARACTER SET utf8mb3 COLLATE utf8mb3_bin;
   ALTER TABLE task_fail MODIFY task_id VARCHAR(255) CHARACTER SET utf8mb3 COLLATE utf8mb3_bin;
   ALTER TABLE task_fail MODIFY dag_id VARCHAR(255) CHARACTER SET utf8mb3 COLLATE utf8mb3_bin;
   
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] boring-cyborg[bot] commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on upgrade migration-job

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1158900453

   Thanks for opening your first issue here! Be sure to follow the issue template!
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1191503582

   > @potiuk would you like me to update my previous suggestion with
   > 
   > > CHARSET/COLLATION for the "id" fields should be explicitly `utf8mb3' and collation 'utf8mb3_bin' NOT 'utf8', 'utf8_bin'
   > > ?
   
   You can do it yourself


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1159822917

   Is this somethign similar to MariaDB #24247 but for RDS ? @ephraimbuddy @dstandish - we seem to have som consistent problems with missing indexes 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1190658878

   Glad I could help.
   
   > Thanks, @potiuk from [the documentation on sql_engine_collation_for_ids](https://airflow.apache.org/docs/apache-airflow/stable/configurations-ref.html#sql-engine-collation-for-ids) I kew about dag_id, task_id and key, but it did not mention external_executor_id warning
   
   PRs to the docs are most welcome. I think it would be great if after your experience you help others describing what you have done and update the docs. Airflow is created by > 2100 contributors - most of them users like you, and actually contributing docs to make them clerer and more helpful to others is one of the best contribution you can make  - paying back by the 100% free software you use. In fact, you are probably the best person in the world now to describe it in the way that will be useful for the others, because you've JUST gone through it, remember all the steps you took and you can write it in the way that it will be understandable for users like you who would struggle with similar problem.
   
   And it's super easy. Look at the [the documentation on sql_engine_collation_for_ids](https://airflow.apache.org/docs/apache-airflow/stable/configurations-ref.html#sql-engine-collation-for-ids) - at the bottom right there is "suggest a change on this page" button that will open up PR which you can edit directly there, in GitHub UI - it's an .rst doc that you can edit as easily as commenting in the issue here.
   
   I heartily encourage you to do so!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1190539682

   > question Am I doing the right thing with the character set and collation?
   
   Yes.  But you should also do it for all colums that use different collation (the different CHARACTER seet for thos should be `uf8mb3`). I just looked through all the tables columns and here is a complete list (most of the new ones should have it correctly set - simply not sure which ones were already in 2.2.5):
   
   * slamiss: dag_id
   * taskmap:  dag_id, task_id, run_id
   * taskreschedule: dag_id, task_id, run_id
   * xcom: dag_id, task_id, run_id, key
   * dagrun: dag_id, run_id
   * dagwarning: dag_id
   * log: dag_id, task_id
   * renderedtifields: dag_id, task_id, run_id
   * taskfail: dag_id, task_id, run_id
   * taskinstance: dag_id, task_id, run_id, external_executor_id
   
   Also I think the above comment sets another trap on you using utf8 and utf8_bin is very bad idea. MySQL 8 set a trap on you here, because if you migrate to the next version, the index will likely not increase but utf8 will become an alias to utf8mb4 (in MySQL 8 it is utf8mb3):
   
   https://dev.mysql.com/doc/refman/8.0/en/charset-unicode-utf8.html
   
   CHARSET/COLLATION for the "id" fields should be explicitly `utf8mb3' and collation 'utf8mb3_bin' NOT 'utf8', 'utf8_bin'
   
   The trap is a bit deeper, because is (as they warn) they remove utf8mb3 in the future then you will be locked witth MySQL8.
   
   My recommendation is as usual here: Switch to Postgres.
   
   
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] dstandish commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
dstandish commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1162117811

   @flrn77 @miimsam what you may want to try to get around this is manually alter your task_instance.task_id column to have collation utf8mb3_bin.  maybe you can try this?   cc @ashb 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] dstandish commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
dstandish commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1162088665

   yeah i have confirmed locally that _if_ task_id is `utf8mb4_general_ci` in task_instance, it will produce that error when you try to create task_map.
   
   @flrn77 @miimsam can you confirm what encodings you have for the task_id column in task_instance?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] miimsam commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
miimsam commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1161744906

   We ran into the same issue when upgrading from 2.2.5 to 2.3.2.
   
   **What happened**
   Upgrade Airflow 2.2.5 -> 2.3.2 fails on migration-job (airflow db upgrade)
   
   **Database**: MySQL 8.0.19
   
   
   ```
   [2022-06-17 10:53:27,533] {{db.py:1448}} INFO - Creating tables
   INFO  [alembic.runtime.migration] Context impl MySQLImpl.
   INFO  [alembic.runtime.migration] Will assume non-transactional DDL.
   INFO  [alembic.runtime.migration] Running upgrade f9da662e7089 -> e655c0453f75, Add ``map_index`` column to TaskInstance to identify task-mapping,
   and a ``task_map`` table to track mapping values from XCom.
   Traceback (most recent call last):
     File "/usr/local/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 1706, in _execute_context
       cursor, statement, parameters, context
     File "/usr/local/lib/python3.7/site-packages/sqlalchemy/engine/default.py", line 716, in do_execute
       cursor.execute(statement, parameters)
     File "/usr/local/lib/python3.7/site-packages/pymysql/cursors.py", line 148, in execute
       result = self._query(query)
     File "/usr/local/lib/python3.7/site-packages/pymysql/cursors.py", line 310, in _query
       conn.query(q)
     File "/usr/local/lib/python3.7/site-packages/pymysql/connections.py", line 548, in query
       self._affected_rows = self._read_query_result(unbuffered=unbuffered)
     File "/usr/local/lib/python3.7/site-packages/pymysql/connections.py", line 775, in _read_query_result
       result.read()
     File "/usr/local/lib/python3.7/site-packages/pymysql/connections.py", line 1156, in read
       first_packet = self.connection._read_packet()
     File "/usr/local/lib/python3.7/site-packages/pymysql/connections.py", line 725, in _read_packet
       packet.raise_for_error()
     File "/usr/local/lib/python3.7/site-packages/pymysql/protocol.py", line 221, in raise_for_error
       err.raise_mysql_exception(self._data)
     File "/usr/local/lib/python3.7/site-packages/pymysql/err.py", line 143, in raise_mysql_exception
       raise errorclass(errno, errval)
   pymysql.err.OperationalError: (1091, "Can't DROP 'task_reschedule_ti_fkey'; check that column/key exists")
   
   The above exception was the direct cause of the following exception:
   
   Traceback (most recent call last):
     File "/usr/local/bin/airflow", line 8, in <module>
       sys.exit(main())
     File "/usr/local/lib/python3.7/site-packages/airflow/__main__.py", line 38, in main
       args.func(args)
     File "/usr/local/lib/python3.7/site-packages/airflow/cli/cli_parser.py", line 51, in command
       return func(*args, **kwargs)
     File "/usr/local/lib/python3.7/site-packages/airflow/utils/cli.py", line 99, in wrapper
       return f(*args, **kwargs)
     File "/usr/local/lib/python3.7/site-packages/airflow/cli/commands/db_command.py", line 82, in upgradedb
       db.upgradedb(to_revision=to_revision, from_revision=from_revision, show_sql_only=args.show_sql_only)
     File "/usr/local/lib/python3.7/site-packages/airflow/utils/session.py", line 71, in wrapper
       return func(*args, session=session, **kwargs)
     File "/usr/local/lib/python3.7/site-packages/airflow/utils/db.py", line 1449, in upgradedb
       command.upgrade(config, revision=to_revision or 'heads')
     File "/usr/local/lib/python3.7/site-packages/alembic/command.py", line 322, in upgrade
       script.run_env()
     File "/usr/local/lib/python3.7/site-packages/alembic/script/base.py", line 569, in run_env
       util.load_python_file(self.dir, "env.py")
     File "/usr/local/lib/python3.7/site-packages/alembic/util/pyfiles.py", line 94, in load_python_file
       module = load_module_py(module_id, path)
     File "/usr/local/lib/python3.7/site-packages/alembic/util/pyfiles.py", line 110, in load_module_py
       spec.loader.exec_module(module)  # type: ignore
     File "<frozen importlib._bootstrap_external>", line 728, in exec_module
     File "<frozen importlib._bootstrap>", line 219, in _call_with_frames_removed
     File "/usr/local/lib/python3.7/site-packages/airflow/migrations/env.py", line 107, in <module>
       run_migrations_online()
     File "/usr/local/lib/python3.7/site-packages/airflow/migrations/env.py", line 101, in run_migrations_online
       context.run_migrations()
     File "<string>", line 8, in run_migrations
     File "/usr/local/lib/python3.7/site-packages/alembic/runtime/environment.py", line 853, in run_migrations
       self.get_context().run_migrations(**kw)
     File "/usr/local/lib/python3.7/site-packages/alembic/runtime/migration.py", line 623, in run_migrations
       step.migration_fn(**kw)
     File "/usr/local/lib/python3.7/site-packages/airflow/migrations/versions/0100_2_3_0_add_taskmap_and_map_id_on_taskinstance.py", line 49, in upgrade
       batch_op.drop_index("idx_task_reschedule_dag_task_run")
     File "/usr/local/lib/python3.7/contextlib.py", line 119, in __exit__
       next(self.gen)
     File "/usr/local/lib/python3.7/site-packages/alembic/operations/base.py", line 376, in batch_alter_table
       impl.flush()
     File "/usr/local/lib/python3.7/site-packages/alembic/operations/batch.py", line 111, in flush
       fn(*arg, **kw)
     File "/usr/local/lib/python3.7/site-packages/alembic/ddl/mysql.py", line 155, in drop_constraint
       super(MySQLImpl, self).drop_constraint(const)
     File "/usr/local/lib/python3.7/site-packages/alembic/ddl/impl.py", line 338, in drop_constraint
       self._exec(schema.DropConstraint(const))
     File "/usr/local/lib/python3.7/site-packages/alembic/ddl/impl.py", line 195, in _exec
       return conn.execute(construct, multiparams)
     File "/usr/local/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 1200, in execute
       return meth(self, multiparams, params, _EMPTY_EXECUTION_OPTS)
     File "/usr/local/lib/python3.7/site-packages/sqlalchemy/sql/ddl.py", line 78, in _execute_on_connection
       self, multiparams, params, execution_options
     File "/usr/local/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 1296, in _execute_ddl
       compiled,
     File "/usr/local/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 1749, in _execute_context
       e, statement, parameters, cursor, context
     File "/usr/local/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 1930, in _handle_dbapi_exception
       sqlalchemy_exception, with_traceback=exc_info[2], from_=e
     File "/usr/local/lib/python3.7/site-packages/sqlalchemy/util/compat.py", line 211, in raise_
       raise exception
     File "/usr/local/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 1706, in _execute_context
       cursor, statement, parameters, context
     File "/usr/local/lib/python3.7/site-packages/sqlalchemy/engine/default.py", line 716, in do_execute
       cursor.execute(statement, parameters)
     File "/usr/local/lib/python3.7/site-packages/pymysql/cursors.py", line 148, in execute
       result = self._query(query)
     File "/usr/local/lib/python3.7/site-packages/pymysql/cursors.py", line 310, in _query
       conn.query(q)
     File "/usr/local/lib/python3.7/site-packages/pymysql/connections.py", line 548, in query
       self._affected_rows = self._read_query_result(unbuffered=unbuffered)
     File "/usr/local/lib/python3.7/site-packages/pymysql/connections.py", line 775, in _read_query_result
       result.read()
     File "/usr/local/lib/python3.7/site-packages/pymysql/connections.py", line 1156, in read
       first_packet = self.connection._read_packet()
     File "/usr/local/lib/python3.7/site-packages/pymysql/connections.py", line 725, in _read_packet
       packet.raise_for_error()
     File "/usr/local/lib/python3.7/site-packages/pymysql/protocol.py", line 221, in raise_for_error
       err.raise_mysql_exception(self._data)
     File "/usr/local/lib/python3.7/site-packages/pymysql/err.py", line 143, in raise_mysql_exception
       raise errorclass(errno, errval)
   sqlalchemy.exc.OperationalError: (pymysql.err.OperationalError) (1091, "Can't DROP 'task_reschedule_ti_fkey'; check that column/key exists")
   [SQL: ALTER TABLE task_reschedule DROP FOREIGN KEY task_reschedule_ti_fkey]
   (Background on this error at: http://sqlalche.me/e/14/e3q8)
   
   ```
   
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] miimsam commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
miimsam commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1162675742

   > yeah i have confirmed locally that _if_ task_id is `utf8mb4_general_ci` in task_instance, it will produce that error when you try to create task_map with mb3 because the FK is not compatible.
   > 
   > @flrn77 @miimsam can you confirm what encodings you have for the task_id column in task_instance?
   
   `task_id` varchar(250) CHARACTER SET utf8mb4 COLLATE utf8mb4_unicode_ci NOT NULL`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] itayB commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
itayB commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1171307508

   my migration failed as well, same setup, RDS MySql 8.0.20, upgrading from 2.2.5 --> 2.3.2, failed on 
   ```
   INFO  [alembic.runtime.migration] Running upgrade 5e3ec427fdd3 -> 786e3737b18f, Add ``timetable_description`` column to DagModel for UI.
   INFO  [alembic.runtime.migration] Running upgrade 786e3737b18f -> f9da662e7089, Add ``LogTemplate`` table to track changes to config values ``log_filename_template``
   INFO  [alembic.runtime.migration] Running upgrade f9da662e7089 -> e655c0453f75, Add ``map_index`` column to TaskInstance to identify task-mapping,
   and a ``task_map`` table to track mapping values from XCom.
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.9/site-packages/sqlalchemy/engine/base.py", line 1705, in _execute_context
       self.dialect.do_execute(
     File "/home/airflow/.local/lib/python3.9/site-packages/sqlalchemy/engine/default.py", line 716, in do_execute
       cursor.execute(statement, parameters)
     File "/home/airflow/.local/lib/python3.9/site-packages/pymysql/cursors.py", line 148, in execute
       result = self._query(query)
     File "/home/airflow/.local/lib/python3.9/site-packages/pymysql/cursors.py", line 310, in _query
       conn.query(q)
     File "/home/airflow/.local/lib/python3.9/site-packages/pymysql/connections.py", line 548, in query
       self._affected_rows = self._read_query_result(unbuffered=unbuffered)
     File "/home/airflow/.local/lib/python3.9/site-packages/pymysql/connections.py", line 775, in _read_query_result
       result.read()
     File "/home/airflow/.local/lib/python3.9/site-packages/pymysql/connections.py", line 1156, in read
       first_packet = self.connection._read_packet()
     File "/home/airflow/.local/lib/python3.9/site-packages/pymysql/connections.py", line 692, in _read_packet
       packet_header = self._read_bytes(4)
     File "/home/airflow/.local/lib/python3.9/site-packages/pymysql/connections.py", line 748, in _read_bytes
       raise err.OperationalError(
   pymysql.err.OperationalError: (2013, 'Lost connection to MySQL server during query')
   
   The above exception was the direct cause of the following exception:
   
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/db.py", line 1629, in create_global_lock
       yield
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/db.py", line 1449, in upgradedb
       command.upgrade(config, revision=to_revision or 'heads')
     File "/home/airflow/.local/lib/python3.9/site-packages/alembic/command.py", line 322, in upgrade
       script.run_env()
     File "/home/airflow/.local/lib/python3.9/site-packages/alembic/script/base.py", line 569, in run_env
       util.load_python_file(self.dir, "env.py")
     File "/home/airflow/.local/lib/python3.9/site-packages/alembic/util/pyfiles.py", line 94, in load_python_file
       module = load_module_py(module_id, path)
     File "/home/airflow/.local/lib/python3.9/site-packages/alembic/util/pyfiles.py", line 110, in load_module_py
       spec.loader.exec_module(module)  # type: ignore
     File "<frozen importlib._bootstrap_external>", line 850, in exec_module
     File "<frozen importlib._bootstrap>", line 228, in _call_with_frames_removed
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/migrations/env.py", line 107, in <module>
       run_migrations_online()
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/migrations/env.py", line 101, in run_migrations_online
       context.run_migrations()
     File "<string>", line 8, in run_migrations
     File "/home/airflow/.local/lib/python3.9/site-packages/alembic/runtime/environment.py", line 853, in run_migrations
       self.get_context().run_migrations(**kw)
     File "/home/airflow/.local/lib/python3.9/site-packages/alembic/runtime/migration.py", line 623, in run_migrations
       step.migration_fn(**kw)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/migrations/versions/0100_2_3_0_add_taskmap_and_map_id_on_taskinstance.py", line 56, in upgrade
       batch_op.create_primary_key("task_instance_pkey", ["dag_id", "task_id", "run_id", "map_index"])
     File "/usr/local/lib/python3.9/contextlib.py", line 126, in __exit__
       next(self.gen)
     File "/home/airflow/.local/lib/python3.9/site-packages/alembic/operations/base.py", line 376, in batch_alter_table
       impl.flush()
     File "/home/airflow/.local/lib/python3.9/site-packages/alembic/operations/batch.py", line 111, in flush
       fn(*arg, **kw)
     File "/home/airflow/.local/lib/python3.9/site-packages/alembic/ddl/impl.py", line 335, in add_constraint
       self._exec(schema.AddConstraint(const))
     File "/home/airflow/.local/lib/python3.9/site-packages/alembic/ddl/impl.py", line 195, in _exec
       return conn.execute(construct, multiparams)
     File "/home/airflow/.local/lib/python3.9/site-packages/sqlalchemy/engine/base.py", line 1200, in execute
       return meth(self, multiparams, params, _EMPTY_EXECUTION_OPTS)
     File "/home/airflow/.local/lib/python3.9/site-packages/sqlalchemy/sql/ddl.py", line 77, in _execute_on_connection
       return connection._execute_ddl(
     File "/home/airflow/.local/lib/python3.9/site-packages/sqlalchemy/engine/base.py", line 1290, in _execute_ddl
       ret = self._execute_context(
     File "/home/airflow/.local/lib/python3.9/site-packages/sqlalchemy/engine/base.py", line 1748, in _execute_context
       self._handle_dbapi_exception(
     File "/home/airflow/.local/lib/python3.9/site-packages/sqlalchemy/engine/base.py", line 1929, in _handle_dbapi_exception
       util.raise_(
     File "/home/airflow/.local/lib/python3.9/site-packages/sqlalchemy/util/compat.py", line 211, in raise_
       raise exception
     File "/home/airflow/.local/lib/python3.9/site-packages/sqlalchemy/engine/base.py", line 1705, in _execute_context
       self.dialect.do_execute(
     File "/home/airflow/.local/lib/python3.9/site-packages/sqlalchemy/engine/default.py", line 716, in do_execute
       cursor.execute(statement, parameters)
     File "/home/airflow/.local/lib/python3.9/site-packages/pymysql/cursors.py", line 148, in execute
       result = self._query(query)
     File "/home/airflow/.local/lib/python3.9/site-packages/pymysql/cursors.py", line 310, in _query
       conn.query(q)
     File "/home/airflow/.local/lib/python3.9/site-packages/pymysql/connections.py", line 548, in query
       self._affected_rows = self._read_query_result(unbuffered=unbuffered)
     File "/home/airflow/.local/lib/python3.9/site-packages/pymysql/connections.py", line 775, in _read_query_result
       result.read()
     File "/home/airflow/.local/lib/python3.9/site-packages/pymysql/connections.py", line 1156, in read
       first_packet = self.connection._read_packet()
     File "/home/airflow/.local/lib/python3.9/site-packages/pymysql/connections.py", line 692, in _read_packet
       packet_header = self._read_bytes(4)
     File "/home/airflow/.local/lib/python3.9/site-packages/pymysql/connections.py", line 748, in _read_bytes
       raise err.OperationalError(
   sqlalchemy.exc.OperationalError: (pymysql.err.OperationalError) (2013, 'Lost connection to MySQL server during query')
   [SQL: ALTER TABLE task_instance ADD CONSTRAINT task_instance_pkey PRIMARY KEY (dag_id, task_id, run_id, map_index)]
   (Background on this error at: http://sqlalche.me/e/14/e3q8)
   ```
   
   and we are currently stuck, trying to recover this step manually :(
   Any tip here? is there's a ref to the migration upgrade steps somewhere?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] itayB commented on issue #24526: upgrading from 2.2.3 or 2.2.5 to 2.3.2 fails on migration-job

Posted by GitBox <gi...@apache.org>.
itayB commented on issue #24526:
URL: https://github.com/apache/airflow/issues/24526#issuecomment-1171496134

   I don't want to mix things here, not sure if that's a similar case - but we managed to overcome the issue by taking the scheduler down and running the following:
   ```
   USE airflow;
   ALTER TABLE task_reschedule
   ADD CONSTRAINT `task_reschedule_ti_fkey` FOREIGN KEY (`dag_id`, `task_id`, `run_id`) REFERENCES `task_instance` (`dag_id`, `task_id`, `run_id`) ON DELETE CASCADE;
   ALTER TABLE task_reschedule
   ADD KEY `idx_task_reschedule_dag_task_run` (`dag_id`,`task_id`,`run_id`);
   ALTER TABLE `task_instance` 
   ADD PRIMARY KEY (`dag_id`,`task_id`,`run_id`);
   ALTER TABLE task_instance DROP COLUMN map_index;
   ```
   
   Then we start the scheduler and it pass this migration step and all the following.
   It was relatively slow compared to our staging environment.
   When we upgrade our staging environment everything was smooth but it has much smaller amount of data in the DB. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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