You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@airflow.apache.org by /dev /local/ca <de...@gmail.com> on 2020/02/13 07:31:25 UTC

Many many problems getting Postgresql working as a backend

I am having problems setting up PG as a backend,

I am getting all sorts of errors and messages about 'MSSQL', 'MySQL',
'SQLite',

that I'm not even using, that is not even part of the equation here ,and I
am very confused and behind schedule getting this beast up and working
minimally.

I set three connection strings in $AIRFLOW_HOME/airflow.cfg to point to
Postgres.

```
sql_alchemy_conn =
postgresql+psycopg2://airflow:<password>@<ip-address>/airflow
broker_url = postgresql+psycopg2://airflow:<password>@<ip-address>/airflow
result_backend =
postgresql+psycopg2://airflow:<password>@<ip-address>/airflow
```

---
I am now running `$airflow initdb`

After running, I see 23 tables (running as pg user: 'airflow':  `SELECT
tablename FROM pg_tables WHERE schemaname = 'public';`

-----------------------
 alembic_version
 dag_pickle
 import_error
 job
 slot_pool
 chart
 known_event_type
 known_event
 xcom
 log
 dag_run
 sla_miss
 connection
 variable
 task_fail
 kube_resource_version
 kube_worker_uuid
 task_instance
 task_reschedule
 users
 serialized_dag
 dag
 dag_tag
(23 rows)

---

But the output indicates errors that I do not want to see (see below).

What went wrong?

How do I backout, run this again? fix this?

---
The only thing I have not done is run this command, I don't know what it
does and what it is for, and I don't even know WTF? 'foobar' is in this
context or why it's in the documentation, someone please explain why I need
to include 'foobar' in this statement, what it refers to, and what it does
- PLEASE.

ALTER ROLE username SET search_path = airflow, foobar;
\

I don't think not running that is the problem.

---
Also this is very concerning: I am getting messages about MSSQL, but I
don't know where that comes from, is this trash that was not taken out?

```
change datetime to datetime2(6) on MSSQL tables
```

What is it doing here with MSSQL?  I don't have MSSQL installed.

---
Also this is very concerning, I am getting messages about MySQL (3), search
for 'MySQL' in this message.

```
1.  INFO  [alembic.runtime.migration] Running upgrade f2ca10b85618 ->
4addfa1236f1, Add fractional seconds to mysql tables
2.  INFO  [alembic.runtime.migration] Running upgrade 947454bf1dff ->
d2ae31099d61, Increase text size for MySQL (not relevant for other DBs'
text types)
3.  INFO  [alembic.runtime.migration] Running upgrade 05f30312d566 ->
f23433877c24, fix mysql not null constraint
```

Why is this? I am not using MySQL?  I don't even have it installed.

---
In addition I am getting messages about SQLite (1), but I don't know where
that comes from either, I'm not using SQLite.

```
INFO  [alembic.runtime.migration] Running upgrade f23433877c24 ->
856955da8476, fix sqlite foreign key
```
---
I want: 'airflow initdb' to run without any error messages, and not get any
message about:
```
1. MSSQL
2. MySQL
3. SQLite
```

Is this possible?

At a minimum, what went wrong here, and how do I fix this?
---

```
[airflow]$airflow initdb
[2020-02-13 07:04:55,706] {settings.py:253} INFO -
settings.configure_orm(): Using pool settings. pool_size=5,
max_overflow=10, pool_recycle=1800, pid=32019
DB: postgresql+psycopg2://airflow:***@<ip-address>/airflow
[2020-02-13 07:04:56,603] {db.py:368} INFO - Creating tables
INFO  [alembic.runtime.migration] Context impl PostgresqlImpl.
INFO  [alembic.runtime.migration] Will assume transactional DDL.
INFO  [alembic.runtime.migration] Running upgrade  -> e3a246e0dc1, current
schema
INFO  [alembic.runtime.migration] Running upgrade e3a246e0dc1 ->
1507a7289a2f, create is_encrypted
INFO  [alembic.runtime.migration] Running upgrade 1507a7289a2f ->
13eb55f81627, maintain history for compatibility with earlier migrations
INFO  [alembic.runtime.migration] Running upgrade 13eb55f81627 ->
338e90f54d61, More logging into task_instance
INFO  [alembic.runtime.migration] Running upgrade 338e90f54d61 ->
52d714495f0, job_id indices
INFO  [alembic.runtime.migration] Running upgrade 52d714495f0 ->
502898887f84, Adding extra to Log
INFO  [alembic.runtime.migration] Running upgrade 502898887f84 ->
1b38cef5b76e, add dagrun
INFO  [alembic.runtime.migration] Running upgrade 1b38cef5b76e ->
2e541a1dcfed, task_duration
INFO  [alembic.runtime.migration] Running upgrade 2e541a1dcfed ->
40e67319e3a9, dagrun_config
INFO  [alembic.runtime.migration] Running upgrade 40e67319e3a9 ->
561833c1c74b, add password column to user
INFO  [alembic.runtime.migration] Running upgrade 561833c1c74b ->
4446e08588, dagrun start end
INFO  [alembic.runtime.migration] Running upgrade 4446e08588 ->
bbc73705a13e, Add notification_sent column to sla_miss
INFO  [alembic.runtime.migration] Running upgrade bbc73705a13e ->
bba5a7cfc896, Add a column to track the encryption state of the 'Extra'
field in connection
INFO  [alembic.runtime.migration] Running upgrade bba5a7cfc896 ->
1968acfc09e3, add is_encrypted column to variable table
INFO  [alembic.runtime.migration] Running upgrade 1968acfc09e3 ->
2e82aab8ef20, rename user table
INFO  [alembic.runtime.migration] Running upgrade 2e82aab8ef20 ->
211e584da130, add TI state index
INFO  [alembic.runtime.migration] Running upgrade 211e584da130 ->
64de9cddf6c9, add task fails journal table
INFO  [alembic.runtime.migration] Running upgrade 64de9cddf6c9 ->
f2ca10b85618, add dag_stats table
INFO  [alembic.runtime.migration] Running upgrade f2ca10b85618 ->
4addfa1236f1, Add fractional seconds to mysql tables
INFO  [alembic.runtime.migration] Running upgrade 4addfa1236f1 ->
8504051e801b, xcom dag task indices
INFO  [alembic.runtime.migration] Running upgrade 8504051e801b ->
5e7d17757c7a, add pid field to TaskInstance
INFO  [alembic.runtime.migration] Running upgrade 5e7d17757c7a ->
127d2bf2dfa7, Add dag_id/state index on dag_run table
INFO  [alembic.runtime.migration] Running upgrade 127d2bf2dfa7 ->
cc1e65623dc7, add max tries column to task instance

ERROR [airflow.models.dagbag.DagBag] Failed to import:
/usr/local/lib/python3.6/site-packages/airflow/example_dags/example_subdag_operator.py
Traceback (most recent call last):
  File "/usr/local/lib/python3.6/site-packages/sqlalchemy/engine/base.py",
line 1246, in _execute_context
    cursor, statement, parameters, context
  File
"/usr/local/lib/python3.6/site-packages/sqlalchemy/engine/default.py", line
588, in do_execute
    cursor.execute(statement, parameters)
psycopg2.errors.UndefinedTable: relation "slot_pool" does not exist
LINE 2: FROM slot_pool
             ^

The above exception was the direct cause of the following exception:

Traceback (most recent call last):
  File "/usr/local/lib/python3.6/site-packages/airflow/models/dagbag.py",
line 243, in process_file
    m = imp.load_source(mod_name, filepath)
  File "/usr/lib64/python3.6/imp.py", line 172, in load_source
    module = _load(spec)
  File "<frozen importlib._bootstrap>", line 684, in _load
  File "<frozen importlib._bootstrap>", line 665, in _load_unlocked
  File "<frozen importlib._bootstrap_external>", line 678, in exec_module
  File "<frozen importlib._bootstrap>", line 219, in
_call_with_frames_removed
  File
"/usr/local/lib/python3.6/site-packages/airflow/example_dags/example_subdag_operator.py",
line 50, in <module>
    dag=dag,
  File "/usr/local/lib/python3.6/site-packages/airflow/utils/db.py", line
74, in wrapper
    return func(*args, **kwargs)
  File
"/usr/local/lib/python3.6/site-packages/airflow/utils/decorators.py", line
98, in wrapper
    result = func(*args, **kwargs)
  File
"/usr/local/lib/python3.6/site-packages/airflow/operators/subdag_operator.py",
line 77, in __init__
    .filter(Pool.pool == self.pool)
  File "/usr/local/lib/python3.6/site-packages/sqlalchemy/orm/query.py",
line 3287, in first
    ret = list(self[0:1])
  File "/usr/local/lib/python3.6/site-packages/sqlalchemy/orm/query.py",
line 3065, in __getitem__
    return list(res)
  File "/usr/local/lib/python3.6/site-packages/sqlalchemy/orm/query.py",
line 3389, in __iter__
    return self._execute_and_instances(context)
  File "/usr/local/lib/python3.6/site-packages/sqlalchemy/orm/query.py",
line 3414, in _execute_and_instances
    result = conn.execute(querycontext.statement, self._params)
  File "/usr/local/lib/python3.6/site-packages/sqlalchemy/engine/base.py",
line 982, in execute
    return meth(self, multiparams, params)
  File "/usr/local/lib/python3.6/site-packages/sqlalchemy/sql/elements.py",
line 293, in _execute_on_connection
    return connection._execute_clauseelement(self, multiparams, params)
  File "/usr/local/lib/python3.6/site-packages/sqlalchemy/engine/base.py",
line 1101, in _execute_clauseelement
    distilled_params,
  File "/usr/local/lib/python3.6/site-packages/sqlalchemy/engine/base.py",
line 1250, in _execute_context
    e, statement, parameters, cursor, context
  File "/usr/local/lib/python3.6/site-packages/sqlalchemy/engine/base.py",
line 1476, in _handle_dbapi_exception
    util.raise_from_cause(sqlalchemy_exception, exc_info)
  File "/usr/local/lib/python3.6/site-packages/sqlalchemy/util/compat.py",
line 398, in raise_from_cause
    reraise(type(exception), exception, tb=exc_tb, cause=cause)
  File "/usr/local/lib/python3.6/site-packages/sqlalchemy/util/compat.py",
line 152, in reraise
    raise value.with_traceback(tb)
  File "/usr/local/lib/python3.6/site-packages/sqlalchemy/engine/base.py",
line 1246, in _execute_context
    cursor, statement, parameters, context
  File
"/usr/local/lib/python3.6/site-packages/sqlalchemy/engine/default.py", line
588, in do_execute
    cursor.execute(statement, parameters)
sqlalchemy.exc.ProgrammingError: (psycopg2.errors.UndefinedTable) relation
"slot_pool" does not exist
LINE 2: FROM slot_pool
             ^

[SQL: SELECT slot_pool.id AS slot_pool_id, slot_pool.pool AS
slot_pool_pool, slot_pool.slots AS slot_pool_slots, slot_pool.description
AS slot_pool_description
FROM slot_pool
WHERE slot_pool.slots = %(slots_1)s AND slot_pool.pool = %(pool_1)s
 LIMIT %(param_1)s]
[parameters: {'slots_1': 1, 'pool_1': 'default_pool', 'param_1': 1}]
(Background on this error at: http://sqlalche.me/e/f405)
INFO  [alembic.runtime.migration] Running upgrade cc1e65623dc7 ->
bdaa763e6c56, Make xcom value column a large binary
INFO  [alembic.runtime.migration] Running upgrade bdaa763e6c56 ->
947454bf1dff, add ti job_id index
INFO  [alembic.runtime.migration] Running upgrade 947454bf1dff ->
d2ae31099d61, Increase text size for MySQL (not relevant for other DBs'
text types)
INFO  [alembic.runtime.migration] Running upgrade d2ae31099d61 ->
0e2a74e0fc9f, Add time zone awareness
INFO  [alembic.runtime.migration] Running upgrade d2ae31099d61 ->
33ae817a1ff4, kubernetes_resource_checkpointing
INFO  [alembic.runtime.migration] Running upgrade 33ae817a1ff4 ->
27c6a30d7c24, kubernetes_resource_checkpointing
INFO  [alembic.runtime.migration] Running upgrade 27c6a30d7c24 ->
86770d1215c0, add kubernetes scheduler uniqueness
INFO  [alembic.runtime.migration] Running upgrade 86770d1215c0,
0e2a74e0fc9f -> 05f30312d566, merge heads
INFO  [alembic.runtime.migration] Running upgrade 05f30312d566 ->
f23433877c24, fix mysql not null constraint
INFO  [alembic.runtime.migration] Running upgrade f23433877c24 ->
856955da8476, fix sqlite foreign key
INFO  [alembic.runtime.migration] Running upgrade 856955da8476 ->
9635ae0956e7, index-faskfail
INFO  [alembic.runtime.migration] Running upgrade 9635ae0956e7 ->
dd25f486b8ea, add idx_log_dag
INFO  [alembic.runtime.migration] Running upgrade dd25f486b8ea ->
bf00311e1990, add index to taskinstance
INFO  [alembic.runtime.migration] Running upgrade 9635ae0956e7 ->
0a2a5b66e19d, add task_reschedule table
INFO  [alembic.runtime.migration] Running upgrade 0a2a5b66e19d,
bf00311e1990 -> 03bc53e68815, merge_heads_2
INFO  [alembic.runtime.migration] Running upgrade 03bc53e68815 ->
41f5f12752f8, add superuser field
INFO  [alembic.runtime.migration] Running upgrade 41f5f12752f8 ->
c8ffec048a3b, add fields to dag
INFO  [alembic.runtime.migration] Running upgrade c8ffec048a3b ->
dd4ecb8fbee3, Add schedule interval to dag
INFO  [alembic.runtime.migration] Running upgrade dd4ecb8fbee3 ->
939bb1e647c8, task reschedule fk on cascade delete
INFO  [alembic.runtime.migration] Running upgrade 939bb1e647c8 ->
6e96a59344a4, Make TaskInstance.pool not nullable
INFO  [alembic.runtime.migration] Running upgrade 6e96a59344a4 ->
d38e04c12aa2, add serialized_dag table
Revision ID: d38e04c12aa2
Revises: 6e96a59344a4
Create Date: 2019-08-01 14:39:35.616417
INFO  [alembic.runtime.migration] Running upgrade d38e04c12aa2 ->
b3b105409875, add root_dag_id to DAG
Revision ID: b3b105409875
Revises: d38e04c12aa2
Create Date: 2019-09-28 23:20:01.744775
INFO  [alembic.runtime.migration] Running upgrade 6e96a59344a4 ->
74effc47d867, change datetime to datetime2(6) on MSSQL tables
INFO  [alembic.runtime.migration] Running upgrade 939bb1e647c8 ->
004c1210f153, increase queue name size limit
INFO  [alembic.runtime.migration] Running upgrade c8ffec048a3b ->
a56c9515abdc, Remove dag_stat table
INFO  [alembic.runtime.migration] Running upgrade a56c9515abdc,
004c1210f153, 74effc47d867, b3b105409875 -> 08364691d074, Merge the four
heads back together
INFO  [alembic.runtime.migration] Running upgrade 08364691d074 ->
fe461863935f, increase_length_for_connection_password
INFO  [alembic.runtime.migration] Running upgrade fe461863935f ->
7939bcff74ba, Add DagTags table
Done.
```

Re: Many many problems getting Postgresql working as a backend

Posted by Ash Berlin-Taylor <as...@apache.org>.
So two things:

1. The "Running upgrades $id -> $id" are saying that our migration tracker is running the migrations. If it's showing something about another database that's just the message. This is expected and when not on mysql/sqlite the migration will do nothing, but we still need to track that it's been applied even if it is a no-op (because migrations have in them a list of migrations they depend on, so we can't easily skip a migration, instead we make the migration do nothing when applied to the wrong database)
2. That error about "relation "slot_pool" does not exist" can be ignored, it's a "bug" (in that we show the errors from the dagbag/load the dag bag at all) -- everything handles the error and correctly applies the migration. See https://issues.apache.org/jira/browse/AIRFLOW-3797 for the fix.
In short: the important bit is the last line, "Done", and the fact that the exit code of the command was 0/success.
I agree though that for a first impression it's pretty terrible and we should fix it!
-ash
On Feb 13 2020, at 7:31 am, /dev /local/ca <de...@gmail.com> wrote:
> I am having problems setting up PG as a backend,
>
> I am getting all sorts of errors and messages about 'MSSQL', 'MySQL', 'SQLite',
>
> that I'm not even using, that is not even part of the equation here ,and I am very confused and behind schedule getting this beast up and working minimally.
> I set three connection strings in $AIRFLOW_HOME/airflow.cfg to point to Postgres.
> ```
> sql_alchemy_conn = postgresql+psycopg2://airflow:<password>@<ip-address>/airflow
> broker_url = postgresql+psycopg2://airflow:<password>@<ip-address>/airflow
> result_backend = postgresql+psycopg2://airflow:<password>@<ip-address>/airflow
> ```
>
> ---
> I am now running `$airflow initdb`
>
> After running, I see 23 tables (running as pg user: 'airflow': `SELECT tablename FROM pg_tables WHERE schemaname = 'public';`
> -----------------------
> alembic_version
> dag_pickle
> import_error
> job
> slot_pool
> chart
> known_event_type
> known_event
> xcom
> log
> dag_run
> sla_miss
> connection
> variable
> task_fail
> kube_resource_version
> kube_worker_uuid
> task_instance
> task_reschedule
> users
> serialized_dag
> dag
> dag_tag
> (23 rows)
>
> ---
> But the output indicates errors that I do not want to see (see below).
> What went wrong?
> How do I backout, run this again? fix this?
> ---
> The only thing I have not done is run this command, I don't know what it does and what it is for, and I don't even know WTF? 'foobar' is in this context or why it's in the documentation, someone please explain why I need to include 'foobar' in this statement, what it refers to, and what it does - PLEASE.
>
>
> ALTER ROLE username SET search_path = airflow, foobar;
> \
>
>
> I don't think not running that is the problem.
> ---
> Also this is very concerning: I am getting messages about MSSQL, but I don't know where that comes from, is this trash that was not taken out?
>
> ```
> change datetime to datetime2(6) on MSSQL tables
> ```
>
> What is it doing here with MSSQL? I don't have MSSQL installed.
> ---
> Also this is very concerning, I am getting messages about MySQL (3), search for 'MySQL' in this message.
>
> ```
> 1. INFO [alembic.runtime.migration] Running upgrade f2ca10b85618 -> 4addfa1236f1, Add fractional seconds to mysql tables
> 2. INFO [alembic.runtime.migration] Running upgrade 947454bf1dff -> d2ae31099d61, Increase text size for MySQL (not relevant for other DBs' text types)
> 3. INFO [alembic.runtime.migration] Running upgrade 05f30312d566 -> f23433877c24, fix mysql not null constraint
> ```
>
> Why is this? I am not using MySQL? I don't even have it installed.
> ---
> In addition I am getting messages about SQLite (1), but I don't know where that comes from either, I'm not using SQLite.
>
> ```
> INFO [alembic.runtime.migration] Running upgrade f23433877c24 -> 856955da8476, fix sqlite foreign key
> ```
> ---
> I want: 'airflow initdb' to run without any error messages, and not get any message about:
> ```
> 1. MSSQL
> 2. MySQL
> 3. SQLite
> ```
>
> Is this possible?
> At a minimum, what went wrong here, and how do I fix this?
> ---
>
> ```
> [airflow]$airflow initdb
> [2020-02-13 07:04:55,706] {settings.py:253} INFO - settings.configure_orm(): Using pool settings. pool_size=5, max_overflow=10, pool_recycle=1800, pid=32019
> DB: postgresql+psycopg2://airflow:***@<ip-address>/airflow
> [2020-02-13 07:04:56,603] {db.py:368} INFO - Creating tables
> INFO [alembic.runtime.migration] Context impl PostgresqlImpl.
> INFO [alembic.runtime.migration] Will assume transactional DDL.
> INFO [alembic.runtime.migration] Running upgrade -> e3a246e0dc1, current schema
> INFO [alembic.runtime.migration] Running upgrade e3a246e0dc1 -> 1507a7289a2f, create is_encrypted
> INFO [alembic.runtime.migration] Running upgrade 1507a7289a2f -> 13eb55f81627, maintain history for compatibility with earlier migrations
> INFO [alembic.runtime.migration] Running upgrade 13eb55f81627 -> 338e90f54d61, More logging into task_instance
> INFO [alembic.runtime.migration] Running upgrade 338e90f54d61 -> 52d714495f0, job_id indices
> INFO [alembic.runtime.migration] Running upgrade 52d714495f0 -> 502898887f84, Adding extra to Log
> INFO [alembic.runtime.migration] Running upgrade 502898887f84 -> 1b38cef5b76e, add dagrun
> INFO [alembic.runtime.migration] Running upgrade 1b38cef5b76e -> 2e541a1dcfed, task_duration
> INFO [alembic.runtime.migration] Running upgrade 2e541a1dcfed -> 40e67319e3a9, dagrun_config
> INFO [alembic.runtime.migration] Running upgrade 40e67319e3a9 -> 561833c1c74b, add password column to user
> INFO [alembic.runtime.migration] Running upgrade 561833c1c74b -> 4446e08588, dagrun start end
> INFO [alembic.runtime.migration] Running upgrade 4446e08588 -> bbc73705a13e, Add notification_sent column to sla_miss
> INFO [alembic.runtime.migration] Running upgrade bbc73705a13e -> bba5a7cfc896, Add a column to track the encryption state of the 'Extra' field in connection
> INFO [alembic.runtime.migration] Running upgrade bba5a7cfc896 -> 1968acfc09e3, add is_encrypted column to variable table
> INFO [alembic.runtime.migration] Running upgrade 1968acfc09e3 -> 2e82aab8ef20, rename user table
> INFO [alembic.runtime.migration] Running upgrade 2e82aab8ef20 -> 211e584da130, add TI state index
> INFO [alembic.runtime.migration] Running upgrade 211e584da130 -> 64de9cddf6c9, add task fails journal table
> INFO [alembic.runtime.migration] Running upgrade 64de9cddf6c9 -> f2ca10b85618, add dag_stats table
> INFO [alembic.runtime.migration] Running upgrade f2ca10b85618 -> 4addfa1236f1, Add fractional seconds to mysql tables
> INFO [alembic.runtime.migration] Running upgrade 4addfa1236f1 -> 8504051e801b, xcom dag task indices
> INFO [alembic.runtime.migration] Running upgrade 8504051e801b -> 5e7d17757c7a, add pid field to TaskInstance
> INFO [alembic.runtime.migration] Running upgrade 5e7d17757c7a -> 127d2bf2dfa7, Add dag_id/state index on dag_run table
> INFO [alembic.runtime.migration] Running upgrade 127d2bf2dfa7 -> cc1e65623dc7, add max tries column to task instance
>
> ERROR [airflow.models.dagbag.DagBag] Failed to import: /usr/local/lib/python3.6/site-packages/airflow/example_dags/example_subdag_operator.py
> Traceback (most recent call last):
> File "/usr/local/lib/python3.6/site-packages/sqlalchemy/engine/base.py", line 1246, in _execute_context
> cursor, statement, parameters, context
> File "/usr/local/lib/python3.6/site-packages/sqlalchemy/engine/default.py", line 588, in do_execute
> cursor.execute(statement, parameters)
> psycopg2.errors.UndefinedTable: relation "slot_pool" does not exist
> LINE 2: FROM slot_pool
> ^
>
> The above exception was the direct cause of the following exception:
> Traceback (most recent call last):
> File "/usr/local/lib/python3.6/site-packages/airflow/models/dagbag.py", line 243, in process_file
> m = imp.load_source(mod_name, filepath)
> File "/usr/lib64/python3.6/imp.py", line 172, in load_source
> module = _load(spec)
> File "<frozen importlib._bootstrap>", line 684, in _load
> File "<frozen importlib._bootstrap>", line 665, in _load_unlocked
> File "<frozen importlib._bootstrap_external>", line 678, in exec_module
> File "<frozen importlib._bootstrap>", line 219, in _call_with_frames_removed
> File "/usr/local/lib/python3.6/site-packages/airflow/example_dags/example_subdag_operator.py", line 50, in <module>
> dag=dag,
> File "/usr/local/lib/python3.6/site-packages/airflow/utils/db.py", line 74, in wrapper
> return func(*args, **kwargs)
> File "/usr/local/lib/python3.6/site-packages/airflow/utils/decorators.py", line 98, in wrapper
> result = func(*args, **kwargs)
> File "/usr/local/lib/python3.6/site-packages/airflow/operators/subdag_operator.py", line 77, in __init__
> .filter(Pool.pool == self.pool)
> File "/usr/local/lib/python3.6/site-packages/sqlalchemy/orm/query.py", line 3287, in first
> ret = list(self[0:1])
> File "/usr/local/lib/python3.6/site-packages/sqlalchemy/orm/query.py", line 3065, in __getitem__
> return list(res)
> File "/usr/local/lib/python3.6/site-packages/sqlalchemy/orm/query.py", line 3389, in __iter__
> return self._execute_and_instances(context)
> File "/usr/local/lib/python3.6/site-packages/sqlalchemy/orm/query.py", line 3414, in _execute_and_instances
> result = conn.execute(querycontext.statement, self._params)
> File "/usr/local/lib/python3.6/site-packages/sqlalchemy/engine/base.py", line 982, in execute
> return meth(self, multiparams, params)
> File "/usr/local/lib/python3.6/site-packages/sqlalchemy/sql/elements.py", line 293, in _execute_on_connection
> return connection._execute_clauseelement(self, multiparams, params)
> File "/usr/local/lib/python3.6/site-packages/sqlalchemy/engine/base.py", line 1101, in _execute_clauseelement
> distilled_params,
> File "/usr/local/lib/python3.6/site-packages/sqlalchemy/engine/base.py", line 1250, in _execute_context
> e, statement, parameters, cursor, context
> File "/usr/local/lib/python3.6/site-packages/sqlalchemy/engine/base.py", line 1476, in _handle_dbapi_exception
> util.raise_from_cause(sqlalchemy_exception, exc_info)
> File "/usr/local/lib/python3.6/site-packages/sqlalchemy/util/compat.py", line 398, in raise_from_cause
> reraise(type(exception), exception, tb=exc_tb, cause=cause)
> File "/usr/local/lib/python3.6/site-packages/sqlalchemy/util/compat.py", line 152, in reraise
> raise value.with_traceback(tb)
> File "/usr/local/lib/python3.6/site-packages/sqlalchemy/engine/base.py", line 1246, in _execute_context
> cursor, statement, parameters, context
> File "/usr/local/lib/python3.6/site-packages/sqlalchemy/engine/default.py", line 588, in do_execute
> cursor.execute(statement, parameters)
> sqlalchemy.exc.ProgrammingError: (psycopg2.errors.UndefinedTable) relation "slot_pool" does not exist
> LINE 2: FROM slot_pool
> ^
>
> [SQL: SELECT slot_pool.id (http://slot_pool.id) AS slot_pool_id, slot_pool.pool AS slot_pool_pool, slot_pool.slots AS slot_pool_slots, slot_pool.description AS slot_pool_description
> FROM slot_pool
> WHERE slot_pool.slots = %(slots_1)s AND slot_pool.pool = %(pool_1)s
> LIMIT %(param_1)s]
> [parameters: {'slots_1': 1, 'pool_1': 'default_pool', 'param_1': 1}]
> (Background on this error at: http://sqlalche.me/e/f405)
> INFO [alembic.runtime.migration] Running upgrade cc1e65623dc7 -> bdaa763e6c56, Make xcom value column a large binary
> INFO [alembic.runtime.migration] Running upgrade bdaa763e6c56 -> 947454bf1dff, add ti job_id index
> INFO [alembic.runtime.migration] Running upgrade 947454bf1dff -> d2ae31099d61, Increase text size for MySQL (not relevant for other DBs' text types)
> INFO [alembic.runtime.migration] Running upgrade d2ae31099d61 -> 0e2a74e0fc9f, Add time zone awareness
> INFO [alembic.runtime.migration] Running upgrade d2ae31099d61 -> 33ae817a1ff4, kubernetes_resource_checkpointing
> INFO [alembic.runtime.migration] Running upgrade 33ae817a1ff4 -> 27c6a30d7c24, kubernetes_resource_checkpointing
> INFO [alembic.runtime.migration] Running upgrade 27c6a30d7c24 -> 86770d1215c0, add kubernetes scheduler uniqueness
> INFO [alembic.runtime.migration] Running upgrade 86770d1215c0, 0e2a74e0fc9f -> 05f30312d566, merge heads
> INFO [alembic.runtime.migration] Running upgrade 05f30312d566 -> f23433877c24, fix mysql not null constraint
> INFO [alembic.runtime.migration] Running upgrade f23433877c24 -> 856955da8476, fix sqlite foreign key
> INFO [alembic.runtime.migration] Running upgrade 856955da8476 -> 9635ae0956e7, index-faskfail
> INFO [alembic.runtime.migration] Running upgrade 9635ae0956e7 -> dd25f486b8ea, add idx_log_dag
> INFO [alembic.runtime.migration] Running upgrade dd25f486b8ea -> bf00311e1990, add index to taskinstance
> INFO [alembic.runtime.migration] Running upgrade 9635ae0956e7 -> 0a2a5b66e19d, add task_reschedule table
> INFO [alembic.runtime.migration] Running upgrade 0a2a5b66e19d, bf00311e1990 -> 03bc53e68815, merge_heads_2
> INFO [alembic.runtime.migration] Running upgrade 03bc53e68815 -> 41f5f12752f8, add superuser field
> INFO [alembic.runtime.migration] Running upgrade 41f5f12752f8 -> c8ffec048a3b, add fields to dag
> INFO [alembic.runtime.migration] Running upgrade c8ffec048a3b -> dd4ecb8fbee3, Add schedule interval to dag
> INFO [alembic.runtime.migration] Running upgrade dd4ecb8fbee3 -> 939bb1e647c8, task reschedule fk on cascade delete
> INFO [alembic.runtime.migration] Running upgrade 939bb1e647c8 -> 6e96a59344a4, Make TaskInstance.pool not nullable
> INFO [alembic.runtime.migration] Running upgrade 6e96a59344a4 -> d38e04c12aa2, add serialized_dag table
> Revision ID: d38e04c12aa2
> Revises: 6e96a59344a4
> Create Date: 2019-08-01 14:39:35.616417
> INFO [alembic.runtime.migration] Running upgrade d38e04c12aa2 -> b3b105409875, add root_dag_id to DAG
> Revision ID: b3b105409875
> Revises: d38e04c12aa2
> Create Date: 2019-09-28 23:20:01.744775
> INFO [alembic.runtime.migration] Running upgrade 6e96a59344a4 -> 74effc47d867, change datetime to datetime2(6) on MSSQL tables
> INFO [alembic.runtime.migration] Running upgrade 939bb1e647c8 -> 004c1210f153, increase queue name size limit
> INFO [alembic.runtime.migration] Running upgrade c8ffec048a3b -> a56c9515abdc, Remove dag_stat table
> INFO [alembic.runtime.migration] Running upgrade a56c9515abdc, 004c1210f153, 74effc47d867, b3b105409875 -> 08364691d074, Merge the four heads back together
> INFO [alembic.runtime.migration] Running upgrade 08364691d074 -> fe461863935f, increase_length_for_connection_password
> INFO [alembic.runtime.migration] Running upgrade fe461863935f -> 7939bcff74ba, Add DagTags table
> Done.
> ```
>
>
>