Discussion thread
Vote thread
JIRA

Unable to render Jira issues macro, execution error.

Release

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

Currently users have to manually create schemas in Flink source/sink mirroring tables in their relational databases in use cases like direct JDBC read/write and consuming CDC. Many users have complaint about this process as the manual work is unnecessary and redundant. Users also need to manually keep up their Flink job with schema changes in databases and figuring out type matches. Any mismatch can lead to a failing Flink job at runtime instead of compile time. All these have been quite unpleasant, resulting in a broken user experience.

The problem is actually general to Flink on all external systems. We just focus on solving that in relational databases in this FLIP.

We want to provide a JDBC catalog interface for Flink to connect to all kinds of relational databases, enabling Flink SQL to 1) retrieve table schema automatically without requiring user inputs DDL 2) check at compile time for any potential schema errors. It will greatly streamline user experiences when using Flink to deal with popular relational databases like Postgres, MySQL, MariaDB, AWS Aurora, etc.

Proposal

We propose to add a `JDBCCatalog` user-face catalog and a `PostgresJDBCCatalog` implementation. With such a fundamental work, implementations for other relational db can be added easily later.

Design

`JDBCCatalog` will be a read only catalog that supports the following operations on db and table:

  • database
    • listDatabases
    • databaseExists
  • table
    • getTable
    • listTables
    • tableExists

Users can only read these objects from databases, but not able to manipulate them yet in this FLIP.

Configurations

To configure a `JDBCCatalog`, users only need to provide

  1. base jdbc url
  2. username
  3. password

default_database will be default to db instance's default db if it's not provided. E.g. in postgres, it's the username

A Flink client can have multiple `JDBCCatalog`s to connect to different db instances.

Interfaces, Classes, and Modules

classes will be all in flink-jdbc module.


public JDBCCatalog extends AbstractCatalog {
	JDBCDialect dialect;
	Catalog internal;
	
	public JDBCCatalog(String catalogName, String defaultDatabase, String userName, String pwd, String baseUrl) {
		JDBCDialect d = JDBCDialects.get(baseUrl).get();

		switch(d) {
			case postgres:
				internal = new PostgresJDBCCatalog(...);
				break;
			case derby:
				...
		}
	} 

	void open() {
		// verify jdbc connection to fail earlier
	}

	void close();

	Optional<TableFactory> getTableFactory() {
		return Optional.of(new JDBCTableSourceSinkFactory());
	}

	List<String> listDatabases() {
		return internal.listDatabases();
	}

	boolean databaseExists(String dbname) {
		// just try to make a connection, true if succeeded, false otherwise
	}

	CatalogTable getTable(ObjectPath op) {
		return internal.getTable(op);
	}

	List<String> listTables(String db) {
		return internal.listTables(db);
	}
}

public PostgresJDBCCatalog extends JDBCCatalog {
	// cannot be instantiated by users
	protected PostgresJDBCCatalog(String catalogName, String defaultDatabase, String userName, String pwd, String baseUrl) {
	}

	boolean tableExists(ObjectPath path) {
		// execute and parse result
	}

	List<String> listDatabases() {
		// execute and parse result
	}

	CatalogTable getTable(ObjectPath op) {
		// execute and parse result
		// map data types and build catalog table
	}

	List<String> listTables() {
		// execute and parse result
	}
}


The base url from constructor should be without databases, like "jdbc:postgresql://localhost:5432/" or "jdbc:postgresql://localhost:5432". The reason being that it's normal for users to switch databases within a catalog  and Postgres doesn't allow changing databases within a session, and thus we need to establish multiple sessions based on the base url for different dbs in a pg instance.

The full url when connecting to the db will be with database name, like "jdbc:postgresql://localhost:5432/db"

Using the catalog

Table API


// java
tEnv.registerCatalog(name, new JDBCCatalog(...))

SQL CLI yaml configs

catalogs:
	- name: ...
	  type: jdbc
	  username: xxx
	  password: yyy
	  base-url: jdbc:<db_type>://<ip>:<port>
      default-database: ... # optional, dbms specific, will be access id in postgres by default

Versions

We rely on the driver itself for backward compatibility. Users can also build jars with their own versions.

Postgres has an additional name space as `schema` besides database. A pg instance can have multiple dbs, each db can have multiple schemas with a default one named "postgres", each schema can have multiple tables.

A few facts to know about how jdbc works in Postgres

1) jdbc connection to Postgres have to be for a specific database without schema name. If there's no db specified in the url, the default db is the username.

2) when querying a table in Postgres, users can use either <schema.table> or just <table>. The schema is optional and defaults to "postgres"  


Based on the above two facts, we propose to match database name space between Flink and Postgres, and make Postgres's <schema.table> corresponding to Flink's table name, as shown below.

Flink Catalog Metaspace Structure

Postgres Metaspace Structure 

catalog name (defined in Flink only)

n/a

database name

database name

table name

<schema name.>table name


The full path of Postgres table in Flink would be "<catalog>.<db>.`<schema.table>`"  if schema is specified.


Flink Type

Postgres Type

smallint

smallint

int

integer

bigint

bigint

float

real

double

double precision

decimal(p,s)

numeric(p,s)

boolean

boolean

timestamp

timestamp without timezone

timestamp with local timezonetimestamp with timezone (depending on pg server timezone)

date

date

time

time without timezone

intervalinterval

string

text

varchar(n)

character varying

char

char, character

binary/varbinary/bytes

bytea

array

[] (array of all the above primitive types)


Flink types not listed above are not supported by Postgres.


*timestamp/time precision is 0-6


Rejected Alternatives

n/a