apache-airflow-providers-postgres 6.2.3rc1__py3-none-any.whl → 6.3.0rc1__py3-none-any.whl

This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.

Potentially problematic release.


This version of apache-airflow-providers-postgres might be problematic. Click here for more details.

@@ -29,7 +29,7 @@ from airflow import __version__ as airflow_version
29
29
 
30
30
  __all__ = ["__version__"]
31
31
 
32
- __version__ = "6.2.3"
32
+ __version__ = "6.3.0"
33
33
 
34
34
  if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse(
35
35
  "2.10.0"
@@ -16,9 +16,11 @@
16
16
  # under the License.
17
17
  from __future__ import annotations
18
18
 
19
+ from collections.abc import Callable
20
+
19
21
  from methodtools import lru_cache
20
22
 
21
- from airflow.providers.common.sql.dialects.dialect import Dialect
23
+ from airflow.providers.common.sql.dialects.dialect import Dialect, T
22
24
 
23
25
 
24
26
  class PostgresDialect(Dialect):
@@ -39,22 +41,71 @@ class PostgresDialect(Dialect):
39
41
  """
40
42
  if schema is None:
41
43
  table, schema = self.extract_schema_from_table(table)
42
- sql = """
43
- select kcu.column_name
44
- from information_schema.table_constraints tco
45
- join information_schema.key_column_usage kcu
46
- on kcu.constraint_name = tco.constraint_name
47
- and kcu.constraint_schema = tco.constraint_schema
48
- and kcu.constraint_name = tco.constraint_name
49
- where tco.constraint_type = 'PRIMARY KEY'
50
- and kcu.table_schema = %s
51
- and kcu.table_name = %s
52
- """
53
44
  pk_columns = [
54
- row[0] for row in self.get_records(sql, (self.unescape_word(schema), self.unescape_word(table)))
45
+ row[0]
46
+ for row in self.get_records(
47
+ """
48
+ select kcu.column_name
49
+ from information_schema.table_constraints tco
50
+ join information_schema.key_column_usage kcu
51
+ on kcu.constraint_name = tco.constraint_name
52
+ and kcu.constraint_schema = tco.constraint_schema
53
+ and kcu.constraint_name = tco.constraint_name
54
+ where tco.constraint_type = 'PRIMARY KEY'
55
+ and kcu.table_schema = %s
56
+ and kcu.table_name = %s
57
+ order by kcu.ordinal_position
58
+ """,
59
+ (self.unescape_word(schema), self.unescape_word(table)),
60
+ )
55
61
  ]
56
62
  return pk_columns or None
57
63
 
64
+ @staticmethod
65
+ def _to_row(row):
66
+ return {
67
+ "name": row[0],
68
+ "type": row[1],
69
+ "nullable": row[2].casefold() == "yes",
70
+ "default": row[3],
71
+ "autoincrement": row[4].casefold() == "always",
72
+ "identity": row[5].casefold() == "yes",
73
+ }
74
+
75
+ @lru_cache(maxsize=None)
76
+ def get_column_names(
77
+ self, table: str, schema: str | None = None, predicate: Callable[[T], bool] = lambda column: True
78
+ ) -> list[str] | None:
79
+ if schema is None:
80
+ table, schema = self.extract_schema_from_table(table)
81
+
82
+ column_names = list(
83
+ row["name"]
84
+ for row in filter(
85
+ predicate,
86
+ map(
87
+ self._to_row,
88
+ self.get_records(
89
+ """
90
+ select column_name,
91
+ data_type,
92
+ is_nullable,
93
+ column_default,
94
+ is_generated,
95
+ is_identity
96
+ from information_schema.columns
97
+ where table_schema = %s
98
+ and table_name = %s
99
+ order by ordinal_position
100
+ """,
101
+ (self.unescape_word(schema), self.unescape_word(table)),
102
+ ),
103
+ ),
104
+ )
105
+ )
106
+ self.log.debug("Column names for table '%s': %s", table, column_names)
107
+ return column_names
108
+
58
109
  def generate_replace_sql(self, table, values, target_fields, **kwargs) -> str:
59
110
  """
60
111
  Generate the REPLACE SQL statement.
@@ -21,12 +21,13 @@ import os
21
21
  from collections.abc import Mapping
22
22
  from contextlib import closing
23
23
  from copy import deepcopy
24
- from typing import TYPE_CHECKING, Any, Literal, TypeAlias, cast, overload
24
+ from typing import TYPE_CHECKING, Any, Literal, Protocol, TypeAlias, cast, overload
25
25
 
26
26
  import psycopg2
27
27
  import psycopg2.extensions
28
28
  import psycopg2.extras
29
- from psycopg2.extras import DictCursor, NamedTupleCursor, RealDictCursor
29
+ from more_itertools import chunked
30
+ from psycopg2.extras import DictCursor, NamedTupleCursor, RealDictCursor, execute_batch
30
31
  from sqlalchemy.engine import URL
31
32
 
32
33
  from airflow.exceptions import (
@@ -36,20 +37,66 @@ from airflow.exceptions import (
36
37
  from airflow.providers.common.sql.hooks.sql import DbApiHook
37
38
  from airflow.providers.postgres.dialects.postgres import PostgresDialect
38
39
 
40
+ USE_PSYCOPG3: bool
41
+ try:
42
+ import psycopg as psycopg # needed for patching in unit tests
43
+ import sqlalchemy
44
+ from packaging.version import Version
45
+
46
+ sqlalchemy_version = Version(sqlalchemy.__version__)
47
+ is_sqla2 = (sqlalchemy_version.major, sqlalchemy_version.minor, sqlalchemy_version.micro) >= (2, 0, 0)
48
+
49
+ USE_PSYCOPG3 = is_sqla2 # implicitly includes `and bool(psycopg)` since the import above succeeded
50
+ except (ImportError, ModuleNotFoundError):
51
+ USE_PSYCOPG3 = False
52
+
53
+ if USE_PSYCOPG3:
54
+ from psycopg.rows import dict_row, namedtuple_row
55
+ from psycopg.types.json import register_default_adapters
56
+
39
57
  if TYPE_CHECKING:
40
58
  from pandas import DataFrame as PandasDataFrame
41
59
  from polars import DataFrame as PolarsDataFrame
42
- from psycopg2.extensions import connection
43
60
 
44
61
  from airflow.providers.common.sql.dialects.dialect import Dialect
45
62
  from airflow.providers.openlineage.sqlparser import DatabaseInfo
46
63
 
64
+ if USE_PSYCOPG3:
65
+ from psycopg.errors import Diagnostic
66
+
47
67
  try:
48
68
  from airflow.sdk import Connection
49
69
  except ImportError:
50
70
  from airflow.models.connection import Connection # type: ignore[assignment]
51
71
 
52
72
  CursorType: TypeAlias = DictCursor | RealDictCursor | NamedTupleCursor
73
+ CursorRow: TypeAlias = dict[str, Any] | tuple[Any, ...]
74
+
75
+
76
+ class CompatConnection(Protocol):
77
+ """Protocol for type hinting psycopg2 and psycopg3 connection objects."""
78
+
79
+ def cursor(self, *args, **kwargs) -> Any: ...
80
+ def commit(self) -> None: ...
81
+ def close(self) -> None: ...
82
+
83
+ # Context manager support
84
+ def __enter__(self) -> CompatConnection: ...
85
+ def __exit__(self, exc_type: Any, exc_val: Any, exc_tb: Any) -> None: ...
86
+
87
+ # Common properties
88
+ @property
89
+ def notices(self) -> list[Any]: ...
90
+
91
+ # psycopg3 specific (optional)
92
+ @property
93
+ def adapters(self) -> Any: ...
94
+
95
+ @property
96
+ def row_factory(self) -> Any: ...
97
+
98
+ # Optional method for psycopg3
99
+ def add_notice_handler(self, handler: Any) -> None: ...
53
100
 
54
101
 
55
102
  class PostgresHook(DbApiHook):
@@ -58,8 +105,8 @@ class PostgresHook(DbApiHook):
58
105
 
59
106
  You can specify ssl parameters in the extra field of your connection
60
107
  as ``{"sslmode": "require", "sslcert": "/path/to/cert.pem", etc}``.
61
- Also you can choose cursor as ``{"cursor": "dictcursor"}``. Refer to the
62
- psycopg2.extras for more details.
108
+ Also, you can choose cursor as ``{"cursor": "dictcursor"}``. Refer to the
109
+ psycopg2.extras or psycopg.rows for more details.
63
110
 
64
111
  Note: For Redshift, use keepalives_idle in the extra connection parameters
65
112
  and set it to less than 300 seconds.
@@ -93,6 +140,8 @@ class PostgresHook(DbApiHook):
93
140
 
94
141
  conn_name_attr = "postgres_conn_id"
95
142
  default_conn_name = "postgres_default"
143
+ default_client_log_level = "warning"
144
+ default_connector_version: int = 2
96
145
  conn_type = "postgres"
97
146
  hook_name = "Postgres"
98
147
  supports_autocommit = True
@@ -113,11 +162,15 @@ class PostgresHook(DbApiHook):
113
162
  self, *args, options: str | None = None, enable_log_db_messages: bool = False, **kwargs
114
163
  ) -> None:
115
164
  super().__init__(*args, **kwargs)
116
- self.conn: connection = None
165
+ self.conn: CompatConnection | None = None
117
166
  self.database: str | None = kwargs.pop("database", None)
118
167
  self.options = options
119
168
  self.enable_log_db_messages = enable_log_db_messages
120
169
 
170
+ @staticmethod
171
+ def __cast_nullable(value, dst_type: type) -> Any:
172
+ return dst_type(value) if value is not None else None
173
+
121
174
  @property
122
175
  def sqlalchemy_url(self) -> URL:
123
176
  conn = self.connection
@@ -125,12 +178,12 @@ class PostgresHook(DbApiHook):
125
178
  if not isinstance(query, dict):
126
179
  raise AirflowException("The parameter 'sqlalchemy_query' must be of type dict!")
127
180
  return URL.create(
128
- drivername="postgresql",
129
- username=conn.login,
130
- password=conn.password,
131
- host=conn.host,
132
- port=conn.port,
133
- database=self.database or conn.schema,
181
+ drivername="postgresql+psycopg" if USE_PSYCOPG3 else "postgresql",
182
+ username=self.__cast_nullable(conn.login, str),
183
+ password=self.__cast_nullable(conn.password, str),
184
+ host=self.__cast_nullable(conn.host, str),
185
+ port=self.__cast_nullable(conn.port, int),
186
+ database=self.__cast_nullable(self.database, str) or self.__cast_nullable(conn.schema, str),
134
187
  query=query,
135
188
  )
136
189
 
@@ -142,8 +195,24 @@ class PostgresHook(DbApiHook):
142
195
  def dialect(self) -> Dialect:
143
196
  return PostgresDialect(self)
144
197
 
198
+ def _notice_handler(self, notice: Diagnostic):
199
+ """Handle notices from the database and log them."""
200
+ self.log.info(str(notice.message_primary).strip())
201
+
145
202
  def _get_cursor(self, raw_cursor: str) -> CursorType:
146
203
  _cursor = raw_cursor.lower()
204
+ if USE_PSYCOPG3:
205
+ if _cursor == "dictcursor":
206
+ return dict_row
207
+ if _cursor == "namedtuplecursor":
208
+ return namedtuple_row
209
+ if _cursor == "realdictcursor":
210
+ raise AirflowException(
211
+ "realdictcursor is not supported with psycopg3. Use dictcursor instead."
212
+ )
213
+ valid_cursors = "dictcursor, namedtuplecursor"
214
+ raise ValueError(f"Invalid cursor passed {_cursor}. Valid options are: {valid_cursors}")
215
+
147
216
  cursor_types = {
148
217
  "dictcursor": psycopg2.extras.DictCursor,
149
218
  "realdictcursor": psycopg2.extras.RealDictCursor,
@@ -154,33 +223,63 @@ class PostgresHook(DbApiHook):
154
223
  valid_cursors = ", ".join(cursor_types.keys())
155
224
  raise ValueError(f"Invalid cursor passed {_cursor}. Valid options are: {valid_cursors}")
156
225
 
157
- def get_conn(self) -> connection:
226
+ def _generate_cursor_name(self):
227
+ """Generate a unique name for server-side cursor."""
228
+ import uuid
229
+
230
+ return f"airflow_cursor_{uuid.uuid4().hex}"
231
+
232
+ def get_conn(self) -> CompatConnection:
158
233
  """Establish a connection to a postgres database."""
159
234
  conn = deepcopy(self.connection)
160
235
 
161
- # check for authentication via AWS IAM
162
236
  if conn.extra_dejson.get("iam", False):
163
- conn.login, conn.password, conn.port = self.get_iam_token(conn)
237
+ login, password, port = self.get_iam_token(conn)
238
+ conn.login = cast("Any", login)
239
+ conn.password = cast("Any", password)
240
+ conn.port = cast("Any", port)
164
241
 
165
- conn_args = {
242
+ conn_args: dict[str, Any] = {
166
243
  "host": conn.host,
167
244
  "user": conn.login,
168
245
  "password": conn.password,
169
246
  "dbname": self.database or conn.schema,
170
247
  "port": conn.port,
171
248
  }
172
- raw_cursor = conn.extra_dejson.get("cursor", False)
173
- if raw_cursor:
174
- conn_args["cursor_factory"] = self._get_cursor(raw_cursor)
175
249
 
176
250
  if self.options:
177
251
  conn_args["options"] = self.options
178
252
 
253
+ # Add extra connection arguments
179
254
  for arg_name, arg_val in conn.extra_dejson.items():
180
255
  if arg_name not in self.ignored_extra_options:
181
256
  conn_args[arg_name] = arg_val
182
257
 
183
- self.conn = psycopg2.connect(**conn_args)
258
+ if USE_PSYCOPG3:
259
+ from psycopg.connection import Connection as pgConnection
260
+
261
+ raw_cursor = conn.extra_dejson.get("cursor")
262
+ if raw_cursor:
263
+ conn_args["row_factory"] = self._get_cursor(raw_cursor)
264
+
265
+ # Use Any type for the connection args to avoid type conflicts
266
+ connection = pgConnection.connect(**cast("Any", conn_args))
267
+ self.conn = cast("CompatConnection", connection)
268
+
269
+ # Register JSON handlers for both json and jsonb types
270
+ # This ensures JSON data is properly decoded from bytes to Python objects
271
+ register_default_adapters(connection)
272
+
273
+ # Add the notice handler AFTER the connection is established
274
+ if self.enable_log_db_messages and hasattr(self.conn, "add_notice_handler"):
275
+ self.conn.add_notice_handler(self._notice_handler)
276
+ else: # psycopg2
277
+ raw_cursor = conn.extra_dejson.get("cursor", False)
278
+ if raw_cursor:
279
+ conn_args["cursor_factory"] = self._get_cursor(raw_cursor)
280
+
281
+ self.conn = cast("CompatConnection", psycopg2.connect(**conn_args))
282
+
184
283
  return self.conn
185
284
 
186
285
  @overload
@@ -231,7 +330,9 @@ class PostgresHook(DbApiHook):
231
330
 
232
331
  engine = self.get_sqlalchemy_engine()
233
332
  with engine.connect() as conn:
234
- return psql.read_sql(sql, con=conn, params=parameters, **kwargs)
333
+ if isinstance(sql, list):
334
+ sql = "; ".join(sql) # Or handle multiple queries differently
335
+ return cast("PandasDataFrame", psql.read_sql(sql, con=conn, params=parameters, **kwargs))
235
336
 
236
337
  elif df_type == "polars":
237
338
  return self._get_polars_df(sql, parameters, **kwargs)
@@ -241,7 +342,7 @@ class PostgresHook(DbApiHook):
241
342
 
242
343
  def copy_expert(self, sql: str, filename: str) -> None:
243
344
  """
244
- Execute SQL using psycopg2's ``copy_expert`` method.
345
+ Execute SQL using psycopg's ``copy_expert`` method.
245
346
 
246
347
  Necessary to execute COPY command without access to a superuser.
247
348
 
@@ -252,14 +353,38 @@ class PostgresHook(DbApiHook):
252
353
  they have to check its existence by themselves.
253
354
  """
254
355
  self.log.info("Running copy expert: %s, filename: %s", sql, filename)
255
- if not os.path.isfile(filename):
256
- with open(filename, "w"):
257
- pass
258
-
259
- with open(filename, "r+") as file, closing(self.get_conn()) as conn, closing(conn.cursor()) as cur:
260
- cur.copy_expert(sql, file)
261
- file.truncate(file.tell())
262
- conn.commit()
356
+ if USE_PSYCOPG3:
357
+ if " from stdin" in sql.lower():
358
+ # Handle COPY FROM STDIN: read from the file and write to the database.
359
+ if not os.path.isfile(filename):
360
+ with open(filename, "w"):
361
+ pass # Create an empty file to prevent errors.
362
+
363
+ with open(filename, "rb") as file, self.get_conn() as conn, conn.cursor() as cur:
364
+ with cur.copy(sql) as copy:
365
+ while data := file.read(8192):
366
+ copy.write(data)
367
+ conn.commit()
368
+ else:
369
+ # Handle COPY TO STDOUT: read from the database and write to the file.
370
+ with open(filename, "wb") as file, self.get_conn() as conn, conn.cursor() as cur:
371
+ with cur.copy(sql) as copy:
372
+ for data in copy:
373
+ file.write(data)
374
+ conn.commit()
375
+ else:
376
+ if not os.path.isfile(filename):
377
+ with open(filename, "w"):
378
+ pass
379
+
380
+ with (
381
+ open(filename, "r+") as file,
382
+ closing(self.get_conn()) as conn,
383
+ closing(conn.cursor()) as cur,
384
+ ):
385
+ cur.copy_expert(sql, file)
386
+ file.truncate(file.tell())
387
+ conn.commit()
263
388
 
264
389
  def get_uri(self) -> str:
265
390
  """
@@ -278,9 +403,9 @@ class PostgresHook(DbApiHook):
278
403
  self.copy_expert(f"COPY {table} TO STDOUT", tmp_file)
279
404
 
280
405
  @staticmethod
281
- def _serialize_cell(cell: object, conn: connection | None = None) -> Any:
406
+ def _serialize_cell_ppg2(cell: object, conn: CompatConnection | None = None) -> Any:
282
407
  """
283
- Serialize a cell.
408
+ Serialize a cell using psycopg2.
284
409
 
285
410
  Psycopg2 adapts all arguments to the ``execute()`` method internally,
286
411
  hence we return the cell without any conversion.
@@ -297,6 +422,24 @@ class PostgresHook(DbApiHook):
297
422
  """
298
423
  return cell
299
424
 
425
+ @staticmethod
426
+ def _serialize_cell_ppg3(cell: object, conn: CompatConnection | None = None) -> Any:
427
+ """Serialize a cell using psycopg3."""
428
+ if isinstance(cell, (dict, list)):
429
+ try:
430
+ from psycopg.types.json import Json
431
+
432
+ return Json(cell)
433
+ except ImportError:
434
+ return cell
435
+ return cell
436
+
437
+ @staticmethod
438
+ def _serialize_cell(cell: object, conn: Any | None = None) -> Any:
439
+ if USE_PSYCOPG3:
440
+ return PostgresHook._serialize_cell_ppg3(cell, conn)
441
+ return PostgresHook._serialize_cell_ppg2(cell, conn)
442
+
300
443
  def get_iam_token(self, conn: Connection) -> tuple[str, str, int]:
301
444
  """
302
445
  Get the IAM token.
@@ -420,11 +563,91 @@ class PostgresHook(DbApiHook):
420
563
  }
421
564
 
422
565
  def get_db_log_messages(self, conn) -> None:
566
+ """Log database messages."""
567
+ if not self.enable_log_db_messages:
568
+ return
569
+
570
+ if USE_PSYCOPG3:
571
+ self.log.debug(
572
+ "With psycopg3, database notices are logged upon creation (via self._notice_handler)."
573
+ )
574
+ return
575
+
576
+ for output in conn.notices:
577
+ self.log.info(output)
578
+
579
+ def insert_rows(
580
+ self,
581
+ table,
582
+ rows,
583
+ target_fields=None,
584
+ commit_every=1000,
585
+ replace=False,
586
+ *,
587
+ executemany=False,
588
+ fast_executemany=False,
589
+ autocommit=False,
590
+ **kwargs,
591
+ ):
423
592
  """
424
- Log all database messages sent to the client during the session.
593
+ Insert a collection of tuples into a table.
594
+
595
+ Rows are inserted in chunks, each chunk (of size ``commit_every``) is
596
+ done in a new transaction.
425
597
 
426
- :param conn: Connection object
598
+ :param table: Name of the target table
599
+ :param rows: The rows to insert into the table
600
+ :param target_fields: The names of the columns to fill in the table
601
+ :param commit_every: The maximum number of rows to insert in one
602
+ transaction. Set to 0 to insert all rows in one transaction.
603
+ :param replace: Whether to replace instead of insert
604
+ :param executemany: If True, all rows are inserted at once in
605
+ chunks defined by the commit_every parameter. This only works if all rows
606
+ have same number of column names, but leads to better performance.
607
+ :param fast_executemany: If True, rows will be inserted using an optimized
608
+ bulk execution strategy (``psycopg2.extras.execute_batch``). This can
609
+ significantly improve performance for large inserts. If set to False,
610
+ the method falls back to the default implementation from
611
+ ``DbApiHook.insert_rows``.
612
+ :param autocommit: What to set the connection's autocommit setting to
613
+ before executing the query.
427
614
  """
428
- if self.enable_log_db_messages:
429
- for output in conn.notices:
430
- self.log.info(output)
615
+ # if fast_executemany is disabled, defer to default implementation of insert_rows in DbApiHook
616
+ if not fast_executemany:
617
+ return super().insert_rows(
618
+ table,
619
+ rows,
620
+ target_fields=target_fields,
621
+ commit_every=commit_every,
622
+ replace=replace,
623
+ executemany=executemany,
624
+ autocommit=autocommit,
625
+ **kwargs,
626
+ )
627
+
628
+ # if fast_executemany is enabled, use optimized execute_batch from psycopg
629
+ nb_rows = 0
630
+ with self._create_autocommit_connection(autocommit) as conn:
631
+ conn.commit()
632
+ with closing(conn.cursor()) as cur:
633
+ for chunked_rows in chunked(rows, commit_every):
634
+ values = list(
635
+ map(
636
+ lambda row: self._serialize_cells(row, conn),
637
+ chunked_rows,
638
+ )
639
+ )
640
+ sql = self._generate_insert_sql(table, values[0], target_fields, replace, **kwargs)
641
+ self.log.debug("Generated sql: %s", sql)
642
+
643
+ try:
644
+ execute_batch(cur, sql, values, page_size=commit_every)
645
+ except Exception as e:
646
+ self.log.error("Generated sql: %s", sql)
647
+ self.log.error("Parameters: %s", values)
648
+ raise e
649
+
650
+ conn.commit()
651
+ nb_rows += len(chunked_rows)
652
+ self.log.info("Loaded %s rows into %s so far", nb_rows, table)
653
+ self.log.info("Done loading. Loaded a total of %s rows into %s", nb_rows, table)
@@ -1,6 +1,6 @@
1
1
  Metadata-Version: 2.4
2
2
  Name: apache-airflow-providers-postgres
3
- Version: 6.2.3rc1
3
+ Version: 6.3.0rc1
4
4
  Summary: Provider package apache-airflow-providers-postgres for Apache Airflow
5
5
  Keywords: airflow-provider,postgres,airflow,integration
6
6
  Author-email: Apache Software Foundation <dev@airflow.apache.org>
@@ -30,9 +30,10 @@ Requires-Dist: apache-airflow-providers-openlineage ; extra == "openlineage"
30
30
  Requires-Dist: pandas>=2.1.2 ; extra == "pandas" and ( python_version <"3.13")
31
31
  Requires-Dist: pandas>=2.2.3 ; extra == "pandas" and ( python_version >="3.13")
32
32
  Requires-Dist: polars>=1.26.0 ; extra == "polars"
33
+ Requires-Dist: psycopg[binary]>=3.2.9 ; extra == "psycopg"
33
34
  Project-URL: Bug Tracker, https://github.com/apache/airflow/issues
34
- Project-URL: Changelog, https://airflow.staged.apache.org/docs/apache-airflow-providers-postgres/6.2.3/changelog.html
35
- Project-URL: Documentation, https://airflow.staged.apache.org/docs/apache-airflow-providers-postgres/6.2.3
35
+ Project-URL: Changelog, https://airflow.staged.apache.org/docs/apache-airflow-providers-postgres/6.3.0/changelog.html
36
+ Project-URL: Documentation, https://airflow.staged.apache.org/docs/apache-airflow-providers-postgres/6.3.0
36
37
  Project-URL: Mastodon, https://fosstodon.org/@airflow
37
38
  Project-URL: Slack Chat, https://s.apache.org/airflow-slack
38
39
  Project-URL: Source Code, https://github.com/apache/airflow
@@ -41,6 +42,7 @@ Provides-Extra: amazon
41
42
  Provides-Extra: openlineage
42
43
  Provides-Extra: pandas
43
44
  Provides-Extra: polars
45
+ Provides-Extra: psycopg
44
46
 
45
47
 
46
48
  .. Licensed to the Apache Software Foundation (ASF) under one
@@ -67,9 +69,8 @@ Provides-Extra: polars
67
69
 
68
70
  Package ``apache-airflow-providers-postgres``
69
71
 
70
- Release: ``6.2.3``
72
+ Release: ``6.3.0``
71
73
 
72
- Release Date: ``|PypiReleaseDate|``
73
74
 
74
75
  `PostgreSQL <https://www.postgresql.org/>`__
75
76
 
@@ -81,12 +82,12 @@ This is a provider package for ``postgres`` provider. All classes for this provi
81
82
  are in ``airflow.providers.postgres`` python package.
82
83
 
83
84
  You can find package information and changelog for the provider
84
- in the `documentation <https://airflow.apache.org/docs/apache-airflow-providers-postgres/6.2.3/>`_.
85
+ in the `documentation <https://airflow.apache.org/docs/apache-airflow-providers-postgres/6.3.0/>`_.
85
86
 
86
87
  Installation
87
88
  ------------
88
89
 
89
- You can install this package on top of an existing Airflow 2 installation (see ``Requirements`` below
90
+ You can install this package on top of an existing Airflow installation (see ``Requirements`` below
90
91
  for the minimum Airflow version supported) via
91
92
  ``pip install apache-airflow-providers-postgres``
92
93
 
@@ -127,5 +128,5 @@ Dependent package
127
128
  ============================================================================================================== ===============
128
129
 
129
130
  The changelog for the provider package can be found in the
130
- `changelog <https://airflow.apache.org/docs/apache-airflow-providers-postgres/6.2.3/changelog.html>`_.
131
+ `changelog <https://airflow.apache.org/docs/apache-airflow-providers-postgres/6.3.0/changelog.html>`_.
131
132
 
@@ -1,13 +1,13 @@
1
1
  airflow/providers/postgres/LICENSE,sha256=gXPVwptPlW1TJ4HSuG5OMPg-a3h43OGMkZRR1rpwfJA,10850
2
- airflow/providers/postgres/__init__.py,sha256=AGf6WOLMkDqDRwtMgRbiZ8xMIXq-l-oK88uqcaOVUsk,1497
2
+ airflow/providers/postgres/__init__.py,sha256=EBlBQkd4wsb32h6i--UVow1UxLY3oTKzALQLc1L5W7s,1497
3
3
  airflow/providers/postgres/get_provider_info.py,sha256=qEEYbClLY3-NH40dBk2u_nOIfvfEIHdXaWsIJ8J5Z68,2626
4
4
  airflow/providers/postgres/assets/__init__.py,sha256=9hdXHABrVpkbpjZgUft39kOFL2xSGeG4GEua0Hmelus,785
5
5
  airflow/providers/postgres/assets/postgres.py,sha256=XNhOJCbOA_soaaiS73JjULMqAM_7PBryhToe8FJREA0,1522
6
6
  airflow/providers/postgres/dialects/__init__.py,sha256=9hdXHABrVpkbpjZgUft39kOFL2xSGeG4GEua0Hmelus,785
7
- airflow/providers/postgres/dialects/postgres.py,sha256=8ygA2Jq2WBPhNOISU8YyitLmahxWnPGYNt8OaU_CBFI,3764
7
+ airflow/providers/postgres/dialects/postgres.py,sha256=WhRMbSQv2u373R62e3buE5J1L3j-XGHhZxrWc8Gtc5s,5551
8
8
  airflow/providers/postgres/hooks/__init__.py,sha256=mlJxuZLkd5x-iq2SBwD3mvRQpt3YR7wjz_nceyF1IaI,787
9
- airflow/providers/postgres/hooks/postgres.py,sha256=_iulx6UoEBcGP2_wxmdoXnYM9I97I4dGGAAkw4mmLTg,17737
10
- apache_airflow_providers_postgres-6.2.3rc1.dist-info/entry_points.txt,sha256=dhtJi6PTWHd6BwKhmI4OtSPvQVI_p0yYWI0eba83HqY,104
11
- apache_airflow_providers_postgres-6.2.3rc1.dist-info/WHEEL,sha256=G2gURzTEtmeR8nrdXUJfNiB3VYVxigPQ-bEQujpNiNs,82
12
- apache_airflow_providers_postgres-6.2.3rc1.dist-info/METADATA,sha256=YNhTrvJxeTW0-3ovqYYXUfoky0h7JjnSPWNp8JL3Nhk,6264
13
- apache_airflow_providers_postgres-6.2.3rc1.dist-info/RECORD,,
9
+ airflow/providers/postgres/hooks/postgres.py,sha256=YwA99YPz_ZpCvBn5yyw4ykhJ9hMGNvEsMmAf5nsfVE8,26908
10
+ apache_airflow_providers_postgres-6.3.0rc1.dist-info/entry_points.txt,sha256=dhtJi6PTWHd6BwKhmI4OtSPvQVI_p0yYWI0eba83HqY,104
11
+ apache_airflow_providers_postgres-6.3.0rc1.dist-info/WHEEL,sha256=G2gURzTEtmeR8nrdXUJfNiB3VYVxigPQ-bEQujpNiNs,82
12
+ apache_airflow_providers_postgres-6.3.0rc1.dist-info/METADATA,sha256=-yMIPOzaKFmX8R55IV-td9dERi-HzogSQcxtawr2FAU,6309
13
+ apache_airflow_providers_postgres-6.3.0rc1.dist-info/RECORD,,