You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 5 Next »

Status

Current state[One of "Under Discussion"]

Discussion thread: here (<- link to https://mail-archives.apache.org/mod_mbox/flink-dev/)

JIRA: Unable to render Jira issues macro, execution error.

Released: 1.11

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 be 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 an `AbstractJDBCCatalog` interface and a `PostgresJDBCCatalog` implementation. With such a fundamental work, implementations for other relational db can be added easily later.

Design

`PostgresJDBCCatalog` 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.

Metaspace Mapping

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 query 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.

Configurations

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

  1. base jdbc url
  2. username
  3. password

Database will be default to username if it's not provided.

A Flink client can have multiple `PostgresJDBCCatalog`s to connect to different pg instances.

Interfaces, Classes, and Modules


// flink-jdbc module
public AbstractJDBCCatalog extends AbstractCatalog {
	public AbstractJDBCCatalog(String catalogName, String defaultDatabase, String userName, String pwd, String baseUrl) 

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

	void close();
}


// a new flink-connector-postgres module
public PostgresJDBCCatalog extends AbstractJDBCCatalog {
	PostgresJDBCCatalog(String catalogName, String defaultDatabase, String userName, String pwd, String baseUrl) {
		
	}

	List<String> listDatabases() {
		// translate to "SELECT datname FROM pg_database;"
	}

	boolean databaseExists(String dbname) {
		return listDatabases.contains(dbname);
	}

	CatalogTable getTable(ObjectPath op) {
		// translate to result set metadata of "SELECT * FROM <tablename>;"
	}

	List<String> listTables() {
		for each db in listDatabases:
			// translate to "SELECT * FROM information_schema.tables
						WHERE table_type = 'BASE TABLE' AND table_schema = <schema_name>
						ORDER BY table_type, table_name;"
	}

	boolean tableExists(ObjectPath path) {
		databaseExists(path.getDatabaseName())
		return listTables().contains(path.getObjectName());
	}
}


The base url from constructor should be without databases, like "jdbc:postgresql://localhost:5432/" or "jdbc:postgresql://localhost:5432"

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

Data Type Mapping


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 timezonetimestamp with timezo (depending on pg server timezone)

date

date

time

time without timezone

time with timezonetime with 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

Versions

flink-connector-postgres module will provide a fat jar including latest postgres jdbc driver and relying on the driver itself for backward compatibility. Users can also build jars with their own versions.


Rejected Alternatives

n/a

  • No labels