Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.


Discussion thread-
Vote thread-
JIRA-
https://lists.apache.org/thread/d1owrg8zh77v0xygcpb93fxt0jpjdkb3
Vote threadhttps://lists.apache.org/thread/7jbmg22lnww31sbfdzztwrzgm6bkhjrj
JIRA

Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyFLINK-31496

Release1.18.0Release-


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

...

draw.io Board Diagram
bordertrue
diagramNamedriver-gateway
simpleViewerfalse
width
linksauto
tbstyletop
lboxtrue
diagramWidth281
revision1

There are 6 7 main classes in Flink Jdbc Driver: FlinkDriver, FlinkDataSource, FlinkConnection, FlinkStatement, FlinkResultSet, FlinkDatabaseMetaData and FlinkResultSetMetaData which implement jdbc interface Driver, DataSource, Connection, Statement, ResultSet, DatabaseMetaData and ResultSetMetaData.

  1. FlinkDriver parses gateway address from url, and creates FlinkConnection
  2. FlinkDataSource manages connection pool for flink jdbc, it will create specific count of connections and choose one for client directly.
  3. FlinkConnection creates Executor according to gateway address. When the Connections is closed, it can close the connection with gateway by Executor
  4. FlinkStatement can get Executor from FlinkConnection, and submit sql query to it. After query is executed, FlinkStatement can get StatementResult from Executor, and create FlinkResultSet
  5. FlinkResultSet is an iterator, it gets results from StatementResult and return them to users
  6. FlinkDatabaseMetaData provides meta data of catalogs, databases and tables
  7. FlinkResultSetMetaData provides meta data of ResultSet such as columns

...

draw.io Board Diagram
bordertrue
diagramNameclasses-driver
simpleViewerfalse
width
linksauto
tbstyletop
lboxtrue
diagramWidth701
revision2

User Case

Flink Jdbc Driver module will be packaged into an independent jar file such as flink-table-jdbc-driver-{version}.jar , which will contains classes of flink jdbc driver and shaded flink classes such as data type. User only need to add jdbc dependency in pom.xml or add the jar in the classpath of external Jdbc Tools such as sqlite.

Code Block
languagejava
<dependency>
    <groupId>org.apache.flink</groupId>
    <artifactId>flink-table-jdbc-driver</artifactId>
    <version>${flink.version}</version>
</dependency>

More information about Flink Jdbc Driver

3

There're Sessions and Operations in SqlGateway. SqlGateway will open a Session for each FlinkConnection , and then do multiple Operations in one Session. When users create a FlinkConnection by FlinkDriver with SqlGateway, it will open an exist or a new Session. Any time users want to issue SQL statements to the database, they require a FlinkStatement instance from FlinkConnection. Once users have a FlinkStatement , they can use issue a query. This will return a FlinkResultSet instance, which contains the entire result. Each operation such as the execution query(Flink job), fetching results in FlinkResultSet will be an Operation in the Session of SqlGateway .

User Case

Flink Jdbc Driver module will be packaged into an independent jar file such as flink-table-jdbc-driver-{version}.jar , which will contains classes of flink jdbc driver and shaded flink classes such as data type. User only need to add jdbc dependency in pom.xml or add the jar in the classpath of external Jdbc Tools such as sqlite.

Code Block
languagejava
<dependency>
    <groupId>org.apache.flink</groupId>
    <artifactId>flink-table-jdbc-driver</artifactId>
    <version>${flink.version}</version>
</dependency>

More information about Flink Jdbc Driver

  1. Driver Name: org.apache.flink.table.jdbc.FlinkDriver
  2. Flink Connection URL: Users can use default catalog and database directly, or set them in url. Users can also set custom parameters in url to open the session in SqlGateway with key1=val1&key2=val2&... 
  3. Driver Name: org.apache.flink.table.jdbc.FlinkDriver
  4. Flink Connection URL: Users can use default catalog and database directly, or set them in url. Users can also set custom parameters in url to open the session in SqlGateway with key1=val1&key2=val2&... 
    1. jdbc:flink://{sql-gateway.host}:{sql-gateway.port}?key1=val1&key2=val2
    2. jdbc:flink://{sql-gateway.host}:{sql-gateway.port}/{catalog name}?key1=val1&key2=val2
    3. jdbc:flink://{sql-gateway.host}:{sql-gateway.port}/{catalog name}/{database name}?key1=val1&key2=val2
  5. Currently SqlGateway does not support authentication, user  and password in connection are invalid.
  6. Use Flink Jdbc Driver in Java code

    Code Block
    languagejava
    String driverName = "org.apache.flink.table.jdbc.FlinkDriver"; String url = "
    1. jdbc:flink://{sql-gateway.host}:{sql-gateway.port}?key1=val1&key2=val2
    1. jdbc:flink://{sql-gateway.host}:{sql-gateway.port}/{catalog
    name}/{database
    1. name}?key1=val1&key2=val2
    2. jdbc:flink://{sql-gateway.host}:{sql-gateway.port}/{catalog name}/{database name}?key1=val1&key2=val2
  7. Currently SqlGateway does not support authentication, user  and password in connection are invalid.
  8. Use Flink Jdbc Driver in Java code

    Code Block
    languagejava
    String driverName = "org.apache.flink.table.jdbc.FlinkDriver";
    String url = "jdbc:flink://{sql-gateway.host}:{sql-gateway.port}/{catalog name}/{database name}?key1=val1&key2=val2";
    Class.forName(driverName);
    
    try (Connection connection = DriverManager.getConnection(url)) {
        try (Statement statement = connection.createStatement()) {
            try (";
    Class.forName(driverName);
    
    try (Connection connection = DriverManager.getConnection(url)) {
        try (Statement statement = connection.createStatement()) {
            try (ResultSet resultSet = statement.execute("SELECT * FROM {Your Table}")) {
                while (resultSet.hasNext()) {
                    Do your work ....
                }
            }
    
    		try (ResultSet resultSet  }
    }
  9. User can also add the flink-table-jdbc-driver-{version}.jar to the classpath of external jdbc tools.

Data Types

The following basic flink data types are supported to convert to sql data type in this FLIP, and more types can be supported as needed in the future.

...

Flink Data Type

...

Java Sql Data Type

...

Java Data Type

...

CharType/VarCharType

...

CHAR/VARCHAR

...

String

...

BooleanType

...

BOOLEAN

...

Boolean

...

TinyIntType

...

TINYINT

...

Byte

...

SmallIntType

...

SMALLINT

...

Short

...

IntType

...

INTEGER

...

Int

...

BigIntType

...

BIGINT

...

Long

...

FloatType

...

FLOAT

...

Float

...

DoubleType

...

DOUBLE

...

Double

...

DecimalType

...

DECIMAL

...

BigDecimal

...

BinaryType/VarBinaryType

...

BINARY/VARBINARY

...

byte[]

...

DateType

...

DATE

...

Date

...

TimeType

...

TIME

...

Time

...

TimestampType

...

TIMESTAMP

...

Timestamp

...

ZonedTimestapType

...

TIMESTAMP_WITH_TIMEZONE

...

ZonedDateTime

...

ArrayType

...

ARRAY

...

Array

...

RowType

...

ROW

...

Row(Flink Row Data)

...

MapType

...

MAP

...

Map<K, V>

  1. = statement.execute("SELECT * FROM T1 JOIN T2 on T1.id=T2.id ...")) {
    			while (resultSet.hasNext()) {
                    Do your work ...
                }
    		}
        }
    }


  2. User can also add the flink-table-jdbc-driver-{version}.jar to the classpath of external jdbc tools.

Data Types

The following basic flink data types are supported to convert to sql data type in this FLIP, and more types can be supported as needed in the future.

Flink Data Type

Java Sql Data Type

Java Data Type

CharType/VarCharType

CHAR/VARCHAR

String

BooleanType

BOOLEAN

Boolean

TinyIntType

TINYINT

Byte

SmallIntType

SMALLINT

Short

IntType

INTEGER

Int

BigIntType

BIGINT

Long

FloatType

FLOAT

Float

DoubleType

DOUBLE

Double

DecimalType

DECIMAL

BigDecimal

BinaryType/VarBinaryType

BINARY/VARBINARY

byte[]

DateType

DATE

Date

TimeType

TIME

Time

TimestampType

TIMESTAMP

Timestamp

ZonedTimestampType

TIMESTAMP_WITH_TIMEZONE

OffsetDateTime

LocalZonedTimestampType

TIMESTAMP_WITH_LOCAL_TIMEZONE

Timestamp

ArrayType

ARRAY

Array

RowType

ROW(Not in java.sql.Types)

Row(Flink Row Data)

MapType

MAP

Map<K, V>

Currently TIMESTAMP_WITH_LOCAL_TIMEZONE is not exist in java.sql.Types, but it is supported by Flink. Users can define a field as (f TIMESTAMP(p) WITH LOCAL TIME ZONE) and set time zone through the connection parameters or dynamic parameters in console by table.local-time-zone. After that, users can get Timestamp which will be automatically converted from stored time data into specific Timestamp  according to the given time zone.

Java Sql Interfaces

There are many methods in Jdbc Driver, while this FLIP only implement the basic methods first and more methods will be implemented later when they are needed. 

  • Methods in FlinkDriver 
Code Block
languagejava
/* Jdbc Driver for flink sql gateway. Only Batch Mode queries are supported. If you force to submit streaming queries, you may get unrecognized updates, deletions and other results in FlinkResultSet. */
public class FlinkDriver implements Driver {
    /* Connect sql gateway with given url and open/create session with given priperties. */
	@Override
    public Connection connect(String url, Properties info) throws SQLException;
}
  • Methods in FlinkDataSource
Code Block
languagejava
/* Jdbc DataSource manages connections for client, we can support more operations in it in the future. */
public class FlinkDataSource implements DataSource {
	/* The max count of connections which the data source holds. */
	private int maxActive;

	/* Set the url of connection. */
	public synchronized void setUrl(String url);

	/* Set the driver class name for the source. */
	public synchronized void setDriverClassName(String driverClassName); 

	/* Set the max active connection for the source. */
    public synchronized void setMaxActive(int maxActive);

    /* Get a connection from data source. */
	@Override
    public Connection getConnection() throws SQLException;
 	@Override 

Public Interface

There are many methods in Jdbc Driver, while this FLIP only implement the basic methods first and more methods will be implemented later when they are needed. 

  • Methods in FlinkDriver 
Code Block
languagejava
/* Jdbc Driver for flink sql gateway */
public class FlinkDriver implements Driver {
    /* Connect sql gateway with given url and open/create session with given priperties. */
	@Override
    public Connection connectgetConnection(String urlusername, PropertiesString infopassword) throws SQLException;
}
  • Methods in FlinkConnection 

...

  • Methods in FlinkResultSet : FlinkResultSet only supports fetching data from iterator StatementResult , it supports getXXX methods and doesn't support deleting, updating or moving the cursor. Compare with ResultSet , there is getKind method in FlinkResultSet to get the RowKind of current record.
Code Block
languagejava
/* ResultSet for flink jdbc driver. */
public class FlinkResultSet implements ResultSet {
	/* Get the row kind for the current record* ResultSet for flink jdbc driver. Only Batch Mode queries are supported. If you force to submit streaming queries, you may get unrecognized updates, deletions and other results. */
	public class RowKind getKind();
FlinkResultSet implements ResultSet {
    /* Return true if there are more resuts in result iterator. */
 	@Override 
    public boolean next() throws SQLException;
    
    /* Close the fetch result operation. */
 	@Override 
    public void close() throws SQLException;
    
    /* Get different values according to data type and column index. */
 	@Override 
    public <V> V getXXX(int columnIndex) throws SQLException.
    
    /* Get different values according to data type and column name. */
 	@Override
    public <V> V getXXX(String columnName) throws SQLException.
}

...

Code Block
languagejava
/* ResultSetMetaData in flink sql driver. */
public class FlinkResultSetMetaData implements ResultSetMetaData {
    /* Get column count in the result set. */
 	@Override 
    public int getColumnCount() throws SQLException;
    
    /* If the column may be null. */
 	@Override 
    public int isNullable(int column) throws SQLException;
    
    /* Get display size for the column. */
 	@Override 
    public int getColumnDisplaySize(int column) throws SQLException;
    
    /* Get column name according to column index. */
 	@Override 
    public String getColumnLabel(int column) throws SQLException;
    public String getColumnName(int column) throws SQLException;
    
    /* Get precision for the column index. */
 	@Override 
    public int getPrecision(int column) throws SQLException;
    
    /* Get column type id for the column index. */
 	@Override 
    public int getColumnType int getColumnType(int column) throws SQLException;
    
    /* Get column type name for the column index. */
 	@Override 
    public String getColumnTypeName(int column) throws SQLException;
    
    /* Get column type class name for the column index. */
 	@Override 
    public String getColumnTypeName(int column) throws SQLException;
    
    /* Get column type class name for the column index. */
 	@Override 
    public String getColumnClassName(int column) throws SQLException;
}

Unsupported Features

...

 getColumnClassName(int column) throws SQLException;
}

Unsupported Features

  1. Don't support transaction such as commit, rollback
  2. Don't support prepare statement, prepare call and etc operations
  3. Don't support management operations such as savepoint and etc

Exception Handling

When an error occurs, Flink Jdbc Driver mainly throws the following exceptions

SQLState ClassSQLState SubClassReasonExceptionOperations
22000 to 02H according to different errorsDescription of data conversion errorSQLDataExceptionGet data error from ResultSet in methods getXXX
0A000Specific feature is not supportedSQLFeatureNotSupportedExceptionAll unimplemented methods will throw this exception
58004The exception or error message from GatewaySQLNonTransientExceptionGateway throws an exception or returns an error message when executing the query
08006The session is not exist in Gateway and client need to create new connection to itSQLNonTransientConnectionExceptionGateway is restarted and the client need to create new connection

We can continue to subdivide and throw different exceptions according to the error information returned by the Gateway in Flink Jdbc Driver in the future

...



[1] https://github.com/ververica/flink-sql-gateway

...