diff --git a/.gitignore b/.gitignore index a1c2a23..e0bf0fc 100644 --- a/.gitignore +++ b/.gitignore @@ -21,3 +21,10 @@ # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml hs_err_pid* + +# IntelliJ IDEA Files +.idea +*.iml + +# Maven Files +target diff --git a/README.md b/README.md index bba3d2f..59b8069 100644 --- a/README.md +++ b/README.md @@ -1 +1,113 @@ -# flink-jdbc-driver \ No newline at end of file +# Flink JDBC Driver + +Flink JDBC driver is a Java library for accessing and manipulating [Apache Flink](https://flink.apache.org/) clusters by connecting to a [Flink SQL gateway](https://github.com/ververica/flink-sql-gateway) as the JDBC server. + +This project is at an early stage. Feel free to file an issue if you meet any problems or have any suggestions. + +# Usage + +Before using Flink JDBC driver, you need to start a [Flink SQL gateway](https://github.com/ververica/flink-sql-gateway) as the JDBC server and binds it with your Flink cluster. We now assume that you have a gateway started and connected to a running Flink cluster. + +## Use with a JDBC Tool +### Use with Beeline + +Beeline is the command line tool for accessing [Apache Hive](https://hive.apache.org/), but it also supports general JDBC drivers. To install Hive and beeline, see [Hive documentation](https://cwiki.apache.org/confluence/display/Hive/GettingStarted#GettingStarted-InstallationandConfiguration). + +1. Download flink-jdbc-driver-(VERSION).jar from the [download page](https://github.com/ververica/flink-jdbc-driver/releases) and add it to `$HIVE_HOME/lib`. +2. Run beeline and connect to a Flink SQL gateway. You can specify the planner (`blink` or `old`) in the query parameter of the url. As Flink SQL gateway currently ignores user names and passwords, just leave them empty. + ``` + beeline> !connect jdbc:flink://localhost:8083?planner=blink + ``` +3. Execute any statement you want. + +**Sample Commands** +``` +Beeline version 2.2.0 by Apache Hive +beeline> !connect jdbc:flink://localhost:8083?planner=blink +Connecting to jdbc:flink://localhost:8083?planner=blink +Enter username for jdbc:flink://localhost:8083?planner=blink: +Enter password for jdbc:flink://localhost:8083?planner=blink: +Connected to: Apache Flink (version 1.10.0) +Driver: Flink Driver (version 0.1) +Transaction isolation: TRANSACTION_REPEATABLE_READ +0: jdbc:flink://localhost:8083> CREATE TABLE T( +. . . . . . . . . . . . . . . > a INT, +. . . . . . . . . . . . . . . > b VARCHAR(10) +. . . . . . . . . . . . . . . > ) WITH ( +. . . . . . . . . . . . . . . > 'connector.type' = 'filesystem', +. . . . . . . . . . . . . . . > 'connector.path' = 'file:///tmp/T.csv', +. . . . . . . . . . . . . . . > 'format.type' = 'csv', +. . . . . . . . . . . . . . . > 'format.derive-schema' = 'true' +. . . . . . . . . . . . . . . > ); +No rows affected (0.158 seconds) +0: jdbc:flink://localhost:8083> INSERT INTO T VALUES (1, 'Hi'), (2, 'Hello'); +No rows affected (4.747 seconds) +0: jdbc:flink://localhost:8083> SELECT * FROM T; ++----+--------+--+ +| a | b | ++----+--------+--+ +| 1 | Hi | +| 2 | Hello | ++----+--------+--+ +2 rows selected (0.994 seconds) +0: jdbc:flink://localhost:8083> +``` + +### Use with Tableau +[Tableau](https://www.tableau.com/) is an interactive data visualization software. It supports *Other Database (JDBC)* connection from version 2018.3. You'll need Tableau with version >= 2018.3 to use Flink JDBC driver. For general usage of *Other Database (JDBC)* in Tableau, see [Tableau documentation](https://help.tableau.com/current/pro/desktop/en-us/examples_otherdatabases_jdbc.htm). + +1. Download flink-jdbc-driver-(VERSION).jar from the [download page](https://github.com/ververica/flink-jdbc-driver/releases) and add it to Tableau driver path. + * Windows: `C:\Program Files\Tableau\Drivers` + * Mac: `~/Library/Tableau/Drivers` + * Linux: `/opt/tableau/tableau_driver/jdbc` +2. Select *Other Database (JDBC)* under *Connect* and fill in the url of Flink SQL gateway. You can specify the planner (`blink` or `old`) in the query parameter of the url. Select *SQL92* dialect and leave user name and password empty. +3. Hit *Login* button and use Tableau as usual. + +### Use with other JDBC Tools +Flink JDBC driver is a library for accessing Flink clusters through the JDBC API. Any tool supporting JDBC API can be used with Flink JDBC driver and [Flink SQL gateway](https://github.com/ververica/flink-sql-gateway). See the documentation of your desired tool on how to use a JDBC driver. + +## Use with Java + +Flink JDBC driver is a library for accessing Flink clusters through the JDBC API. For the general usage of JDBC in Java, see [JDBC tutorial](https://docs.oracle.com/javase/tutorial/jdbc/index.html) or [Oracle JDBC documentation](https://www.oracle.com/technetwork/java/javase/tech/index-jsp-136101.html). + +1. Download flink-jdbc-driver-(VERSION).jar from the [download page](https://github.com/ververica/flink-jdbc-driver/releases) and add it to your classpath. +2. Connect to a Flink SQL gateway in your Java code. You can specify the planner (`blink` or `old`) in the query parameter of the url. +3. Execute any statement you want. + +**Sample.java** +```java +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.Statement; + +public class Sample { + public static void main(String[] args) throws Exception { + Connection connection = DriverManager.getConnection("jdbc:flink://localhost:8083?planner=blink"); + Statement statement = connection.createStatement(); + + statement.executeUpdate("CREATE TABLE T(\n" + + " a INT,\n" + + " b VARCHAR(10)\n" + + ") WITH (\n" + + " 'connector.type' = 'filesystem',\n" + + " 'connector.path' = 'file:///tmp/T.csv',\n" + + " 'format.type' = 'csv',\n" + + " 'format.derive-schema' = 'true'\n" + + ")"); + ResultSet rs = statement.executeQuery("SELECT * FROM T"); + while (rs.next()) { + System.out.println(rs.getInt(1) + ", " + rs.getString(2)); + } + + statement.close(); + connection.close(); + } +} +``` + +**Output** +``` +1, Hi +2, Hello +``` diff --git a/pom.xml b/pom.xml new file mode 100644 index 0000000..7ee14c6 --- /dev/null +++ b/pom.xml @@ -0,0 +1,303 @@ + + + + + 4.0.0 + + flink-jdbc-driver + com.ververica + flink-jdbc-driver + 0.1-SNAPSHOT + + jar + + + 1.10.0 + 0.1-SNAPSHOT + 1.8 + 1.7.15 + 2.11 + + + ${java.version} + ${java.version} + + + 1C + + ${flink.forkCount} + true + + + + + + + org.apache.flink + flink-runtime_${scala.binary.version} + ${flink.version} + + + org.apache.flink + flink-table-common + ${flink.version} + + + com.ververica + flink-sql-gateway + ${flink.sql.gateway.version} + + + org.slf4j + slf4j-nop + ${slf4j.version} + runtime + + + + + + org.apache.flink + flink-table-api-java + ${flink.version} + test + + + org.apache.flink + flink-table-planner-blink_${scala.binary.version} + ${flink.version} + test + + + org.apache.flink + flink-test-utils-junit + ${flink.version} + test + + + org.apache.flink + flink-test-utils_${scala.binary.version} + ${flink.version} + test + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.22.1 + + ${test.groups} + ${flink.forkCount} + ${flink.reuseForks} + + 0${surefire.forkNumber} + ${test.scheduler.type} + + -Xms256m -Xmx2048m -Dmvn.forkNumber=${surefire.forkNumber} -XX:+UseG1GC + + + + + default-test + test + + test + + + + **/*Test.* + + + + + + integration-tests + integration-test + + test + + + + **/*ITCase.* + + false + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + false + + + org.slf4j:slf4j-api + org.slf4j:slf4j-nop + org.apache.flink:flink-shaded-asm-7 + org.apache.flink:flink-shaded-jackson + org.apache.flink:flink-shaded-netty + org.apache.flink:flink-annotations + org.apache.flink:flink-metrics-core + org.apache.flink:flink-core + org.apache.flink:flink-runtime_${scala.binary.version} + org.apache.flink:flink-table-common + com.ververica:flink-sql-gateway + + + + + *:* + + ** + + + + org.apache.flink:flink-runtime_${scala.binary.version} + + org/apache/flink/runtime/rest/** + org/apache/flink/runtime/net/** + org/apache/flink/runtime/io/network/netty/** + org/apache/flink/runtime/util/** + + + + com.ververica:flink-sql-gateway + + com/ververica/flink/table/gateway/rest/** + + + + + + + + + + org.apache.rat + apache-rat-plugin + 0.12 + false + + + verify + + check + + + + + false + 0 + + + + AL2 + Apache License 2.0 + + + Licensed to the Apache Software Foundation (ASF) under one + + + + + + Apache License 2.0 + + + + + **/.*/** + **/*.prefs + **/*.log + + **/README.md + .github/** + + **/target/** + + **/.idea/** + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.17 + + + com.puppycrawl.tools + checkstyle + + 8.14 + + + + + validate + validate + + check + + + + + /tools/maven/suppressions.xml + true + /tools/maven/checkstyle.xml + true + true + + + + + + diff --git a/src/main/java/com/ververica/flink/table/jdbc/FlinkConnection.java b/src/main/java/com/ververica/flink/table/jdbc/FlinkConnection.java new file mode 100644 index 0000000..56f2d75 --- /dev/null +++ b/src/main/java/com/ververica/flink/table/jdbc/FlinkConnection.java @@ -0,0 +1,465 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.flink.table.jdbc; + +import com.ververica.flink.table.gateway.rest.message.StatementExecuteResponseBody; +import com.ververica.flink.table.gateway.rest.result.ResultSet; +import com.ververica.flink.table.jdbc.rest.RestUtils; +import com.ververica.flink.table.jdbc.rest.SessionClient; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.types.Either; +import org.apache.flink.util.Preconditions; + +import java.sql.Array; +import java.sql.Blob; +import java.sql.CallableStatement; +import java.sql.Clob; +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.NClob; +import java.sql.PreparedStatement; +import java.sql.SQLClientInfoException; +import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; +import java.sql.SQLWarning; +import java.sql.SQLXML; +import java.sql.Savepoint; +import java.sql.Statement; +import java.sql.Struct; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.Executor; + +/** + * Flink JDBC connection. + */ +public class FlinkConnection implements Connection { + + private final SessionClient session; + private boolean closed; + + public FlinkConnection(String url) throws Exception { + this.closed = false; + this.session = createSession(url.substring(FlinkDriver.URL_PREFIX.length())); + } + + @Override + public Statement createStatement() throws SQLException { + return new FlinkStatement(session, this); + } + + @Override + public void close() throws SQLException { + if (closed) { + return; + } + + try { + session.close(); + } catch (Exception e) { + throw new SQLException(e); + } + closed = true; + } + + @Override + public boolean isClosed() throws SQLException { + return closed; + } + + @Override + public void setCatalog(String catalog) throws SQLException { + session.submitStatement("USE CATALOG " + catalog); + } + + @Override + public String getCatalog() throws SQLException { + StatementExecuteResponseBody response; + response = session.submitStatement("SHOW CURRENT CATALOG"); + Preconditions.checkArgument( + response.getResults().size() == 1, + "SHOW CURRENT CATALOG should return exactly one result set. This is a bug."); + + Either jobIdOrResultSet = + RestUtils.getEitherJobIdOrResultSet(response.getResults().get(0)); + Preconditions.checkArgument( + jobIdOrResultSet.isRight(), + "SHOW CURRENT CATALOG should immediately return a result. This is a bug."); + + ResultSet resultSet = jobIdOrResultSet.right(); + Preconditions.checkArgument( + resultSet.getData().size() == 1, + "SHOW CURRENT CATALOG should return exactly one row of result. This is a bug."); + + return resultSet.getData().get(0).toString(); + } + + @Override + public DatabaseMetaData getMetaData() throws SQLException { + return new FlinkDatabaseMetaData(session, this); + } + + @Override + public void setSchema(String schema) throws SQLException { + session.submitStatement("USE " + schema); + } + + @Override + public String getSchema() throws SQLException { + StatementExecuteResponseBody response; + response = session.submitStatement("SHOW CURRENT DATABASE"); + Preconditions.checkArgument( + response.getResults().size() == 1, + "SHOW CURRENT DATABASE should return exactly one result set. This is a bug."); + + Either jobIdOrResultSet = + RestUtils.getEitherJobIdOrResultSet(response.getResults().get(0)); + Preconditions.checkArgument( + jobIdOrResultSet.isRight(), + "SHOW CURRENT DATABASE should immediately return a result. This is a bug."); + + ResultSet resultSet = jobIdOrResultSet.right(); + Preconditions.checkArgument( + resultSet.getData().size() == 1, + "SHOW CURRENT DATABASE should return exactly one row of result. This is a bug."); + + return resultSet.getData().get(0).toString(); + } + + @Override + public PreparedStatement prepareStatement(String sql) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#prepareStatement is not supported"); + } + + @Override + public CallableStatement prepareCall(String sql) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#prepareCall is not supported"); + } + + @Override + public String nativeSQL(String sql) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#nativeSQL is not supported"); + } + + @Override + public void setAutoCommit(boolean autoCommit) throws SQLException { + // TODO + // we currently do not support this, + // but we can't throw a SQLException because we want to support beeline + } + + @Override + public boolean getAutoCommit() throws SQLException { + // TODO + // we currently do not support this, + // but we can't throw a SQLException because we want to support beeline + return true; + } + + @Override + public void commit() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#commit is not supported"); + } + + @Override + public void rollback() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#rollback is not supported"); + } + + @Override + public void setReadOnly(boolean readOnly) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#setReadOnly is not supported"); + } + + @Override + public boolean isReadOnly() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#isReadOnly is not supported"); + } + + @Override + public void setTransactionIsolation(int level) throws SQLException { + // TODO + // we currently do not support this, + // but we can't throw a SQLException because we want to support beeline + } + + @Override + public int getTransactionIsolation() throws SQLException { + // TODO + // we currently do not support this, + // but we can't throw a SQLException because we want to support beeline + return Connection.TRANSACTION_NONE; + } + + @Override + public SQLWarning getWarnings() throws SQLException { + // TODO + // we currently do not support this, + // but we can't throw a SQLException because we want to support beeline + return null; + } + + @Override + public void clearWarnings() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#clearWarnings is not supported"); + } + + @Override + public Statement createStatement(int resultSetType, int resultSetConcurrency) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#createStatement is not supported"); + } + + @Override + public PreparedStatement prepareStatement(String sql, int resultSetType, int resultSetConcurrency) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#prepareStatement is not supported"); + } + + @Override + public CallableStatement prepareCall(String sql, int resultSetType, int resultSetConcurrency) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#prepareCall is not supported"); + } + + @Override + public Map> getTypeMap() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#getTypeMap is not supported"); + } + + @Override + public void setTypeMap(Map> map) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#setTypeMap is not supported"); + } + + @Override + public void setHoldability(int holdability) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#setHoldability is not supported"); + } + + @Override + public int getHoldability() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#getHoldability is not supported"); + } + + @Override + public Savepoint setSavepoint() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#setSavepoint is not supported"); + } + + @Override + public Savepoint setSavepoint(String name) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#setSavepoint is not supported"); + } + + @Override + public void rollback(Savepoint savepoint) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#rollback is not supported"); + } + + @Override + public void releaseSavepoint(Savepoint savepoint) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#releaseSavepoint is not supported"); + } + + @Override + public Statement createStatement(int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#createStatement is not supported"); + } + + @Override + public PreparedStatement prepareStatement(String sql, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#prepareStatement is not supported"); + } + + @Override + public CallableStatement prepareCall(String sql, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#prepareCall is not supported"); + } + + @Override + public PreparedStatement prepareStatement(String sql, int autoGeneratedKeys) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#prepareStatement is not supported"); + } + + @Override + public PreparedStatement prepareStatement(String sql, int[] columnIndexes) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#prepareStatement is not supported"); + } + + @Override + public PreparedStatement prepareStatement(String sql, String[] columnNames) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#prepareStatement is not supported"); + } + + @Override + public Clob createClob() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#createClob is not supported"); + } + + @Override + public Blob createBlob() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#createBlob is not supported"); + } + + @Override + public NClob createNClob() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#createNClob is not supported"); + } + + @Override + public SQLXML createSQLXML() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#createSQLXML is not supported"); + } + + @Override + public boolean isValid(int timeout) throws SQLException { + // TODO support timeout + if (timeout < 0) { + throw new SQLException("Timeout must not be negative"); + } + + try { + session.sendHeartbeat(); + return true; + } catch (SQLException e) { + return false; + } + } + + @Override + public void setClientInfo(String name, String value) throws SQLClientInfoException { + throw new SQLClientInfoException(); + } + + @Override + public void setClientInfo(Properties properties) throws SQLClientInfoException { + throw new SQLClientInfoException(); + } + + @Override + public String getClientInfo(String name) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#getClientInfo is not supported"); + } + + @Override + public Properties getClientInfo() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#getClientInfo is not supported"); + } + + @Override + public Array createArrayOf(String typeName, Object[] elements) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#createArrayOf is not supported"); + } + + @Override + public Struct createStruct(String typeName, Object[] attributes) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#createStruct is not supported"); + } + + @Override + public void abort(Executor executor) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#abort is not supported"); + } + + @Override + public void setNetworkTimeout(Executor executor, int milliseconds) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#setNetworkTimeout is not supported"); + } + + @Override + public int getNetworkTimeout() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#getNetworkTimeout is not supported"); + } + + @Override + public T unwrap(Class iface) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#unwrap is not supported"); + } + + @Override + public boolean isWrapperFor(Class iface) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkConnection#isWrapperFor is not supported"); + } + + private UrlInfo parseUrl(String url) { + String neededParams = "These url parameters are needed: planner"; + + String host; + int port; + String planner = null; + + int argumentStart = url.indexOf('?'); + if (argumentStart < 0) { + throw new IllegalArgumentException(neededParams); + } else { + int colonPos = url.indexOf(':'); + if (colonPos < 0) { + throw new IllegalArgumentException("Cannot read port from string " + url); + } else { + host = url.substring(0, colonPos); + try { + port = Integer.valueOf(url.substring(colonPos + 1, argumentStart)); + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid port format"); + } + } + } + + for (String kv : url.substring(argumentStart + 1).split("&")) { + int equalPos = kv.indexOf('='); + if (equalPos < 0) { + throw new IllegalArgumentException("Invalid url parameter kv pair " + kv); + } + + String key = kv.substring(0, equalPos); + String value = kv.substring(equalPos + 1); + + if (key.equals("planner")) { + planner = value; + } else { + throw new IllegalArgumentException("Unknown url parameter key " + key); + } + } + + if (planner == null) { + throw new IllegalArgumentException(neededParams); + } + + return new UrlInfo(host, port, planner); + } + + private SessionClient createSession(String url) throws Exception { + UrlInfo urlInfo = parseUrl(url); + return new SessionClient(urlInfo.host, urlInfo.port, "Flink-JDBC", urlInfo.planner, "batch", "Flink-JDBC-Connection-IO"); + } + + /** + * Contents of Flink JDBC url. + */ + private static class UrlInfo { + final String host; + final int port; + final String planner; + + UrlInfo(String host, int port, String planner) { + this.host = host; + this.port = port; + this.planner = planner; + } + } +} diff --git a/src/main/java/com/ververica/flink/table/jdbc/FlinkDatabaseMetaData.java b/src/main/java/com/ververica/flink/table/jdbc/FlinkDatabaseMetaData.java new file mode 100644 index 0000000..efa2dc4 --- /dev/null +++ b/src/main/java/com/ververica/flink/table/jdbc/FlinkDatabaseMetaData.java @@ -0,0 +1,1489 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.flink.table.jdbc; + +import com.ververica.flink.table.gateway.rest.message.GetInfoResponseBody; +import com.ververica.flink.table.gateway.rest.message.StatementExecuteResponseBody; +import com.ververica.flink.table.gateway.rest.result.ColumnInfo; +import com.ververica.flink.table.gateway.rest.result.TableSchemaUtil; +import com.ververica.flink.table.jdbc.rest.RestUtils; +import com.ververica.flink.table.jdbc.rest.SessionClient; +import com.ververica.flink.table.jdbc.resulthandler.ResultHandlerFactory; +import com.ververica.flink.table.jdbc.type.FlinkSqlType; +import com.ververica.flink.table.jdbc.type.FlinkSqlTypes; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.types.Either; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; +import java.sql.RowIdLifetime; +import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.regex.Pattern; + +/** + * Flink JDBC database meta data. + */ +public class FlinkDatabaseMetaData implements DatabaseMetaData { + + public static final String DRIVER_VERSION = "0.1"; + public static final String JDBC_VERSION = "4.2"; + + private static final String[] SUPPORTED_TABLE_TYPES = new String[] { + "TABLE", "VIEW" + }; + + private GetInfoResponseBody infoResponse; + + private final SessionClient session; + private final FlinkConnection connection; + + public FlinkDatabaseMetaData(SessionClient session, FlinkConnection connection) { + this.session = session; + this.connection = connection; + } + + @Override + public boolean allProceduresAreCallable() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#allProceduresAreCallable is not supported"); + } + + @Override + public boolean allTablesAreSelectable() throws SQLException { + return true; + } + + @Override + public String getURL() throws SQLException { + return FlinkDriver.URL_PREFIX + session.getServerHost() + ":" + session.getServerPort() + + "?planner=" + session.getPlanner(); + } + + @Override + public String getUserName() throws SQLException { + return null; + } + + @Override + public boolean isReadOnly() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#isReadOnly is not supported"); + } + + @Override + public boolean nullsAreSortedHigh() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#nullsAreSortedHigh is not supported"); + } + + @Override + public boolean nullsAreSortedLow() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#nullsAreSortedLow is not supported"); + } + + @Override + public boolean nullsAreSortedAtStart() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#nullsAreSortedAtStart is not supported"); + } + + @Override + public boolean nullsAreSortedAtEnd() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#nullsAreSortedAtEnd is not supported"); + } + + @Override + public String getDatabaseProductName() throws SQLException { + return getInfoResponse().getProductName(); + } + + @Override + public String getDatabaseProductVersion() throws SQLException { + return getInfoResponse().getFlinkVersion(); + } + + @Override + public String getDriverName() throws SQLException { + return "Flink Driver"; + } + + @Override + public String getDriverVersion() throws SQLException { + return DRIVER_VERSION; + } + + @Override + public int getDriverMajorVersion() { + return Integer.valueOf(DRIVER_VERSION.split("\\.")[0]); + } + + @Override + public int getDriverMinorVersion() { + return Integer.valueOf(DRIVER_VERSION.split("\\.")[1]); + } + + @Override + public boolean usesLocalFiles() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#usesLocalFiles is not supported"); + } + + @Override + public boolean usesLocalFilePerTable() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#usesLocalFilePerTable is not supported"); + } + + @Override + public boolean supportsMixedCaseIdentifiers() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsMixedCaseIdentifiers is not supported"); + } + + @Override + public boolean storesUpperCaseIdentifiers() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#storesUpperCaseIdentifiers is not supported"); + } + + @Override + public boolean storesLowerCaseIdentifiers() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#storesLowerCaseIdentifiers is not supported"); + } + + @Override + public boolean storesMixedCaseIdentifiers() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#storesMixedCaseIdentifiers is not supported"); + } + + @Override + public boolean supportsMixedCaseQuotedIdentifiers() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#supportsMixedCaseQuotedIdentifiers is not supported"); + } + + @Override + public boolean storesUpperCaseQuotedIdentifiers() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#storesUpperCaseQuotedIdentifiers is not supported"); + } + + @Override + public boolean storesLowerCaseQuotedIdentifiers() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#storesLowerCaseQuotedIdentifiers is not supported"); + } + + @Override + public boolean storesMixedCaseQuotedIdentifiers() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#storesMixedCaseQuotedIdentifiers is not supported"); + } + + @Override + public String getIdentifierQuoteString() throws SQLException { + // TODO verify this + return "`"; + } + + @Override + public String getSQLKeywords() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getSQLKeywords is not supported"); + } + + @Override + public String getNumericFunctions() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getNumericFunctions is not supported"); + } + + @Override + public String getStringFunctions() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getStringFunctions is not supported"); + } + + @Override + public String getSystemFunctions() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getSystemFunctions is not supported"); + } + + @Override + public String getTimeDateFunctions() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getTimeDateFunctions is not supported"); + } + + @Override + public String getSearchStringEscape() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getSearchStringEscape is not supported"); + } + + @Override + public String getExtraNameCharacters() throws SQLException { + // TODO + // we currently do not support this, + // but we can't throw a SQLException because we want to support beeline + return ""; + } + + @Override + public boolean supportsAlterTableWithAddColumn() throws SQLException { + return true; + } + + @Override + public boolean supportsAlterTableWithDropColumn() throws SQLException { + return true; + } + + @Override + public boolean supportsColumnAliasing() throws SQLException { + return true; + } + + @Override + public boolean nullPlusNonNullIsNull() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#nullPlusNonNullIsNull is not supported"); + } + + @Override + public boolean supportsConvert() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsConvert is not supported"); + } + + @Override + public boolean supportsConvert(int fromType, int toType) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsConvert is not supported"); + } + + @Override + public boolean supportsTableCorrelationNames() throws SQLException { + return true; + } + + @Override + public boolean supportsDifferentTableCorrelationNames() throws SQLException { + return false; + } + + @Override + public boolean supportsExpressionsInOrderBy() throws SQLException { + return true; + } + + @Override + public boolean supportsOrderByUnrelated() throws SQLException { + return true; + } + + @Override + public boolean supportsGroupBy() throws SQLException { + return true; + } + + @Override + public boolean supportsGroupByUnrelated() throws SQLException { + return true; + } + + @Override + public boolean supportsGroupByBeyondSelect() throws SQLException { + return true; + } + + @Override + public boolean supportsLikeEscapeClause() throws SQLException { + return true; + } + + @Override + public boolean supportsMultipleResultSets() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsMultipleResultSets is not supported"); + } + + @Override + public boolean supportsMultipleTransactions() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsMultipleTransactions is not supported"); + } + + @Override + public boolean supportsNonNullableColumns() throws SQLException { + return true; + } + + @Override + public boolean supportsMinimumSQLGrammar() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsMinimumSQLGrammar is not supported"); + } + + @Override + public boolean supportsCoreSQLGrammar() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsCoreSQLGrammar is not supported"); + } + + @Override + public boolean supportsExtendedSQLGrammar() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsExtendedSQLGrammar is not supported"); + } + + @Override + public boolean supportsANSI92EntryLevelSQL() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsANSI92EntryLevelSQL is not supported"); + } + + @Override + public boolean supportsANSI92IntermediateSQL() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsANSI92IntermediateSQL is not supported"); + } + + @Override + public boolean supportsANSI92FullSQL() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsANSI92FullSQL is not supported"); + } + + @Override + public boolean supportsIntegrityEnhancementFacility() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#supportsIntegrityEnhancementFacility is not supported"); + } + + @Override + public boolean supportsOuterJoins() throws SQLException { + return true; + } + + @Override + public boolean supportsFullOuterJoins() throws SQLException { + return true; + } + + @Override + public boolean supportsLimitedOuterJoins() throws SQLException { + return true; + } + + @Override + public String getSchemaTerm() throws SQLException { + return "database"; + } + + @Override + public String getProcedureTerm() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getProcedureTerm is not supported"); + } + + @Override + public String getCatalogTerm() throws SQLException { + return "catalog"; + } + + @Override + public boolean isCatalogAtStart() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#isCatalogAtStart is not supported"); + } + + @Override + public String getCatalogSeparator() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getCatalogSeparator is not supported"); + } + + @Override + public boolean supportsSchemasInDataManipulation() throws SQLException { + return true; + } + + @Override + public boolean supportsSchemasInProcedureCalls() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsSchemasInProcedureCalls is not supported"); + } + + @Override + public boolean supportsSchemasInTableDefinitions() throws SQLException { + return true; + } + + @Override + public boolean supportsSchemasInIndexDefinitions() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#supportsSchemasInIndexDefinitions is not supported"); + } + + @Override + public boolean supportsSchemasInPrivilegeDefinitions() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#supportsSchemasInPrivilegeDefinitions is not supported"); + } + + @Override + public boolean supportsCatalogsInDataManipulation() throws SQLException { + return true; + } + + @Override + public boolean supportsCatalogsInProcedureCalls() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#supportsCatalogsInProcedureCalls is not supported"); + } + + @Override + public boolean supportsCatalogsInTableDefinitions() throws SQLException { + return true; + } + + @Override + public boolean supportsCatalogsInIndexDefinitions() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#supportsCatalogsInIndexDefinitions is not supported"); + } + + @Override + public boolean supportsCatalogsInPrivilegeDefinitions() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#supportsCatalogsInPrivilegeDefinitions is not supported"); + } + + @Override + public boolean supportsPositionedDelete() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsPositionedDelete is not supported"); + } + + @Override + public boolean supportsPositionedUpdate() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsPositionedUpdate is not supported"); + } + + @Override + public boolean supportsSelectForUpdate() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsSelectForUpdate is not supported"); + } + + @Override + public boolean supportsStoredProcedures() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsStoredProcedures is not supported"); + } + + @Override + public boolean supportsSubqueriesInComparisons() throws SQLException { + return true; + } + + @Override + public boolean supportsSubqueriesInExists() throws SQLException { + return true; + } + + @Override + public boolean supportsSubqueriesInIns() throws SQLException { + return true; + } + + @Override + public boolean supportsSubqueriesInQuantifieds() throws SQLException { + return true; + } + + @Override + public boolean supportsCorrelatedSubqueries() throws SQLException { + return true; + } + + @Override + public boolean supportsUnion() throws SQLException { + return true; + } + + @Override + public boolean supportsUnionAll() throws SQLException { + return true; + } + + @Override + public boolean supportsOpenCursorsAcrossCommit() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsOpenCursorsAcrossCommit is not supported"); + } + + @Override + public boolean supportsOpenCursorsAcrossRollback() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#supportsOpenCursorsAcrossRollback is not supported"); + } + + @Override + public boolean supportsOpenStatementsAcrossCommit() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#supportsOpenStatementsAcrossCommit is not supported"); + } + + @Override + public boolean supportsOpenStatementsAcrossRollback() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#supportsOpenStatementsAcrossRollback is not supported"); + } + + @Override + public int getMaxBinaryLiteralLength() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxBinaryLiteralLength is not supported"); + } + + @Override + public int getMaxCharLiteralLength() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxCharLiteralLength is not supported"); + } + + @Override + public int getMaxColumnNameLength() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxColumnNameLength is not supported"); + } + + @Override + public int getMaxColumnsInGroupBy() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxColumnsInGroupBy is not supported"); + } + + @Override + public int getMaxColumnsInIndex() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxColumnsInIndex is not supported"); + } + + @Override + public int getMaxColumnsInOrderBy() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxColumnsInOrderBy is not supported"); + } + + @Override + public int getMaxColumnsInSelect() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxColumnsInSelect is not supported"); + } + + @Override + public int getMaxColumnsInTable() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxColumnsInTable is not supported"); + } + + @Override + public int getMaxConnections() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxConnections is not supported"); + } + + @Override + public int getMaxCursorNameLength() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxCursorNameLength is not supported"); + } + + @Override + public int getMaxIndexLength() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxIndexLength is not supported"); + } + + @Override + public int getMaxSchemaNameLength() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxSchemaNameLength is not supported"); + } + + @Override + public int getMaxProcedureNameLength() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxProcedureNameLength is not supported"); + } + + @Override + public int getMaxCatalogNameLength() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxCatalogNameLength is not supported"); + } + + @Override + public int getMaxRowSize() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxRowSize is not supported"); + } + + @Override + public boolean doesMaxRowSizeIncludeBlobs() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#doesMaxRowSizeIncludeBlobs is not supported"); + } + + @Override + public int getMaxStatementLength() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxStatementLength is not supported"); + } + + @Override + public int getMaxStatements() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxStatements is not supported"); + } + + @Override + public int getMaxTableNameLength() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxTableNameLength is not supported"); + } + + @Override + public int getMaxTablesInSelect() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxTablesInSelect is not supported"); + } + + @Override + public int getMaxUserNameLength() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getMaxUserNameLength is not supported"); + } + + @Override + public int getDefaultTransactionIsolation() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getDefaultTransactionIsolation is not supported"); + } + + @Override + public boolean supportsTransactions() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsTransactions is not supported"); + } + + @Override + public boolean supportsTransactionIsolationLevel(int level) throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#supportsTransactionIsolationLevel is not supported"); + } + + @Override + public boolean supportsDataDefinitionAndDataManipulationTransactions() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#supportsDataDefinitionAndDataManipulationTransactions is not supported"); + } + + @Override + public boolean supportsDataManipulationTransactionsOnly() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#supportsDataManipulationTransactionsOnly is not supported"); + } + + @Override + public boolean dataDefinitionCausesTransactionCommit() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#dataDefinitionCausesTransactionCommit is not supported"); + } + + @Override + public boolean dataDefinitionIgnoredInTransactions() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#dataDefinitionIgnoredInTransactions is not supported"); + } + + @Override + public ResultSet getProcedures(String catalog, String schemaPattern, String procedureNamePattern) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getProcedures is not supported"); + } + + @Override + public ResultSet getProcedureColumns(String catalog, String schemaPattern, String procedureNamePattern, String columnNamePattern) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getProcedureColumns is not supported"); + } + + @Override + public ResultSet getTables(String catalog, String schemaPattern, String tableNamePattern, String[] types) throws SQLException { + if (tableNamePattern == null) { + tableNamePattern = "%"; + } + if (types == null) { + types = SUPPORTED_TABLE_TYPES; + } + + if ("".equals(catalog) || "".equals(schemaPattern)) { + // every Flink database belongs to a catalog and a database + return FlinkResultSet.of(new com.ververica.flink.table.gateway.rest.result.ResultSet( + new GetTableResultColumnInfos().getColumnInfos(), Collections.emptyList())); + } + + String oldCatalog = connection.getCatalog(); + String oldDatabase = connection.getSchema(); + + List candidates = new ArrayList<>(); + ResultSet databaseResult = getSchemas(catalog, schemaPattern); + while (databaseResult.next()) { + appendTablesAndViewsInDatabase( + databaseResult.getString(2), databaseResult.getString(1), candidates); + } + candidates.sort((o1, o2) -> { + if (o1.type.equals(o2.type)) { + if (o1.catalog.equals(o2.catalog)) { + if (o1.database.equals(o2.database)) { + return o1.table.compareTo(o2.table); + } + return o1.database.compareTo(o2.database); + } + return o1.catalog.compareTo(o2.catalog); + } + return o1.type.compareTo(o2.type); + }); + + connection.setCatalog(oldCatalog); + connection.setSchema(oldDatabase); + + // match with table name pattern + List matches = new ArrayList<>(); + GetTableResultColumnInfos columnInfos = new GetTableResultColumnInfos(); + Pattern javaPattern = FlinkJdbcUtils.sqlPatternToJavaPattern(tableNamePattern); + List typesList = Arrays.asList(types); + for (TableResultData candidate : candidates) { + if (typesList.contains(candidate.type) && javaPattern.matcher(candidate.table).matches()) { + matches.add(columnInfos.process(candidate)); + } + } + return FlinkResultSet.of(new com.ververica.flink.table.gateway.rest.result.ResultSet( + columnInfos.getColumnInfos(), matches)); + } + + @Override + public ResultSet getSchemas() throws SQLException { + return getSchemas(null, null); + } + + @Override + public ResultSet getCatalogs() throws SQLException { + final String catalogColumn = "TABLE_CAT"; + + ResultSet result = getImmediateSingleSqlResultSet("SHOW CATALOGS"); + + // we have to recreate a result set to + // change the column name to the one specified by the java doc + // and order the result + List names = new ArrayList<>(); + int maxCatalogNameLength = 1; + while (result.next()) { + names.add(result.getString(1)); + } + names.sort(String::compareTo); + + List rows = new ArrayList<>(); + for (String name : names) { + rows.add(Row.of(name)); + maxCatalogNameLength = Math.max(maxCatalogNameLength, name.length()); + } + + return FlinkResultSet.of(new com.ververica.flink.table.gateway.rest.result.ResultSet( + Collections.singletonList( + ColumnInfo.create(catalogColumn, new VarCharType(true, maxCatalogNameLength))), + rows)); + } + + @Override + public ResultSet getTableTypes() throws SQLException { + final String tableTypeColumn = "TABLE_TYPE"; + + List rows = new ArrayList<>(); + int maxTypeNameLength = 1; + for (String type : SUPPORTED_TABLE_TYPES) { + rows.add(Row.of(type)); + maxTypeNameLength = Math.max(maxTypeNameLength, type.length()); + } + + return FlinkResultSet.of(new com.ververica.flink.table.gateway.rest.result.ResultSet( + Collections.singletonList(ColumnInfo.create(tableTypeColumn, new VarCharType(false, maxTypeNameLength))), + rows)); + } + + @Override + public ResultSet getColumns(String catalog, String schemaPattern, String tableNamePattern, String columnNamePattern) throws SQLException { + if (columnNamePattern == null) { + columnNamePattern = "%"; + } + + String oldCatalog = connection.getCatalog(); + String oldDatabase = connection.getSchema(); + + List candidates = new ArrayList<>(); + ResultSet tableResult = getTables(catalog, schemaPattern, tableNamePattern, SUPPORTED_TABLE_TYPES); + while (tableResult.next()) { + appendColumnsInTable( + tableResult.getString(1), + tableResult.getString(2), + tableResult.getString(3), + candidates); + } + candidates.sort((o1, o2) -> { + if (o1.catalog.equals(o2.catalog)) { + if (o1.database.equals(o2.database)) { + if (o1.table.equals(o2.table)) { + return o1.pos - o2.pos; + } + return o1.table.compareTo(o2.table); + } + return o1.database.compareTo(o2.database); + } + return o1.catalog.compareTo(o2.catalog); + }); + + connection.setCatalog(oldCatalog); + connection.setSchema(oldDatabase); + + // match with column name pattern + List matches = new ArrayList<>(); + GetColumnResultColumnInfos columnInfos = new GetColumnResultColumnInfos(); + Pattern javaPattern = FlinkJdbcUtils.sqlPatternToJavaPattern(columnNamePattern); + for (ColumnResultData candidate : candidates) { + if (javaPattern.matcher(candidate.column).matches()) { + matches.add(columnInfos.process(candidate)); + } + } + return FlinkResultSet.of(new com.ververica.flink.table.gateway.rest.result.ResultSet( + columnInfos.getColumnInfos(), matches)); + } + + @Override + public ResultSet getColumnPrivileges(String catalog, String schema, String table, String columnNamePattern) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getColumnPrivileges is not supported"); + } + + @Override + public ResultSet getTablePrivileges(String catalog, String schemaPattern, String tableNamePattern) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getTablePrivileges is not supported"); + } + + @Override + public ResultSet getBestRowIdentifier(String catalog, String schema, String table, int scope, boolean nullable) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getBestRowIdentifier is not supported"); + } + + @Override + public ResultSet getVersionColumns(String catalog, String schema, String table) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getVersionColumns is not supported"); + } + + @Override + public ResultSet getPrimaryKeys(String catalog, String schema, String table) throws SQLException { + String oldCatalog = connection.getCatalog(); + String oldDatabase = connection.getSchema(); + + ResultSet ret; + List matches = new ArrayList<>(); + GetPrimaryKeyResultColumnInfos columnInfos = new GetPrimaryKeyResultColumnInfos(); + + TableSchema tableSchema = getTableSchema(catalog, schema, table); + if (tableSchema.getPrimaryKey().isPresent()) { + List keyNames = tableSchema.getPrimaryKey().get().getColumns(); + for (TableColumn column : tableSchema.getTableColumns()) { + int pkIdx = keyNames.indexOf(column.getName()); + if (pkIdx >= 0) { + matches.add(columnInfos.process(catalog, schema, table, column.getName(), pkIdx)); + } + } + ret = FlinkResultSet.of(new com.ververica.flink.table.gateway.rest.result.ResultSet( + columnInfos.getColumnInfos(), matches)); + } else { + // no primary keys + ret = FlinkResultSet.of(new com.ververica.flink.table.gateway.rest.result.ResultSet( + columnInfos.getColumnInfos(), Collections.emptyList())); + } + + connection.setCatalog(oldCatalog); + connection.setSchema(oldDatabase); + return ret; + } + + @Override + public ResultSet getImportedKeys(String catalog, String schema, String table) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getImportedKeys is not supported"); + } + + @Override + public ResultSet getExportedKeys(String catalog, String schema, String table) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getExportedKeys is not supported"); + } + + @Override + public ResultSet getCrossReference(String parentCatalog, String parentSchema, String parentTable, String foreignCatalog, String foreignSchema, String foreignTable) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getCrossReference is not supported"); + } + + @Override + public ResultSet getTypeInfo() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getTypeInfo is not supported"); + } + + @Override + public ResultSet getIndexInfo(String catalog, String schema, String table, boolean unique, boolean approximate) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getIndexInfo is not supported"); + } + + @Override + public boolean supportsResultSetType(int type) throws SQLException { + return type == ResultSet.TYPE_FORWARD_ONLY; + } + + @Override + public boolean supportsResultSetConcurrency(int type, int concurrency) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsResultSetConcurrency is not supported"); + } + + @Override + public boolean ownUpdatesAreVisible(int type) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#ownUpdatesAreVisible is not supported"); + } + + @Override + public boolean ownDeletesAreVisible(int type) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#ownDeletesAreVisible is not supported"); + } + + @Override + public boolean ownInsertsAreVisible(int type) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#ownInsertsAreVisible is not supported"); + } + + @Override + public boolean othersUpdatesAreVisible(int type) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#othersUpdatesAreVisible is not supported"); + } + + @Override + public boolean othersDeletesAreVisible(int type) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#othersDeletesAreVisible is not supported"); + } + + @Override + public boolean othersInsertsAreVisible(int type) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#othersInsertsAreVisible is not supported"); + } + + @Override + public boolean updatesAreDetected(int type) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#updatesAreDetected is not supported"); + } + + @Override + public boolean deletesAreDetected(int type) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#deletesAreDetected is not supported"); + } + + @Override + public boolean insertsAreDetected(int type) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#insertsAreDetected is not supported"); + } + + @Override + public boolean supportsBatchUpdates() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsBatchUpdates is not supported"); + } + + @Override + public ResultSet getUDTs(String catalog, String schemaPattern, String typeNamePattern, int[] types) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getUDTs is not supported"); + } + + @Override + public Connection getConnection() throws SQLException { + return connection; + } + + @Override + public boolean supportsSavepoints() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsSavepoints is not supported"); + } + + @Override + public boolean supportsNamedParameters() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsNamedParameters is not supported"); + } + + @Override + public boolean supportsMultipleOpenResults() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsMultipleOpenResults is not supported"); + } + + @Override + public boolean supportsGetGeneratedKeys() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsGetGeneratedKeys is not supported"); + } + + @Override + public ResultSet getSuperTypes(String catalog, String schemaPattern, String typeNamePattern) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getSuperTypes is not supported"); + } + + @Override + public ResultSet getSuperTables(String catalog, String schemaPattern, String tableNamePattern) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getSuperTypes is not supported"); + } + + @Override + public ResultSet getAttributes(String catalog, String schemaPattern, String typeNamePattern, String attributeNamePattern) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getAttributes is not supported"); + } + + @Override + public boolean supportsResultSetHoldability(int holdability) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsResultSetHoldability is not supported"); + } + + @Override + public int getResultSetHoldability() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getResultSetHoldability is not supported"); + } + + @Override + public int getDatabaseMajorVersion() throws SQLException { + return Integer.valueOf(getInfoResponse().getFlinkVersion().split("\\.")[0]); + } + + @Override + public int getDatabaseMinorVersion() throws SQLException { + return Integer.valueOf(getInfoResponse().getFlinkVersion().split("\\.")[1]); + } + + @Override + public int getJDBCMajorVersion() throws SQLException { + return Integer.valueOf(JDBC_VERSION.split("\\.")[0]); + } + + @Override + public int getJDBCMinorVersion() throws SQLException { + return Integer.valueOf(JDBC_VERSION.split("\\.")[1]); + } + + @Override + public int getSQLStateType() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getSQLStateType is not supported"); + } + + @Override + public boolean locatorsUpdateCopy() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#locatorsUpdateCopy is not supported"); + } + + @Override + public boolean supportsStatementPooling() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#supportsStatementPooling is not supported"); + } + + @Override + public RowIdLifetime getRowIdLifetime() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getRowIdLifetime is not supported"); + } + + @Override + public ResultSet getSchemas(String catalog, String schemaPattern) throws SQLException { + if (schemaPattern == null) { + schemaPattern = "%"; + } + + if ("".equals(catalog)) { + // every Flink database belongs to a catalog + return FlinkResultSet.of(new com.ververica.flink.table.gateway.rest.result.ResultSet( + new GetSchemaColumnInfos().getColumnInfos(), Collections.emptyList())); + } + + List candidates = new ArrayList<>(); + if (catalog == null) { + String oldCatalog = connection.getCatalog(); + ResultSet catalogResult = getCatalogs(); + while (catalogResult.next()) { + appendDatabasesInCatalog(catalogResult.getString(1), candidates); + } + connection.setCatalog(oldCatalog); + } else { + String oldCatalog = connection.getCatalog(); + appendDatabasesInCatalog(catalog, candidates); + connection.setCatalog(oldCatalog); + } + candidates.sort((o1, o2) -> { + if (o1.catalog.equals(o2.catalog)) { + return o1.database.compareTo(o2.database); + } + return o1.catalog.compareTo(o2.catalog); + }); + + // match with schema pattern + List matches = new ArrayList<>(); + GetSchemaColumnInfos columnInfos = new GetSchemaColumnInfos(); + Pattern javaPattern = FlinkJdbcUtils.sqlPatternToJavaPattern(schemaPattern); + for (SchemaResultData candidate : candidates) { + if (javaPattern.matcher(candidate.database).matches()) { + matches.add(columnInfos.process(candidate)); + } + } + return FlinkResultSet.of(new com.ververica.flink.table.gateway.rest.result.ResultSet( + columnInfos.getColumnInfos(), matches)); + } + + @Override + public boolean supportsStoredFunctionsUsingCallSyntax() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#supportsStoredFunctionsUsingCallSyntax is not supported"); + } + + @Override + public boolean autoCommitFailureClosesAllResultSets() throws SQLException { + throw new SQLFeatureNotSupportedException( + "FlinkDatabaseMetaData#autoCommitFailureClosesAllResultSets is not supported"); + } + + @Override + public ResultSet getClientInfoProperties() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getClientInfoProperties is not supported"); + } + + @Override + public ResultSet getFunctions(String catalog, String schemaPattern, String functionNamePattern) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getFunctions is not supported"); + } + + @Override + public ResultSet getFunctionColumns(String catalog, String schemaPattern, String functionNamePattern, String columnNamePattern) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getFunctionColumns is not supported"); + } + + @Override + public ResultSet getPseudoColumns(String catalog, String schemaPattern, String tableNamePattern, String columnNamePattern) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#getPseudoColumns is not supported"); + } + + @Override + public boolean generatedKeyAlwaysReturned() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#generatedKeyAlwaysReturned is not supported"); + } + + @Override + public T unwrap(Class iface) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#unwrap is not supported"); + } + + @Override + public boolean isWrapperFor(Class iface) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkDatabaseMetaData#isWrapperFor is not supported"); + } + + private StatementExecuteResponseBody getImmediateSingleResponse(String stmt) + throws SQLException { + StatementExecuteResponseBody response; + response = session.submitStatement(stmt); + Preconditions.checkState( + response.getResults().size() == 1, + stmt + " should return exactly 1 result set. This is a bug."); + Either jobIdOrResultSet = + RestUtils.getEitherJobIdOrResultSet(response.getResults().get(0)); + Preconditions.checkState( + jobIdOrResultSet.isRight(), + stmt + " should directly return the result. This is a bug."); + return response; + } + + private ResultSet getImmediateSingleSqlResultSet(String stmt) throws SQLException { + StatementExecuteResponseBody response = getImmediateSingleResponse(stmt); + return new FlinkResultSet( + session, + RestUtils.getEitherJobIdOrResultSet(response.getResults().get(0)), + ResultHandlerFactory.getResultHandlerByStatementType(response.getStatementTypes().get(0)), + 0, + null); + } + + private void appendDatabasesInCatalog( + String catalog, + List candidates) throws SQLException { + connection.setCatalog(catalog); + ResultSet result = getImmediateSingleSqlResultSet("SHOW DATABASES"); + while (result.next()) { + candidates.add(new SchemaResultData(catalog, result.getString(1))); + } + } + + private void appendTablesAndViewsInDatabase( + String catalog, + String database, + List candidates) throws SQLException { + connection.setCatalog(catalog); + connection.setSchema(database); + ResultSet result = getImmediateSingleSqlResultSet("SHOW TABLES"); + while (result.next()) { + candidates.add(new TableResultData(catalog, database, result.getString(1), result.getString(2))); + } + } + + private TableSchema getTableSchema(String catalog, String database, String table) throws SQLException { + connection.setCatalog(catalog); + connection.setSchema(database); + StatementExecuteResponseBody response = getImmediateSingleResponse("DESCRIBE " + table); + // we use raw results here to get table schema + ResultSet result = new FlinkResultSet( + session, + RestUtils.getEitherJobIdOrResultSet(response.getResults().get(0)), + ResultHandlerFactory.getDefaultResultHandler(), + 0, + null); + + boolean hasNext = result.next(); + Preconditions.checkState( + hasNext, + "DESCRIBE statement must return exactly " + + "one serialized table schema json string. This is a bug."); + try { + return TableSchemaUtil.readTableSchemaFromJson(result.getString(1)); + } catch (JsonProcessingException e) { + throw new SQLException("Failed to parse json to table schema", e); + } + } + + private void appendColumnsInTable( + String catalog, + String database, + String table, + List candidates) throws SQLException { + TableSchema tableSchema = getTableSchema(catalog, database, table); + int idx = 0; + for (TableColumn column : tableSchema.getTableColumns()) { + candidates.add(new ColumnResultData( + catalog, database, table, column.getName(), ++idx, column.getType().getLogicalType())); + } + } + + private GetInfoResponseBody getInfoResponse() throws SQLException { + if (infoResponse == null) { + infoResponse = session.getInfo(); + } + return infoResponse; + } + + /** + * Candidate result for schema related interface. + */ + private static class SchemaResultData { + final String catalog; + final String database; + + SchemaResultData(String catalog, String database) { + this.catalog = catalog; + this.database = database; + } + } + + /** + * Helper class to generate {@link FlinkResultSet} + * for FlinkDatabaseMetaData#getSchemas interface. + */ + private static class GetSchemaColumnInfos { + private int maxCatalogNameLength = 1; + private int maxDatabaseNameLength = 1; + + Row process(SchemaResultData data) { + maxCatalogNameLength = Math.max(maxCatalogNameLength, data.catalog.length()); + maxDatabaseNameLength = Math.max(maxDatabaseNameLength, data.database.length()); + return Row.of(data.database, data.catalog); + } + + List getColumnInfos() { + // according to the java doc of DatabaseMetaData#getSchemas + return Arrays.asList( + ColumnInfo.create("TABLE_SCHEM", new VarCharType(false, maxDatabaseNameLength)), + ColumnInfo.create("TABLE_CATALOG", new VarCharType(true, maxCatalogNameLength))); + } + } + + /** + * Candidate result for table related interface. + */ + private static class TableResultData { + final String catalog; + final String database; + final String table; + final String type; + + TableResultData(String catalog, String database, String table, String type) { + this.catalog = catalog; + this.database = database; + this.table = table; + this.type = type; + } + } + + /** + * Helper class to generate {@link FlinkResultSet} + * for FlinkDatabaseMetaData#getTables interface. + */ + private static class GetTableResultColumnInfos { + private int maxCatalogNameLength = 1; + private int maxDatabaseNameLength = 1; + private int maxTableNameLength = 1; + + Row process(TableResultData data) { + maxCatalogNameLength = Math.max(maxCatalogNameLength, data.catalog.length()); + maxDatabaseNameLength = Math.max(maxDatabaseNameLength, data.database.length()); + maxTableNameLength = Math.max(maxTableNameLength, data.table.length()); + return Row.of(data.catalog, data.database, data.table, data.type, null, null, null, null, null, null); + } + + List getColumnInfos() { + // according to the java doc of DatabaseMetaData#getTables + return Arrays.asList( + ColumnInfo.create("TABLE_CAT", new VarCharType(true, maxCatalogNameLength)), + ColumnInfo.create("TABLE_SCHEM", new VarCharType(true, maxDatabaseNameLength)), + ColumnInfo.create("TABLE_NAME", new VarCharType(false, maxTableNameLength)), + // currently can only be TABLE or VIEW + ColumnInfo.create("TABLE_TYPE", new VarCharType(false, 5)), + // currently these columns are null + ColumnInfo.create("REMARKS", new VarCharType(true, 1)), + ColumnInfo.create("TYPE_CAT", new VarCharType(true, 1)), + ColumnInfo.create("TYPE_SCHEM", new VarCharType(true, 1)), + ColumnInfo.create("TYPE_NAME", new VarCharType(true, 1)), + ColumnInfo.create("SELF_REFERENCING_COL_NAME", new VarCharType(true, 1)), + ColumnInfo.create("REF_GENERATION", new VarCharType(true, 1))); + } + } + + /** + * Candidate result for column related interface. + */ + private static class ColumnResultData { + final String catalog; + final String database; + final String table; + final String column; + final int pos; + final LogicalType logicalType; + final FlinkSqlType sqlType; + + ColumnResultData( + String catalog, + String database, + String table, + String column, + int pos, + LogicalType logicalType) { + this.catalog = catalog; + this.database = database; + this.table = table; + this.column = column; + this.pos = pos; + this.logicalType = logicalType; + this.sqlType = FlinkSqlTypes.getType(logicalType); + } + } + + /** + * Helper class to generate {@link FlinkResultSet} + * for FlinkDatabaseMetaData#getColumns interface. + */ + private static class GetColumnResultColumnInfos { + private int maxCatalogNameLength = 1; + private int maxDatabaseNameLength = 1; + private int maxTableNameLength = 1; + private int maxColumnNameLength = 1; + private int maxTypeNameLength = 1; + + Row process(ColumnResultData data) { + maxCatalogNameLength = Math.max(maxCatalogNameLength, data.catalog.length()); + maxDatabaseNameLength = Math.max(maxDatabaseNameLength, data.database.length()); + maxTableNameLength = Math.max(maxTableNameLength, data.table.length()); + maxColumnNameLength = Math.max(maxColumnNameLength, data.column.length()); + maxTypeNameLength = Math.max(maxTypeNameLength, data.logicalType.toString().length()); + + boolean isNumeric = FlinkSqlTypes.isNumeric(data.sqlType); + boolean isChar = FlinkSqlTypes.isChar(data.sqlType); + + return Row.of( + data.catalog, // TABLE_CAT + data.database, // TABLE_SCHEM + data.table, // TABLE_NAME + data.column, // COLUMN_NAME + data.sqlType.getSqlType(), // DATA_TYPE + data.logicalType.toString(), // TYPE_NAME + data.sqlType.getPrecision(), // COLUMN_SIZE + null, // BUFFER_LENGTH unused + isNumeric ? data.sqlType.getSqlType() : null, // DECIMAL_DIGITS + isNumeric ? 10 : null, // NUM_PREC_RADIX + data.logicalType.isNullable() ? columnNullable : columnNoNulls, // NULLABLE + null, // REMARKS + null, // COLUMN_DEF + null, // SQL_DATA_TYPE unused + null, // SQL_DATETIME_SUB unused + isChar ? data.sqlType.getPrecision() : null, // CHAR_OCTET_LENGTH + data.pos, // ORDINAL_POSITION + data.logicalType.isNullable() ? "YES" : "NO", // IS_NULLABLE + null, // SCOPE_CATALOG + null, // SCOPE_SCHEMA + null, // SCOPE_TABLE + null, // SOURCE_DATA_TYPE + "", // IS_AUTOINCREMENT + "" // IS_GENERATEDCOLUMN + ); + } + + public List getColumnInfos() { + // according to the java doc of DatabaseMetaData#getColumns + return Arrays.asList( + ColumnInfo.create("TABLE_CAT", new VarCharType(true, maxCatalogNameLength)), + ColumnInfo.create("TABLE_SCHEM", new VarCharType(true, maxDatabaseNameLength)), + ColumnInfo.create("TABLE_NAME", new VarCharType(false, maxTableNameLength)), + ColumnInfo.create("COLUMN_NAME", new VarCharType(false, maxColumnNameLength)), + ColumnInfo.create("DATA_TYPE", new IntType(false)), + ColumnInfo.create("TYPE_NAME", new VarCharType(false, maxTypeNameLength)), + ColumnInfo.create("COLUMN_SIZE", new IntType(false)), + ColumnInfo.create("BUFFER_LENGTH", new IntType(true)), + ColumnInfo.create("DECIMAL_DIGITS", new IntType(true)), + ColumnInfo.create("NUM_PREC_RADIX", new IntType(true)), + ColumnInfo.create("NULLABLE", new IntType(false)), + ColumnInfo.create("REMARKS", new VarCharType(true, 1)), + ColumnInfo.create("COLUMN_DEF", new VarCharType(true, 1)), + ColumnInfo.create("SQL_DATA_TYPE", new IntType(true)), + ColumnInfo.create("SQL_DATETIME_SUB", new IntType(true)), + ColumnInfo.create("CHAR_OCTET_LENGTH", new IntType(true)), + ColumnInfo.create("ORDINAL_POSITION", new IntType(false)), + ColumnInfo.create("IS_NULLABLE", new VarCharType(false, 3)), // YES or NO + ColumnInfo.create("SCOPE_CATALOG", new VarCharType(true, 1)), + ColumnInfo.create("SCOPE_SCHEMA", new VarCharType(true, 1)), + ColumnInfo.create("SCOPE_TABLE", new VarCharType(true, 1)), + ColumnInfo.create("SOURCE_DATA_TYPE", new SmallIntType(true)), + ColumnInfo.create("IS_AUTOINCREMENT", new VarCharType(false, 1)), // empty string + ColumnInfo.create("IS_GENERATEDCOLUMN", new VarCharType(false, 1)) // empty string + ); + } + } + + /** + * Helper class to generate {@link FlinkResultSet} + * for FlinkDatabaseMetaData#getPrimaryKeys interface. + */ + private static class GetPrimaryKeyResultColumnInfos { + private int maxCatalogNameLength = 1; + private int maxDatabaseNameLength = 1; + private int maxTableNameLength = 1; + private int maxColumnNameLength = 1; + + Row process(String catalog, String database, String table, String column, int pkSeq) { + maxCatalogNameLength = Math.max(maxCatalogNameLength, catalog.length()); + maxDatabaseNameLength = Math.max(maxDatabaseNameLength, database.length()); + maxTableNameLength = Math.max(maxTableNameLength, table.length()); + maxColumnNameLength = Math.max(maxColumnNameLength, column.length()); + return Row.of(catalog, database, table, column, pkSeq, null); + } + + public List getColumnInfos() { + // according to the java doc of DatabaseMetaData#getPrimaryKeys + return Arrays.asList( + ColumnInfo.create("TABLE_CAT", new VarCharType(true, maxCatalogNameLength)), + ColumnInfo.create("TABLE_SCHEM", new VarCharType(true, maxDatabaseNameLength)), + ColumnInfo.create("TABLE_NAME", new VarCharType(false, maxTableNameLength)), + ColumnInfo.create("COLUMN_NAME", new VarCharType(false, maxColumnNameLength)), + ColumnInfo.create("KEY_SEQ", new SmallIntType(false)), + ColumnInfo.create("PK_NAME", new VarCharType(true, 1))); + } + } +} diff --git a/src/main/java/com/ververica/flink/table/jdbc/FlinkDriver.java b/src/main/java/com/ververica/flink/table/jdbc/FlinkDriver.java new file mode 100644 index 0000000..12b491c --- /dev/null +++ b/src/main/java/com/ververica/flink/table/jdbc/FlinkDriver.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.flink.table.jdbc; + +import java.sql.Connection; +import java.sql.Driver; +import java.sql.DriverPropertyInfo; +import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; +import java.util.Properties; +import java.util.logging.Logger; + +/** + * Flink JDBC driver. + */ +public class FlinkDriver implements Driver { + + public static final String URL_PREFIX = "jdbc:flink://"; + + static { + try { + java.sql.DriverManager.registerDriver(new FlinkDriver()); + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + @Override + public Connection connect(String url, Properties info) throws SQLException { + try { + return acceptsURL(url) ? new FlinkConnection(url) : null; + } catch (Exception e) { + throw new SQLException(e); + } + } + + @Override + public boolean acceptsURL(String url) throws SQLException { + return url.startsWith(URL_PREFIX); + } + + @Override + public DriverPropertyInfo[] getPropertyInfo(String url, Properties info) throws SQLException { + return new DriverPropertyInfo[0]; + } + + @Override + public int getMajorVersion() { + return Integer.valueOf(FlinkDatabaseMetaData.DRIVER_VERSION.split("\\.")[0]); + } + + @Override + public int getMinorVersion() { + return Integer.valueOf(FlinkDatabaseMetaData.DRIVER_VERSION.split("\\.")[1]); + } + + @Override + public boolean jdbcCompliant() { + return false; + } + + @Override + public Logger getParentLogger() throws SQLFeatureNotSupportedException { + throw new SQLFeatureNotSupportedException("FlinkDriver#getParentLogger is not supported"); + } +} diff --git a/src/main/java/com/ververica/flink/table/jdbc/FlinkJdbcUtils.java b/src/main/java/com/ververica/flink/table/jdbc/FlinkJdbcUtils.java new file mode 100644 index 0000000..7ad5115 --- /dev/null +++ b/src/main/java/com/ververica/flink/table/jdbc/FlinkJdbcUtils.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.flink.table.jdbc; + +import java.util.regex.Pattern; + +/** + * Util class for Flink JDBC. + */ +public class FlinkJdbcUtils { + + public static Pattern sqlPatternToJavaPattern(String sqlPattern) { + return Pattern.compile(sqlPattern + .replace("%", ".*") + .replace("_", ".?")); + } +} diff --git a/src/main/java/com/ververica/flink/table/jdbc/FlinkResultSet.java b/src/main/java/com/ververica/flink/table/jdbc/FlinkResultSet.java new file mode 100644 index 0000000..b7e4d9e --- /dev/null +++ b/src/main/java/com/ververica/flink/table/jdbc/FlinkResultSet.java @@ -0,0 +1,1471 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.flink.table.jdbc; + +import com.ververica.flink.table.gateway.rest.message.ResultFetchResponseBody; +import com.ververica.flink.table.gateway.rest.result.ColumnInfo; +import com.ververica.flink.table.jdbc.rest.SessionClient; +import com.ververica.flink.table.jdbc.resulthandler.ResultHandler; +import com.ververica.flink.table.jdbc.resulthandler.ResultHandlerFactory; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.types.Either; +import org.apache.flink.types.Row; + +import java.io.InputStream; +import java.io.Reader; +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.net.URL; +import java.sql.Array; +import java.sql.Blob; +import java.sql.Clob; +import java.sql.Date; +import java.sql.NClob; +import java.sql.Ref; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.RowId; +import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; +import java.sql.SQLWarning; +import java.sql.SQLXML; +import java.sql.Statement; +import java.sql.Time; +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.util.Calendar; +import java.util.List; +import java.util.Map; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * Flink JDBC result set. + */ +public class FlinkResultSet implements ResultSet { + + // If an empty array is fetched as result, at least sleep this long millis before next attempt + private static final int DEFAULT_INIT_SLEEP_MILLIS = 10; + // If an empty array is fetched as result, at most sleep this long millis before next attempt + private static final int DEFAULT_MAX_SLEEP_MILLIS = 1000; + + private final SessionClient session; + private final Either jobIdOrResultSet; + private final ResultHandler resultHandler; + private int fetchSize; + private final long maxRows; + private final FlinkStatement statement; + + private final AtomicRowData rowData; + + private boolean wasNull; + private boolean closed; + + public FlinkResultSet( + SessionClient session, + Either jobIdOrResultSet, + ResultHandler resultHandler, + long maxRows, + FlinkStatement statement) throws SQLException { + this.session = session; + this.jobIdOrResultSet = jobIdOrResultSet; + this.resultHandler = resultHandler; + this.fetchSize = 0; + this.maxRows = maxRows; + this.statement = statement; + + this.rowData = new AtomicRowData(); + + this.wasNull = false; + this.closed = false; + + } + + @Override + public boolean next() throws SQLException { + checkClosed(); + return rowData.nextRow(); + } + + @Override + public synchronized void close() throws SQLException { + if (closed) { + return; + } + + if (jobIdOrResultSet.isLeft()) { + // no need to lock, closing while fetching new results should throw exception + session.cancelJob(jobIdOrResultSet.left()); + } + closed = true; + } + + @Override + public boolean wasNull() throws SQLException { + checkClosed(); + return wasNull; + } + + @Override + public String getString(int columnIndex) throws SQLException { + Object o = getColumnValue(columnIndex); + if (o == null) { + return null; + } else if (o instanceof byte[]) { + return new String((byte[]) o); + } else { + return o.toString(); + } + } + + @Override + public boolean getBoolean(int columnIndex) throws SQLException { + Object o = getColumnValue(columnIndex); + if (o == null) { + return false; + } else if (o instanceof Boolean) { + return (Boolean) o; + } else if (o instanceof Number) { + return ((Number) o).intValue() != 0; + } else if (o instanceof String) { + // we follow Hive's implementation here, might not be very standard + return !(o.equals("0")); + } + throw new SQLException("Cannot convert column " + columnIndex + " to boolean"); + } + + @Override + public byte getByte(int columnIndex) throws SQLException { + Object o = getColumnValue(columnIndex); + if (o == null) { + return 0; + } else if (o instanceof Number) { + return ((Number) o).byteValue(); + } else if (o instanceof String) { + try { + return Byte.parseByte((String) o); + } catch (NumberFormatException e) { + throw new SQLException("Cannot convert column " + columnIndex + " to byte"); + } + } + throw new SQLException("Cannot convert column " + columnIndex + " to byte"); + } + + @Override + public short getShort(int columnIndex) throws SQLException { + Object o = getColumnValue(columnIndex); + if (o == null) { + return 0; + } else if (o instanceof Number) { + return ((Number) o).shortValue(); + } else if (o instanceof String) { + try { + return Short.parseShort((String) o); + } catch (NumberFormatException e) { + throw new SQLException("Cannot convert column " + columnIndex + " to short"); + } + } + throw new SQLException("Cannot convert column " + columnIndex + " to short"); + } + + @Override + public int getInt(int columnIndex) throws SQLException { + Object o = getColumnValue(columnIndex); + if (o == null) { + return 0; + } else if (o instanceof Number) { + return ((Number) o).intValue(); + } else if (o instanceof String) { + try { + return Integer.parseInt((String) o); + } catch (NumberFormatException e) { + throw new SQLException("Cannot convert column " + columnIndex + " to int"); + } + } + throw new SQLException("Cannot convert column " + columnIndex + " to int"); + } + + @Override + public long getLong(int columnIndex) throws SQLException { + Object o = getColumnValue(columnIndex); + if (o == null) { + return 0; + } else if (o instanceof Number) { + return ((Number) o).longValue(); + } else if (o instanceof String) { + try { + return Long.parseLong((String) o); + } catch (NumberFormatException e) { + throw new SQLException("Cannot convert column " + columnIndex + " to long"); + } + } + throw new SQLException("Cannot convert column " + columnIndex + " to long"); + } + + @Override + public float getFloat(int columnIndex) throws SQLException { + Object o = getColumnValue(columnIndex); + if (o == null) { + return 0; + } else if (o instanceof Number) { + return ((Number) o).floatValue(); + } else if (o instanceof String) { + try { + return Float.parseFloat((String) o); + } catch (NumberFormatException e) { + throw new SQLException("Cannot convert column " + columnIndex + " to float"); + } + } + throw new SQLException("Cannot convert column " + columnIndex + " to float"); + } + + @Override + public double getDouble(int columnIndex) throws SQLException { + Object o = getColumnValue(columnIndex); + if (o == null) { + return 0; + } else if (o instanceof Number) { + return ((Number) o).doubleValue(); + } else if (o instanceof String) { + try { + return Double.parseDouble((String) o); + } catch (NumberFormatException e) { + throw new SQLException("Cannot convert column " + columnIndex + " to double"); + } + } + throw new SQLException("Cannot convert column " + columnIndex + " to double"); + } + + @Override + public BigDecimal getBigDecimal(int columnIndex, int scale) throws SQLException { + Object o = getColumnValue(columnIndex); + if (o == null) { + return null; + } else if (o instanceof BigDecimal) { + return ((BigDecimal) o).setScale(scale, RoundingMode.HALF_EVEN); + } + throw new SQLException("Cannot convert column " + columnIndex + " to big decimal"); + } + + @Override + public byte[] getBytes(int columnIndex) throws SQLException { + Object o = getColumnValue(columnIndex); + if (o == null) { + return null; + } else if (o instanceof byte[]) { + return (byte[]) o; + } else if (o instanceof String) { + return ((String) o).getBytes(); + } + throw new SQLException("Cannot convert column " + columnIndex + " to bytes"); + } + + @Override + public Date getDate(int columnIndex) throws SQLException { + Object o = getColumnValue(columnIndex); + if (o == null) { + return null; + } else if (o instanceof Date) { + return (Date) o; + } else if (o instanceof LocalDate) { + return Date.valueOf((LocalDate) o); + } else if (o instanceof String) { + try { + return Date.valueOf((String) o); + } catch (IllegalArgumentException e) { + throw new SQLException("Cannot convert column " + columnIndex + " to date", e); + } + } + throw new SQLException("Cannot convert column " + columnIndex + " to date"); + } + + @Override + public Time getTime(int columnIndex) throws SQLException { + Object o = getColumnValue(columnIndex); + if (o == null) { + return null; + } else if (o instanceof Time) { + return (Time) o; + } else if (o instanceof LocalTime) { + return Time.valueOf((LocalTime) o); + } else if (o instanceof String) { + try { + return Time.valueOf((String) o); + } catch (IllegalArgumentException e) { + throw new SQLException("Cannot convert column " + columnIndex + " to time", e); + } + } + throw new SQLException("Cannot convert column " + columnIndex + " to time"); + } + + @Override + public Timestamp getTimestamp(int columnIndex) throws SQLException { + Object o = getColumnValue(columnIndex); + if (o == null) { + return null; + } else if (o instanceof Timestamp) { + return (Timestamp) o; + } else if (o instanceof LocalDateTime) { + return Timestamp.valueOf((LocalDateTime) o); + } else if (o instanceof OffsetDateTime) { + return Timestamp.valueOf(((OffsetDateTime) o).toLocalDateTime()); + } else if (o instanceof Instant) { + return Timestamp.from((Instant) o); + } else if (o instanceof String) { + try { + return Timestamp.valueOf((String) o); + } catch (IllegalArgumentException e) { + throw new SQLException("Cannot convert column " + columnIndex + " to timestamp", e); + } + } + throw new SQLException("Cannot convert column " + columnIndex + " to timestamp"); + } + + @Override + public InputStream getAsciiStream(int columnIndex) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getAsciiStream is not supported"); + } + + @Override + public InputStream getUnicodeStream(int columnIndex) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getUnicodeStream is not supported"); + } + + @Override + public InputStream getBinaryStream(int columnIndex) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getBinaryStream is not supported"); + } + + @Override + public String getString(String columnLabel) throws SQLException { + return getString(findColumn(columnLabel)); + } + + @Override + public boolean getBoolean(String columnLabel) throws SQLException { + return getBoolean(findColumn(columnLabel)); + } + + @Override + public byte getByte(String columnLabel) throws SQLException { + return getByte(findColumn(columnLabel)); + } + + @Override + public short getShort(String columnLabel) throws SQLException { + return getShort(findColumn(columnLabel)); + } + + @Override + public int getInt(String columnLabel) throws SQLException { + return getInt(findColumn(columnLabel)); + } + + @Override + public long getLong(String columnLabel) throws SQLException { + return getLong(findColumn(columnLabel)); + } + + @Override + public float getFloat(String columnLabel) throws SQLException { + return getFloat(findColumn(columnLabel)); + } + + @Override + public double getDouble(String columnLabel) throws SQLException { + return getDouble(findColumn(columnLabel)); + } + + @Override + public BigDecimal getBigDecimal(String columnLabel, int scale) throws SQLException { + return getBigDecimal(findColumn(columnLabel), scale); + } + + @Override + public byte[] getBytes(String columnLabel) throws SQLException { + return getBytes(findColumn(columnLabel)); + } + + @Override + public Date getDate(String columnLabel) throws SQLException { + return getDate(findColumn(columnLabel)); + } + + @Override + public Time getTime(String columnLabel) throws SQLException { + return getTime(findColumn(columnLabel)); + } + + @Override + public Timestamp getTimestamp(String columnLabel) throws SQLException { + return getTimestamp(findColumn(columnLabel)); + } + + @Override + public InputStream getAsciiStream(String columnLabel) throws SQLException { + return getAsciiStream(findColumn(columnLabel)); + } + + @Override + public InputStream getUnicodeStream(String columnLabel) throws SQLException { + return getUnicodeStream(findColumn(columnLabel)); + } + + @Override + public InputStream getBinaryStream(String columnLabel) throws SQLException { + return getBinaryStream(findColumn(columnLabel)); + } + + @Override + public SQLWarning getWarnings() throws SQLException { + // TODO + // we currently do not support this, + // but we can't throw a SQLException because we want to support beeline + return null; + } + + @Override + public void clearWarnings() throws SQLException { + // TODO + // we currently do not support this, + // but we can't throw a SQLException because we want to support beeline + } + + @Override + public String getCursorName() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getCursorName is not supported"); + } + + @Override + public ResultSetMetaData getMetaData() throws SQLException { + return new FlinkResultSetMetaData(rowData.getColumnInfos()); + } + + @Override + public Object getObject(int columnIndex) throws SQLException { + return getColumnValue(columnIndex); + } + + @Override + public Object getObject(String columnLabel) throws SQLException { + return getObject(findColumn(columnLabel)); + } + + @Override + public int findColumn(String columnLabel) throws SQLException { + checkClosed(); + + List columnInfos = rowData.getColumnInfos(); + for (int i = 0; i < columnInfos.size(); i++) { + if (columnInfos.get(i).getName().equals(columnLabel)) { + return i + 1; + } + } + + throw new SQLException("Column label " + columnLabel + " not found"); + } + + @Override + public Reader getCharacterStream(int columnIndex) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getCharacterStream is not supported"); + } + + @Override + public Reader getCharacterStream(String columnLabel) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getCharacterStream is not supported"); + } + + @Override + public BigDecimal getBigDecimal(int columnIndex) throws SQLException { + Object o = getColumnValue(columnIndex); + if (o == null) { + return null; + } else if (o instanceof BigDecimal) { + return (BigDecimal) o; + } + throw new SQLException("Cannot convert column " + columnIndex + " to big decimal"); + } + + @Override + public BigDecimal getBigDecimal(String columnLabel) throws SQLException { + return getBigDecimal(findColumn(columnLabel)); + } + + @Override + public boolean isBeforeFirst() throws SQLException { + return rowData.getRowCount() == 0; + } + + @Override + public boolean isAfterLast() throws SQLException { + return rowData.isAfterLast(); + } + + @Override + public boolean isFirst() throws SQLException { + return rowData.getRowCount() == 1; + } + + @Override + public boolean isLast() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#isLast is not supported"); + } + + @Override + public void beforeFirst() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#beforeFirst is not supported"); + } + + @Override + public void afterLast() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#afterLast is not supported"); + } + + @Override + public boolean first() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#first is not supported"); + } + + @Override + public boolean last() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#last is not supported"); + } + + @Override + public int getRow() throws SQLException { + if (rowData.isAfterLast()) { + return 0; + } else { + return (int) rowData.getRowCount(); + } + } + + @Override + public boolean absolute(int row) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#absolute is not supported"); + } + + @Override + public boolean relative(int rows) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#relative is not supported"); + } + + @Override + public boolean previous() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#previous is not supported"); + } + + @Override + public void setFetchDirection(int direction) throws SQLException { + if (direction != ResultSet.FETCH_FORWARD) { + throw new SQLFeatureNotSupportedException("Flink JDBC only supports ResultSet.FETCH_FORWARD"); + } + } + + @Override + public int getFetchDirection() throws SQLException { + return ResultSet.FETCH_FORWARD; + } + + @Override + public void setFetchSize(int rows) throws SQLException { + if (rows < 0) { + throw new SQLException("Fetch size must not be negative"); + } + fetchSize = rows; + } + + @Override + public int getFetchSize() throws SQLException { + return fetchSize; + } + + @Override + public int getType() throws SQLException { + return ResultSet.TYPE_FORWARD_ONLY; + } + + @Override + public int getConcurrency() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getConcurrency is not supported"); + } + + @Override + public boolean rowUpdated() throws SQLException { + return false; + } + + @Override + public boolean rowInserted() throws SQLException { + return false; + } + + @Override + public boolean rowDeleted() throws SQLException { + return false; + } + + @Override + public void updateNull(int columnIndex) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateNull is not supported"); + } + + @Override + public void updateBoolean(int columnIndex, boolean x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateBoolean is not supported"); + } + + @Override + public void updateByte(int columnIndex, byte x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateByte is not supported"); + } + + @Override + public void updateShort(int columnIndex, short x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateShort is not supported"); + } + + @Override + public void updateInt(int columnIndex, int x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateInt is not supported"); + } + + @Override + public void updateLong(int columnIndex, long x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateLong is not supported"); + } + + @Override + public void updateFloat(int columnIndex, float x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateFloat is not supported"); + } + + @Override + public void updateDouble(int columnIndex, double x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateDouble is not supported"); + } + + @Override + public void updateBigDecimal(int columnIndex, BigDecimal x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateBigDecimal is not supported"); + } + + @Override + public void updateString(int columnIndex, String x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateString is not supported"); + } + + @Override + public void updateBytes(int columnIndex, byte[] x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateBytes is not supported"); + } + + @Override + public void updateDate(int columnIndex, Date x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateDate is not supported"); + } + + @Override + public void updateTime(int columnIndex, Time x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateTime is not supported"); + } + + @Override + public void updateTimestamp(int columnIndex, Timestamp x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateTimestamp is not supported"); + } + + @Override + public void updateAsciiStream(int columnIndex, InputStream x, int length) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateAsciiStream is not supported"); + } + + @Override + public void updateBinaryStream(int columnIndex, InputStream x, int length) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateBinaryStream is not supported"); + } + + @Override + public void updateCharacterStream(int columnIndex, Reader x, int length) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateCharacterStream is not supported"); + } + + @Override + public void updateObject(int columnIndex, Object x, int scaleOrLength) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateObject is not supported"); + } + + @Override + public void updateObject(int columnIndex, Object x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateObject is not supported"); + } + + @Override + public void updateNull(String columnLabel) throws SQLException { + updateNull(findColumn(columnLabel)); + } + + @Override + public void updateBoolean(String columnLabel, boolean x) throws SQLException { + updateBoolean(findColumn(columnLabel), x); + } + + @Override + public void updateByte(String columnLabel, byte x) throws SQLException { + updateByte(findColumn(columnLabel), x); + } + + @Override + public void updateShort(String columnLabel, short x) throws SQLException { + updateShort(findColumn(columnLabel), x); + } + + @Override + public void updateInt(String columnLabel, int x) throws SQLException { + updateInt(findColumn(columnLabel), x); + } + + @Override + public void updateLong(String columnLabel, long x) throws SQLException { + updateLong(findColumn(columnLabel), x); + } + + @Override + public void updateFloat(String columnLabel, float x) throws SQLException { + updateFloat(findColumn(columnLabel), x); + } + + @Override + public void updateDouble(String columnLabel, double x) throws SQLException { + updateDouble(findColumn(columnLabel), x); + } + + @Override + public void updateBigDecimal(String columnLabel, BigDecimal x) throws SQLException { + updateBigDecimal(findColumn(columnLabel), x); + } + + @Override + public void updateString(String columnLabel, String x) throws SQLException { + updateString(findColumn(columnLabel), x); + } + + @Override + public void updateBytes(String columnLabel, byte[] x) throws SQLException { + updateBytes(findColumn(columnLabel), x); + } + + @Override + public void updateDate(String columnLabel, Date x) throws SQLException { + updateDate(findColumn(columnLabel), x); + } + + @Override + public void updateTime(String columnLabel, Time x) throws SQLException { + updateTime(findColumn(columnLabel), x); + } + + @Override + public void updateTimestamp(String columnLabel, Timestamp x) throws SQLException { + updateTimestamp(findColumn(columnLabel), x); + } + + @Override + public void updateAsciiStream(String columnLabel, InputStream x, int length) throws SQLException { + updateAsciiStream(findColumn(columnLabel), x, length); + } + + @Override + public void updateBinaryStream(String columnLabel, InputStream x, int length) throws SQLException { + updateBinaryStream(findColumn(columnLabel), x, length); + } + + @Override + public void updateCharacterStream(String columnLabel, Reader reader, int length) throws SQLException { + updateCharacterStream(findColumn(columnLabel), reader, length); + } + + @Override + public void updateObject(String columnLabel, Object x, int scaleOrLength) throws SQLException { + updateObject(findColumn(columnLabel), x, scaleOrLength); + } + + @Override + public void updateObject(String columnLabel, Object x) throws SQLException { + updateObject(findColumn(columnLabel), x); + } + + @Override + public void insertRow() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#insertRow is not supported"); + } + + @Override + public void updateRow() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateRow is not supported"); + } + + @Override + public void deleteRow() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#deleteRow is not supported"); + } + + @Override + public void refreshRow() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#refreshRow is not supported"); + } + + @Override + public void cancelRowUpdates() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#cancelRowUpdates is not supported"); + } + + @Override + public void moveToInsertRow() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#moveToInsertRow is not supported"); + } + + @Override + public void moveToCurrentRow() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#moveToCurrentRow is not supported"); + } + + @Override + public Statement getStatement() throws SQLException { + return statement; + } + + @Override + public Object getObject(int columnIndex, Map> map) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getObject is not supported"); + } + + @Override + public Ref getRef(int columnIndex) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getRef is not supported"); + } + + @Override + public Blob getBlob(int columnIndex) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getBlob is not supported"); + } + + @Override + public Clob getClob(int columnIndex) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getClob is not supported"); + } + + @Override + public Array getArray(int columnIndex) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getArray is not supported"); + } + + @Override + public Object getObject(String columnLabel, Map> map) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getObject is not supported"); + } + + @Override + public Ref getRef(String columnLabel) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getRef is not supported"); + } + + @Override + public Blob getBlob(String columnLabel) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getBlob is not supported"); + } + + @Override + public Clob getClob(String columnLabel) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getClob is not supported"); + } + + @Override + public Array getArray(String columnLabel) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getArray is not supported"); + } + + @Override + public Date getDate(int columnIndex, Calendar cal) throws SQLException { + Date date = getDate(columnIndex); + if (date == null) { + return null; + } else { + return new Date( + new Timestamp(date.getTime()) + .toLocalDateTime() + .atZone(cal.getTimeZone().toZoneId()) + .toInstant() + .toEpochMilli()); + } + } + + @Override + public Date getDate(String columnLabel, Calendar cal) throws SQLException { + return getDate(findColumn(columnLabel), cal); + } + + @Override + public Time getTime(int columnIndex, Calendar cal) throws SQLException { + Time time = getTime(columnIndex); + if (time == null) { + return null; + } else { + return new Time( + new Timestamp(time.getTime()) + .toLocalDateTime() + .atZone(cal.getTimeZone().toZoneId()) + .toInstant() + .toEpochMilli()); + } + } + + @Override + public Time getTime(String columnLabel, Calendar cal) throws SQLException { + return getTime(findColumn(columnLabel), cal); + } + + @Override + public Timestamp getTimestamp(int columnIndex, Calendar cal) throws SQLException { + Timestamp timestamp = getTimestamp(columnIndex); + if (timestamp == null) { + return null; + } else { + return new Timestamp( + timestamp + .toLocalDateTime() + .atZone(cal.getTimeZone().toZoneId()) + .toInstant() + .toEpochMilli()); + } + } + + @Override + public Timestamp getTimestamp(String columnLabel, Calendar cal) throws SQLException { + return getTimestamp(findColumn(columnLabel), cal); + } + + @Override + public URL getURL(int columnIndex) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getURL is not supported"); + } + + @Override + public URL getURL(String columnLabel) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getURL is not supported"); + } + + @Override + public void updateRef(int columnIndex, Ref x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateRef is not supported"); + } + + @Override + public void updateRef(String columnLabel, Ref x) throws SQLException { + updateRef(findColumn(columnLabel), x); + } + + @Override + public void updateBlob(int columnIndex, Blob x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateBlob is not supported"); + } + + @Override + public void updateBlob(String columnLabel, Blob x) throws SQLException { + updateBlob(findColumn(columnLabel), x); + } + + @Override + public void updateClob(int columnIndex, Clob x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateClob is not supported"); + } + + @Override + public void updateClob(String columnLabel, Clob x) throws SQLException { + updateClob(findColumn(columnLabel), x); + } + + @Override + public void updateArray(int columnIndex, Array x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateArray is not supported"); + } + + @Override + public void updateArray(String columnLabel, Array x) throws SQLException { + updateArray(findColumn(columnLabel), x); + } + + @Override + public RowId getRowId(int columnIndex) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getRowId is not supported"); + } + + @Override + public RowId getRowId(String columnLabel) throws SQLException { + return getRowId(findColumn(columnLabel)); + } + + @Override + public void updateRowId(int columnIndex, RowId x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateRowId is not supported"); + } + + @Override + public void updateRowId(String columnLabel, RowId x) throws SQLException { + updateRowId(findColumn(columnLabel), x); + } + + @Override + public int getHoldability() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getHoldability is not supported"); + } + + @Override + public boolean isClosed() throws SQLException { + return closed; + } + + @Override + public void updateNString(int columnIndex, String nString) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateNString is not supported"); + } + + @Override + public void updateNString(String columnLabel, String nString) throws SQLException { + updateNString(findColumn(columnLabel), nString); + } + + @Override + public void updateNClob(int columnIndex, NClob nClob) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateNClob is not supported"); + } + + @Override + public void updateNClob(String columnLabel, NClob nClob) throws SQLException { + updateNClob(findColumn(columnLabel), nClob); + } + + @Override + public NClob getNClob(int columnIndex) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getNClob is not supported"); + } + + @Override + public NClob getNClob(String columnLabel) throws SQLException { + return getNClob(findColumn(columnLabel)); + } + + @Override + public SQLXML getSQLXML(int columnIndex) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getSQLXML is not supported"); + } + + @Override + public SQLXML getSQLXML(String columnLabel) throws SQLException { + return getSQLXML(findColumn(columnLabel)); + } + + @Override + public void updateSQLXML(int columnIndex, SQLXML xmlObject) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateSQLXML is not supported"); + } + + @Override + public void updateSQLXML(String columnLabel, SQLXML xmlObject) throws SQLException { + updateSQLXML(findColumn(columnLabel), xmlObject); + } + + @Override + public String getNString(int columnIndex) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getNString is not supported"); + } + + @Override + public String getNString(String columnLabel) throws SQLException { + return getNString(findColumn(columnLabel)); + } + + @Override + public Reader getNCharacterStream(int columnIndex) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getNCharacterStream is not supported"); + } + + @Override + public Reader getNCharacterStream(String columnLabel) throws SQLException { + return getNCharacterStream(findColumn(columnLabel)); + } + + @Override + public void updateNCharacterStream(int columnIndex, Reader x, long length) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateNCharacterStream is not supported"); + } + + @Override + public void updateNCharacterStream(String columnLabel, Reader reader, long length) throws SQLException { + updateNCharacterStream(findColumn(columnLabel), reader, length); + } + + @Override + public void updateAsciiStream(int columnIndex, InputStream x, long length) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateAsciiStream is not supported"); + } + + @Override + public void updateBinaryStream(int columnIndex, InputStream x, long length) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateBinaryStream is not supported"); + } + + @Override + public void updateCharacterStream(int columnIndex, Reader x, long length) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateCharacterStream is not supported"); + } + + @Override + public void updateAsciiStream(String columnLabel, InputStream x, long length) throws SQLException { + updateAsciiStream(findColumn(columnLabel), x, length); + } + + @Override + public void updateBinaryStream(String columnLabel, InputStream x, long length) throws SQLException { + updateBinaryStream(findColumn(columnLabel), x, length); + } + + @Override + public void updateCharacterStream(String columnLabel, Reader reader, long length) throws SQLException { + updateCharacterStream(findColumn(columnLabel), reader, length); + } + + @Override + public void updateBlob(int columnIndex, InputStream inputStream, long length) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateBlob is not supported"); + } + + @Override + public void updateBlob(String columnLabel, InputStream inputStream, long length) throws SQLException { + updateBlob(findColumn(columnLabel), inputStream, length); + } + + @Override + public void updateClob(int columnIndex, Reader reader, long length) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateClob is not supported"); + } + + @Override + public void updateClob(String columnLabel, Reader reader, long length) throws SQLException { + updateClob(findColumn(columnLabel), reader, length); + } + + @Override + public void updateNClob(int columnIndex, Reader reader, long length) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateNClob is not supported"); + } + + @Override + public void updateNClob(String columnLabel, Reader reader, long length) throws SQLException { + updateNClob(findColumn(columnLabel), reader, length); + } + + @Override + public void updateNCharacterStream(int columnIndex, Reader x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateNCharacterStream is not supported"); + } + + @Override + public void updateNCharacterStream(String columnLabel, Reader reader) throws SQLException { + updateNCharacterStream(findColumn(columnLabel), reader); + } + + @Override + public void updateAsciiStream(int columnIndex, InputStream x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateAsciiStream is not supported"); + } + + @Override + public void updateBinaryStream(int columnIndex, InputStream x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateBinaryStream is not supported"); + } + + @Override + public void updateCharacterStream(int columnIndex, Reader x) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateCharacterStream is not supported"); + } + + @Override + public void updateAsciiStream(String columnLabel, InputStream x) throws SQLException { + updateAsciiStream(findColumn(columnLabel), x); + } + + @Override + public void updateBinaryStream(String columnLabel, InputStream x) throws SQLException { + updateBinaryStream(findColumn(columnLabel), x); + } + + @Override + public void updateCharacterStream(String columnLabel, Reader reader) throws SQLException { + updateCharacterStream(findColumn(columnLabel), reader); + } + + @Override + public void updateBlob(int columnIndex, InputStream inputStream) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateBlob is not supported"); + } + + @Override + public void updateBlob(String columnLabel, InputStream inputStream) throws SQLException { + updateBlob(findColumn(columnLabel), inputStream); + } + + @Override + public void updateClob(int columnIndex, Reader reader) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateClob is not supported"); + } + + @Override + public void updateClob(String columnLabel, Reader reader) throws SQLException { + updateClob(findColumn(columnLabel), reader); + } + + @Override + public void updateNClob(int columnIndex, Reader reader) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#updateNClob is not supported"); + } + + @Override + public void updateNClob(String columnLabel, Reader reader) throws SQLException { + updateNClob(findColumn(columnLabel), reader); + } + + @Override + public T getObject(int columnIndex, Class type) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#getObject is not supported"); + } + + @Override + public T getObject(String columnLabel, Class type) throws SQLException { + return getObject(findColumn(columnLabel), type); + } + + @Override + public T unwrap(Class iface) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#unwrap is not supported"); + } + + @Override + public boolean isWrapperFor(Class iface) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSet#isWrapperFor is not supported"); + } + + public static FlinkResultSet of(com.ververica.flink.table.gateway.rest.result.ResultSet resultSet) throws SQLException { + return new FlinkResultSet( + null, Either.Right(resultSet), ResultHandlerFactory.getDefaultResultHandler(), 0L, null); + } + + private Object getColumnValue(int columnIndex) throws SQLException { + checkClosed(); + checkHasData(); + checkIndexBound(columnIndex); + + Object o = rowData.getCurrentRow().getField(columnIndex - 1); + wasNull = (o == null); + return o; + } + + private void checkClosed() throws SQLException { + if (closed) { + throw new SQLException("This result set is already closed"); + } + } + + private void checkHasData() throws SQLException { + if (rowData.getRowCount() == 0) { + throw new SQLException( + "This result set is pointing before the first result. Please call next() first."); + } + if (rowData.isAfterLast()) { + throw new SQLException( + "This result set is pointing after the last result. No more results will be provided."); + } + } + + private void checkIndexBound(int columnIndex) throws SQLException { + int columnNum = rowData.getColumnInfos().size(); + if (columnIndex <= 0) { + throw new SQLException("Column index must be positive."); + } + if (columnIndex > columnNum) { + throw new SQLException( + "Column index " + columnIndex + " out of bound. There are only " + columnNum + " columns."); + } + } + + /** + * An atomic iterator-like data structure which read results from SQL gateway. + */ + private class AtomicRowData { + + private Row currentRow; + private int currentIdxInResponse; + // rowCount = Long.MAX_VALUE indicates that it is pointing after the last result + private long rowCount; + + private long currentToken; + private com.ververica.flink.table.gateway.rest.result.ResultSet currentResultSet; + private boolean hasMoreResponse; + + private final List columnInfos; + + private ReadWriteLock lock; + + AtomicRowData() throws SQLException { + this.currentIdxInResponse = -1; + this.rowCount = 0L; + + this.currentToken = -1L; + this.hasMoreResponse = true; + + this.lock = new ReentrantReadWriteLock(); + + // we fetch the first response here to get column infos + fetchNextResponse(false); + columnInfos = currentResultSet.getColumns(); + } + + boolean nextRow() throws SQLException { + if (noMoreRows()) { + rowCount = Long.MAX_VALUE; + return false; + } + + lock.writeLock().lock(); + try { + if (currentIdxInResponse + 1 < currentResultSet.getData().size()) { + // current batch of results hasn't been consumed + currentIdxInResponse++; + currentRow = currentResultSet.getData().get(currentIdxInResponse); + rowCount++; + return true; + } else if (fetchNextResponse(true)) { + // a new batch of results arrives + currentIdxInResponse = 0; + currentRow = currentResultSet.getData().get(0); + rowCount++; + return true; + } else { + // no more results + rowCount = Long.MAX_VALUE; + return false; + } + } finally { + lock.writeLock().unlock(); + } + } + + boolean noMoreRows() { + lock.readLock().lock(); + boolean ret = (maxRows > 0 && rowCount + 1 > maxRows) || !hasMoreResponse; + lock.readLock().unlock(); + return ret; + } + + boolean isAfterLast() { + lock.readLock().lock(); + boolean ret = rowCount == Long.MAX_VALUE; + lock.readLock().unlock(); + return ret; + } + + Row getCurrentRow() { + lock.readLock().lock(); + Row ret = currentRow; + lock.readLock().unlock(); + return ret; + } + + long getRowCount() { + lock.readLock().lock(); + long ret = rowCount; + lock.readLock().unlock(); + return ret; + } + + List getColumnInfos() { + return columnInfos; + } + + private boolean fetchNextResponse(boolean needData) throws SQLException { + // a quick check for more response + if (!hasMoreResponse) { + return false; + } + + if (jobIdOrResultSet.isRight()) { + // we can get results directly + if (currentResultSet == null) { + currentResultSet = resultHandler.handleResult(jobIdOrResultSet.right()); + hasMoreResponse = true; + } else { + hasMoreResponse = false; + } + return hasMoreResponse; + } + + // do the actual remote fetching work + int sleepMillis = DEFAULT_INIT_SLEEP_MILLIS; + while (true) { + currentToken++; + ResultFetchResponseBody response; + if (fetchSize > 0) { + response = session.fetchResult(jobIdOrResultSet.left(), currentToken, fetchSize); + } else { + response = session.fetchResult(jobIdOrResultSet.left(), currentToken); + } + hasMoreResponse = (response.getNextResultUri() != null); + + if (!hasMoreResponse) { + // no more response + return false; + } + + // response contains data + currentResultSet = response.getResults().get(0); + if (currentResultSet.getData().isEmpty() && needData) { + // empty array as result but we need data, sleep before next attempt + try { + Thread.sleep(sleepMillis); + sleepMillis = Math.min(sleepMillis * 2, DEFAULT_MAX_SLEEP_MILLIS); + } catch (InterruptedException e) { + throw new SQLException( + "Interrupted while fetching more results for job " + jobIdOrResultSet.left(), e); + } + } else { + // we get a new result set, possibly empty if we don't need data + currentResultSet = resultHandler.handleResult(currentResultSet); + break; + } + } + + return true; + } + } +} diff --git a/src/main/java/com/ververica/flink/table/jdbc/FlinkResultSetMetaData.java b/src/main/java/com/ververica/flink/table/jdbc/FlinkResultSetMetaData.java new file mode 100644 index 0000000..46c27ba --- /dev/null +++ b/src/main/java/com/ververica/flink/table/jdbc/FlinkResultSetMetaData.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.flink.table.jdbc; + +import com.ververica.flink.table.gateway.rest.result.ColumnInfo; +import com.ververica.flink.table.jdbc.type.FlinkSqlTypes; + +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; +import java.util.List; + +/** + * Fink JDBC result set meta data. + */ +public class FlinkResultSetMetaData implements ResultSetMetaData { + + private final List columns; + + public FlinkResultSetMetaData(List columns) { + this.columns = columns; + } + + @Override + public int getColumnCount() throws SQLException { + return columns.size(); + } + + @Override + public boolean isAutoIncrement(int column) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#isAutoIncrement is not supported"); + } + + @Override + public boolean isCaseSensitive(int column) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#isCaseSensitive is not supported"); + } + + @Override + public boolean isSearchable(int column) throws SQLException { + checkIndexBound(column); + return true; + } + + @Override + public boolean isCurrency(int column) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#isCurrency is not supported"); + } + + @Override + public int isNullable(int column) throws SQLException { + checkIndexBound(column); + return columns.get(column - 1).getLogicalType().isNullable() ? + ResultSetMetaData.columnNullable : + ResultSetMetaData.columnNoNulls; + } + + @Override + public boolean isSigned(int column) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#isSigned is not supported"); + } + + @Override + public int getColumnDisplaySize(int column) throws SQLException { + checkIndexBound(column); + return FlinkSqlTypes.getType(columns.get(column - 1).getLogicalType()).getDisplaySize(); + } + + @Override + public String getColumnLabel(int column) throws SQLException { + return getColumnName(column); + } + + @Override + public String getColumnName(int column) throws SQLException { + checkIndexBound(column); + return columns.get(column - 1).getName(); + } + + @Override + public String getSchemaName(int column) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#getSchemaName is not supported"); + } + + @Override + public int getPrecision(int column) throws SQLException { + checkIndexBound(column); + return FlinkSqlTypes.getType(columns.get(column - 1).getLogicalType()).getPrecision(); + } + + @Override + public int getScale(int column) throws SQLException { + checkIndexBound(column); + return FlinkSqlTypes.getType(columns.get(column - 1).getLogicalType()).getScale(); + } + + @Override + public String getTableName(int column) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#getTableName is not supported"); + } + + @Override + public String getCatalogName(int column) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#getCatalogName is not supported"); + } + + @Override + public int getColumnType(int column) throws SQLException { + checkIndexBound(column); + return FlinkSqlTypes.getType(columns.get(column - 1).getLogicalType()).getSqlType(); + } + + @Override + public String getColumnTypeName(int column) throws SQLException { + checkIndexBound(column); + return columns.get(column - 1).getType(); + } + + @Override + public boolean isReadOnly(int column) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#isReadOnly is not supported"); + } + + @Override + public boolean isWritable(int column) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#isWritable is not supported"); + } + + @Override + public boolean isDefinitelyWritable(int column) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#isDefinitelyWritable is not supported"); + } + + @Override + public String getColumnClassName(int column) throws SQLException { + checkIndexBound(column); + return columns.get(column - 1).getLogicalType().getDefaultConversion().getName(); + } + + @Override + public T unwrap(Class iface) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#unwrap is not supported"); + } + + @Override + public boolean isWrapperFor(Class iface) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#isWrapperFor is not supported"); + } + + private void checkIndexBound(int column) throws SQLException { + int columnNum = columns.size(); + if (column <= 0) { + throw new SQLException("Column index must be positive."); + } + if (column > columnNum) { + throw new SQLException( + "Column index " + column + " out of bound. There are only " + columnNum + " columns."); + } + } +} diff --git a/src/main/java/com/ververica/flink/table/jdbc/FlinkStatement.java b/src/main/java/com/ververica/flink/table/jdbc/FlinkStatement.java new file mode 100644 index 0000000..8dbca59 --- /dev/null +++ b/src/main/java/com/ververica/flink/table/jdbc/FlinkStatement.java @@ -0,0 +1,564 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.flink.table.jdbc; + +import com.ververica.flink.table.gateway.rest.message.StatementExecuteResponseBody; +import com.ververica.flink.table.jdbc.rest.RestUtils; +import com.ververica.flink.table.jdbc.rest.SessionClient; +import com.ververica.flink.table.jdbc.resulthandler.ResultHandlerFactory; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.types.Either; +import org.apache.flink.util.Preconditions; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; +import java.sql.SQLWarning; +import java.sql.Statement; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * Flink JDBC statement. + */ +public class FlinkStatement implements Statement { + + private static final List QUERY_COMMANDS = Arrays.asList( + "SELECT", + "SHOW_MODULES", + "SHOW_CATALOGS", + "SHOW_CURRENT_CATALOG", + "SHOW_DATABASES", + "SHOW_CURRENT_DATABASE", + "SHOW_TABLES", + "SHOW_FUNCTIONS", + "DESCRIBE", + "EXPLAIN"); + + private final SessionClient session; + private final FlinkConnection connection; + + private final AtomicReference currentStatements; + + private long maxRows; + private int queryTimeout; + private int fetchSize; + + private boolean closed; + + public FlinkStatement(SessionClient session, FlinkConnection connection) { + this.session = session; + this.connection = connection; + + this.currentStatements = new AtomicReference<>(); + + this.maxRows = 0; + this.queryTimeout = 0; + this.fetchSize = 0; + + this.closed = false; + } + + @Override + public ResultSet executeQuery(String sql) throws SQLException { + boolean isQuery = execute(sql); + if (!isQuery) { + throw new SQLException(sql + " is not a query"); + } + return getResultSet(); + } + + @Override + public int executeUpdate(String sql) throws SQLException { + return (int) executeLargeUpdate(sql); + } + + @Override + public void close() throws SQLException { + if (closed) { + return; + } + + cancel(); + closed = true; + } + + @Override + public int getMaxFieldSize() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#getMaxFieldSize is not supported"); + } + + @Override + public void setMaxFieldSize(int max) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#setMaxFieldSize is not supported"); + } + + @Override + public int getMaxRows() throws SQLException { + return (int) getLargeMaxRows(); + } + + @Override + public void setMaxRows(int max) throws SQLException { + setLargeMaxRows(max); + } + + @Override + public void setEscapeProcessing(boolean enable) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#setEscapeProcessing is not supported"); + } + + @Override + public int getQueryTimeout() throws SQLException { + checkClosed(); + return queryTimeout; + } + + @Override + public void setQueryTimeout(int seconds) throws SQLException { + checkClosed(); + if (seconds < 0) { + throw new SQLException("Query timeout must not be negative."); + } + + queryTimeout = seconds; + } + + @Override + public void cancel() throws SQLException { + checkClosed(); + + AtomicStatements statements = currentStatements.get(); + if (statements == null) { + // do nothing + return; + } + statements.cancel(); + currentStatements.set(null); + } + + @Override + public SQLWarning getWarnings() throws SQLException { + // TODO + // we currently do not support this, + // but we can't throw a SQLException because we want to support beeline + return null; + } + + @Override + public void clearWarnings() throws SQLException { + // TODO + // we currently do not support this, + // but we can't throw a SQLException because we want to support beeline + } + + @Override + public void setCursorName(String name) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#setCursorName is not supported"); + } + + @Override + public boolean execute(String sql) throws SQLException { + cancel(); + AtomicStatements statements = new AtomicResultSetStatements(sql); + statements.runNext(); + currentStatements.set(statements); + return statements.isQuery(); + } + + @Override + public ResultSet getResultSet() throws SQLException { + checkClosed(); + checkHasStatements(); + + AtomicStatements statements = currentStatements.get(); + if (!statements.isQuery()) { + throw new SQLException("Current result is not a result set. Please call getUpdateCount() instead."); + } + + Object ret = statements.getCurrentResult(); + if (ret instanceof ResultSet) { + return (ResultSet) ret; + } else { + throw new SQLException("Current result is not a result set."); + } + } + + @Override + public int getUpdateCount() throws SQLException { + return (int) getLargeUpdateCount(); + } + + @Override + public boolean getMoreResults() throws SQLException { + checkClosed(); + checkHasStatements(); + AtomicStatements statements = currentStatements.get(); + statements.runNext(); + return statements.isQuery(); + } + + @Override + public void setFetchDirection(int direction) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#setFetchDirection is not supported"); + } + + @Override + public int getFetchDirection() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#getFetchDirection is not supported"); + } + + @Override + public void setFetchSize(int rows) throws SQLException { + checkClosed(); + if (rows < 0) { + throw new SQLException("Fetch size must not be negative."); + } + + fetchSize = rows; + } + + @Override + public int getFetchSize() throws SQLException { + checkClosed(); + return fetchSize; + } + + @Override + public int getResultSetConcurrency() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#getResultSetConcurrency is not supported"); + } + + @Override + public int getResultSetType() throws SQLException { + return ResultSet.TYPE_FORWARD_ONLY; + } + + @Override + public void addBatch(String sql) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#addBatch is not supported"); + } + + @Override + public void clearBatch() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#clearBatch is not supported"); + } + + @Override + public int[] executeBatch() throws SQLException { + long[] result = executeLargeBatch(); + int[] ret = new int[result.length]; + for (int i = 0; i < result.length; i++) { + ret[i] = (int) result[i]; + } + return ret; + } + + @Override + public Connection getConnection() throws SQLException { + return connection; + } + + @Override + public boolean getMoreResults(int current) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#getMoreResults is not supported"); + } + + @Override + public ResultSet getGeneratedKeys() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#getGeneratedKeys is not supported"); + } + + @Override + public int executeUpdate(String sql, int autoGeneratedKeys) throws SQLException { + return (int) executeLargeUpdate(sql, autoGeneratedKeys); + } + + @Override + public int executeUpdate(String sql, int[] columnIndexes) throws SQLException { + return (int) executeLargeUpdate(sql, columnIndexes); + } + + @Override + public int executeUpdate(String sql, String[] columnNames) throws SQLException { + return (int) executeLargeUpdate(sql, columnNames); + } + + @Override + public boolean execute(String sql, int autoGeneratedKeys) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#execute is not supported"); + } + + @Override + public boolean execute(String sql, int[] columnIndexes) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#execute is not supported"); + } + + @Override + public boolean execute(String sql, String[] columnNames) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#execute is not supported"); + } + + @Override + public int getResultSetHoldability() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#execute is not supported"); + } + + @Override + public boolean isClosed() throws SQLException { + return closed; + } + + @Override + public void setPoolable(boolean poolable) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#setPoolable is not supported"); + } + + @Override + public boolean isPoolable() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#isPoolable is not supported"); + } + + @Override + public void closeOnCompletion() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#closeOnCompletion is not supported"); + } + + @Override + public boolean isCloseOnCompletion() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#isCloseOnCompletion is not supported"); + } + + @Override + public long getLargeUpdateCount() throws SQLException { + checkClosed(); + checkHasStatements(); + + AtomicStatements statements = currentStatements.get(); + if (statements.isQuery()) { + throw new SQLException("Current result is not an update count. Please call getResultSet() instead."); + } + + if (statements.afterLastStatement()) { + // According to the java doc of getMoreResults() + return -1L; + } else { + Object ret = statements.getCurrentResult(); + if (ret instanceof ResultSet) { + ResultSet rs = (ResultSet) ret; + if (rs.next()) { + try { + return rs.getLong(1); + } catch (SQLException e) { + throw new SQLException("Current result is not an update count."); + } + } else { + throw new SQLException("Current result is not an update count."); + } + } else { + throw new SQLException("Current result is not an update count."); + } + } + } + + @Override + public void setLargeMaxRows(long max) throws SQLException { + checkClosed(); + if (max < 0) { + throw new SQLException("Max rows must not be negative."); + } + + maxRows = max; + } + + @Override + public long getLargeMaxRows() throws SQLException { + checkClosed(); + return maxRows; + } + + @Override + public long[] executeLargeBatch() throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#executeLargeBatch is not supported"); + } + + @Override + public long executeLargeUpdate(String sql) throws SQLException { + boolean isQuery = execute(sql); + if (isQuery) { + throw new SQLException(sql + " is not an update statement"); + } + return getLargeUpdateCount(); + } + + @Override + public long executeLargeUpdate(String sql, int autoGeneratedKeys) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#executeLargeUpdate is not supported"); + } + + @Override + public long executeLargeUpdate(String sql, int[] columnIndexes) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#executeLargeUpdate is not supported"); + } + + @Override + public long executeLargeUpdate(String sql, String[] columnNames) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#executeLargeUpdate is not supported"); + } + + @Override + public T unwrap(Class iface) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#unwrap is not supported"); + } + + @Override + public boolean isWrapperFor(Class iface) throws SQLException { + throw new SQLFeatureNotSupportedException("FlinkStatement#isWrapperFor is not supported"); + } + + private void checkClosed() throws SQLException { + if (closed) { + throw new SQLException("This result set is already closed"); + } + } + + private void checkHasStatements() throws SQLException { + if (currentStatements.get() == null) { + throw new SQLException("No statement is running"); + } + } + + /** + * A group of statements executed in order, + * with atomic read results / change current statement interface. + * + * @param statement result type + */ + private interface AtomicStatements { + + boolean runNext() throws SQLException; + + R getCurrentResult(); + + boolean isQuery(); + + boolean afterLastStatement(); + + void cancel() throws SQLException; + } + + /** + * A group of statements executed in order, + * with atomic read results / change current statement interface. + * + *

These statements produce {@link ResultSet} as results. + */ + private class AtomicResultSetStatements implements AtomicStatements { + + private final String[] statements; + private int lastExecutedIdx; + + private Either jobIdOrResultSet; + private ResultSet currentResultSet; + private boolean isQuery; + + private ReadWriteLock lock; + + AtomicResultSetStatements(String stmt) { + this.statements = stmt.split(";"); + this.lastExecutedIdx = -1; + + this.lock = new ReentrantReadWriteLock(); + } + + @Override + public boolean runNext() throws SQLException { + lock.writeLock().lock(); + if (lastExecutedIdx < statements.length) { + lastExecutedIdx++; + } + if (lastExecutedIdx >= statements.length) { + // According to the java doc of getMoreResults() + isQuery = false; + lock.writeLock().unlock(); + return false; + } + String sql = statements[lastExecutedIdx]; + + try { + StatementExecuteResponseBody response; + response = queryTimeout > 0 ? + session.submitStatement(sql, queryTimeout * 1000L) : + session.submitStatement(sql); + + Preconditions.checkState( + response.getResults().size() == 1 && response.getStatementTypes().size() == 1, + "Statement " + sql + " should produce exactly 1 result set. This is a bug."); + jobIdOrResultSet = RestUtils.getEitherJobIdOrResultSet(response.getResults().get(0)); + currentResultSet = new FlinkResultSet( + session, + jobIdOrResultSet, + ResultHandlerFactory.getResultHandlerByStatementType(response.getStatementTypes().get(0)), + maxRows, + FlinkStatement.this); + currentResultSet.setFetchSize(fetchSize); + isQuery = QUERY_COMMANDS.contains(response.getStatementTypes().get(0)); + return true; + } finally { + lock.writeLock().unlock(); + } + } + + @Override + public boolean isQuery() { + lock.readLock().lock(); + boolean ret = isQuery; + lock.readLock().unlock(); + return ret; + } + + @Override + public boolean afterLastStatement() { + lock.readLock().lock(); + boolean ret = lastExecutedIdx >= statements.length; + lock.readLock().unlock(); + return ret; + } + + @Override + public ResultSet getCurrentResult() { + lock.readLock().lock(); + ResultSet ret = currentResultSet; + lock.readLock().unlock(); + return ret; + } + + @Override + public void cancel() throws SQLException { + lock.writeLock().lock(); + currentResultSet.close(); + lock.writeLock().unlock(); + } + } +} diff --git a/src/main/java/com/ververica/flink/table/jdbc/rest/RestUtils.java b/src/main/java/com/ververica/flink/table/jdbc/rest/RestUtils.java new file mode 100644 index 0000000..bff3429 --- /dev/null +++ b/src/main/java/com/ververica/flink/table/jdbc/rest/RestUtils.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.flink.table.jdbc.rest; + +import com.ververica.flink.table.gateway.rest.result.ConstantNames; +import com.ververica.flink.table.gateway.rest.result.ResultSet; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.types.Either; + +/** + * Utility class to handle REST data structures. + */ +public class RestUtils { + + public static JobID getJobID(ResultSet resultSet) { + if (resultSet.getColumns().size() != 1) { + throw new IllegalArgumentException("Should contain only one column. This is a bug."); + } + if (resultSet.getColumns().get(0).getName().equals(ConstantNames.JOB_ID)) { + String jobId = (String) resultSet.getData().get(0).getField(0); + return JobID.fromHexString(jobId); + } else { + throw new IllegalArgumentException("Column name should be " + ConstantNames.JOB_ID + ". This is a bug."); + } + } + + public static Either getEitherJobIdOrResultSet(ResultSet resultSet) { + if (resultSet.getColumns().size() == 1 && resultSet.getColumns().get(0).getName() + .equals(ConstantNames.JOB_ID)) { + String jobId = (String) resultSet.getData().get(0).getField(0); + return Either.Left(JobID.fromHexString(jobId)); + } else { + return Either.Right(resultSet); + } + } +} diff --git a/src/main/java/com/ververica/flink/table/jdbc/rest/SessionClient.java b/src/main/java/com/ververica/flink/table/jdbc/rest/SessionClient.java new file mode 100644 index 0000000..f49cbb6 --- /dev/null +++ b/src/main/java/com/ververica/flink/table/jdbc/rest/SessionClient.java @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.flink.table.jdbc.rest; + +import com.ververica.flink.table.gateway.rest.handler.GetInfoHeaders; +import com.ververica.flink.table.gateway.rest.handler.JobCancelHeaders; +import com.ververica.flink.table.gateway.rest.handler.ResultFetchHeaders; +import com.ververica.flink.table.gateway.rest.handler.SessionCloseHeaders; +import com.ververica.flink.table.gateway.rest.handler.SessionCreateHeaders; +import com.ververica.flink.table.gateway.rest.handler.SessionHeartbeatHeaders; +import com.ververica.flink.table.gateway.rest.handler.StatementExecuteHeaders; +import com.ververica.flink.table.gateway.rest.message.GetInfoResponseBody; +import com.ververica.flink.table.gateway.rest.message.ResultFetchMessageParameters; +import com.ververica.flink.table.gateway.rest.message.ResultFetchRequestBody; +import com.ververica.flink.table.gateway.rest.message.ResultFetchResponseBody; +import com.ververica.flink.table.gateway.rest.message.SessionCreateRequestBody; +import com.ververica.flink.table.gateway.rest.message.SessionJobMessageParameters; +import com.ververica.flink.table.gateway.rest.message.SessionMessageParameters; +import com.ververica.flink.table.gateway.rest.message.StatementExecuteRequestBody; +import com.ververica.flink.table.gateway.rest.message.StatementExecuteResponseBody; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.rest.RestClient; +import org.apache.flink.runtime.rest.RestClientConfiguration; +import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.util.ExecutorUtils; + +import java.sql.SQLException; +import java.util.Collections; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +/** + * A client to connect to Flink SQL gateway. + */ +public class SessionClient { + + private final String serverHost; + private final int serverPort; + private final String sessionName; + private final String planner; + private final String executionType; + private final RestClient restClient; + + private final ExecutorService executor; + private volatile String sessionId; + private volatile boolean isClosed = false; + + public SessionClient( + String serverHost, + int serverPort, + String sessionName, + String planner, + String executionType, + String threadName) + throws Exception { + this.serverHost = serverHost; + this.serverPort = serverPort; + this.sessionName = sessionName; + this.planner = planner; + this.executionType = executionType; + this.executor = Executors.newFixedThreadPool(4, new ExecutorThreadFactory(threadName)); + this.restClient = new RestClient(RestClientConfiguration.fromConfiguration(new Configuration()), executor); + + connectInternal(); + } + + public String getServerHost() { + return serverHost; + } + + public int getServerPort() { + return serverPort; + } + + public String getPlanner() { + return planner; + } + + private void connectInternal() throws Exception { + this.sessionId = restClient.sendRequest( + serverHost, + serverPort, + SessionCreateHeaders.getInstance(), + EmptyMessageParameters.getInstance(), + new SessionCreateRequestBody(sessionName, planner, executionType, Collections.emptyMap())) + .get().getSessionId(); + } + + public synchronized void close() throws Exception { + if (isClosed) { + return; + } + isClosed = true; + try { + restClient.sendRequest( + serverHost, + serverPort, + SessionCloseHeaders.getInstance(), + new SessionMessageParameters(sessionId), + EmptyRequestBody.getInstance()).get(); + } finally { + restClient.shutdown(Time.seconds(5)); + ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, executor); + } + } + + public synchronized void sendHeartbeat() throws SQLException { + checkState(); + try { + restClient.sendRequest( + serverHost, + serverPort, + SessionHeartbeatHeaders.getInstance(), + new SessionMessageParameters(sessionId), + EmptyRequestBody.getInstance()) + .get(); + } catch (Exception e) { + throw new SQLException("Failed to send heartbeat to server", e); + } + } + + public StatementExecuteResponseBody submitStatement(String stmt) throws SQLException { + return submitStatement(stmt, Long.MAX_VALUE); + } + + public synchronized StatementExecuteResponseBody submitStatement(String stmt, long executionTimeoutMillis) + throws SQLException { + checkState(); + try { + return restClient.sendRequest( + serverHost, + serverPort, + StatementExecuteHeaders.getInstance(), + new SessionMessageParameters(sessionId), + new StatementExecuteRequestBody(stmt, executionTimeoutMillis)) + .get(); + } catch (Exception e) { + throw new SQLException("Failed to submit statement `" + stmt + "` to server", e); + } + } + + public synchronized void cancelJob(JobID jobId) throws SQLException { + checkState(); + try { + restClient.sendRequest( + serverHost, + serverPort, + JobCancelHeaders.getInstance(), + new SessionJobMessageParameters(sessionId, jobId), + EmptyRequestBody.getInstance()) + .get(); + } catch (Exception e) { + throw new SQLException("Failed to cancel job " + jobId.toString(), e); + } + } + + public synchronized ResultFetchResponseBody fetchResult(JobID jobId, long token) throws SQLException { + return fetchResult(jobId, token, null); + } + + public synchronized ResultFetchResponseBody fetchResult( + JobID jobId, long token, Integer fetchSize) throws SQLException { + checkState(); + try { + return restClient.sendRequest( + serverHost, + serverPort, + ResultFetchHeaders.getInstance(), + new ResultFetchMessageParameters(sessionId, jobId, token), + new ResultFetchRequestBody(fetchSize)) + .get(); + } catch (Exception e) { + throw new SQLException( + "Failed to fetch result for job " + jobId.toString() + + " (token = " + token + ", fetchSize = " + fetchSize + ")", + e.getCause()); + } + } + + public GetInfoResponseBody getInfo() throws SQLException { + checkState(); + try { + return restClient.sendRequest( + serverHost, + serverPort, + GetInfoHeaders.getInstance(), + EmptyMessageParameters.getInstance(), + EmptyRequestBody.getInstance()) + .get(); + } catch (Exception e) { + throw new SQLException("Failed to get server info", e); + } + } + + private void checkState() { + if (isClosed) { + throw new IllegalStateException("Session is already closed."); + } + } +} diff --git a/src/main/java/com/ververica/flink/table/jdbc/resulthandler/DefaultResultHandler.java b/src/main/java/com/ververica/flink/table/jdbc/resulthandler/DefaultResultHandler.java new file mode 100644 index 0000000..d78c532 --- /dev/null +++ b/src/main/java/com/ververica/flink/table/jdbc/resulthandler/DefaultResultHandler.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.flink.table.jdbc.resulthandler; + +import com.ververica.flink.table.gateway.rest.result.ResultSet; + +/** + * A result handler that directly forwards the {@link ResultSet} produced by the REST API. + */ +public class DefaultResultHandler implements ResultHandler { + + @Override + public ResultSet handleResult(ResultSet raw) { + return raw; + } +} diff --git a/src/main/java/com/ververica/flink/table/jdbc/resulthandler/DescribeResultHandler.java b/src/main/java/com/ververica/flink/table/jdbc/resulthandler/DescribeResultHandler.java new file mode 100644 index 0000000..1341a42 --- /dev/null +++ b/src/main/java/com/ververica/flink/table/jdbc/resulthandler/DescribeResultHandler.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.flink.table.jdbc.resulthandler; + +import com.ververica.flink.table.gateway.rest.result.ColumnInfo; +import com.ververica.flink.table.gateway.rest.result.ConstantNames; +import com.ververica.flink.table.gateway.rest.result.ResultSet; +import com.ververica.flink.table.gateway.rest.result.TableSchemaUtil; + +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** + * A result handler that change the {@link ResultSet} produced by the describe statement of REST API + * to a form that can be printed to screen. + */ +public class DescribeResultHandler implements ResultHandler { + + @Override + public ResultSet handleResult(ResultSet raw) { + List rawColumnInfos = raw.getColumns(); + Preconditions.checkArgument( + rawColumnInfos.size() == 1 && + rawColumnInfos.get(0).getName().equals(ConstantNames.SCHEMA) && + rawColumnInfos.get(0).getLogicalType() instanceof VarCharType, + "Invalid DESCRIBE result schema"); + Preconditions.checkArgument( + raw.getData().size() == 1, + "DESCRIBE result should contain exactly 1 json string record"); + + List newColumnInfos = Arrays.asList( + new ColumnInfo("column_name", rawColumnInfos.get(0).getType()), + new ColumnInfo("column_type", rawColumnInfos.get(0).getType()), + ColumnInfo.create("nullable", new BooleanType(false)), + ColumnInfo.create("primary_key", new BooleanType(false))); + + Row rawRow = raw.getData().get(0); + String json = rawRow.getField(0).toString(); + TableSchema schema; + try { + schema = TableSchemaUtil.readTableSchemaFromJson(json); + } catch (JsonProcessingException e) { + throw new RuntimeException("Failed to parse json to table schema", e); + } + List primaryKeys; + if (schema.getPrimaryKey().isPresent()) { + primaryKeys = schema.getPrimaryKey().get().getColumns(); + } else { + primaryKeys = Collections.emptyList(); + } + + List newRows = new ArrayList<>(); + for (TableColumn column : schema.getTableColumns()) { + String name = column.getName(); + LogicalType type = column.getType().getLogicalType(); + newRows.add(Row.of(name, type.toString(), type.isNullable(), primaryKeys.contains(name))); + } + + return new ResultSet(newColumnInfos, newRows); + } +} diff --git a/src/main/java/com/ververica/flink/table/jdbc/resulthandler/ResultHandler.java b/src/main/java/com/ververica/flink/table/jdbc/resulthandler/ResultHandler.java new file mode 100644 index 0000000..1ad6ea1 --- /dev/null +++ b/src/main/java/com/ververica/flink/table/jdbc/resulthandler/ResultHandler.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.flink.table.jdbc.resulthandler; + +import com.ververica.flink.table.gateway.rest.result.ResultSet; + +/** + * An interface which change {@link ResultSet}s directly returned by REST API to the form we want. + */ +public interface ResultHandler { + + ResultSet handleResult(ResultSet raw); +} diff --git a/src/main/java/com/ververica/flink/table/jdbc/resulthandler/ResultHandlerFactory.java b/src/main/java/com/ververica/flink/table/jdbc/resulthandler/ResultHandlerFactory.java new file mode 100644 index 0000000..1fcc650 --- /dev/null +++ b/src/main/java/com/ververica/flink/table/jdbc/resulthandler/ResultHandlerFactory.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.flink.table.jdbc.resulthandler; + +/** + * Factory to create {@link ResultHandler}s. + */ +public class ResultHandlerFactory { + + public static ResultHandler getDefaultResultHandler() { + return new DefaultResultHandler(); + } + + public static ResultHandler getResultHandlerByStatementType(String statementType) { + if ("DESCRIBE".equals(statementType)) { + return new DescribeResultHandler(); + } else { + return getDefaultResultHandler(); + } + } +} diff --git a/src/main/java/com/ververica/flink/table/jdbc/type/FlinkSqlType.java b/src/main/java/com/ververica/flink/table/jdbc/type/FlinkSqlType.java new file mode 100644 index 0000000..24b353a --- /dev/null +++ b/src/main/java/com/ververica/flink/table/jdbc/type/FlinkSqlType.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.flink.table.jdbc.type; + +/** + * A data structure which records the information JDBC needed for SQL types. + */ +public class FlinkSqlType { + + private final int sqlType; + private final int precision; + private final int scale; + private final int displaySize; + + FlinkSqlType(int sqlType, int precision, int scale, int displaySize) { + this.sqlType = sqlType; + this.precision = precision; + this.scale = scale; + this.displaySize = displaySize; + } + + public int getSqlType() { + return sqlType; + } + + public int getPrecision() { + return precision; + } + + public int getScale() { + return scale; + } + + public int getDisplaySize() { + return displaySize; + } +} diff --git a/src/main/java/com/ververica/flink/table/jdbc/type/FlinkSqlTypes.java b/src/main/java/com/ververica/flink/table/jdbc/type/FlinkSqlTypes.java new file mode 100644 index 0000000..379e16b --- /dev/null +++ b/src/main/java/com/ververica/flink/table/jdbc/type/FlinkSqlTypes.java @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.flink.table.jdbc.type; + +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.CharType; +import org.apache.flink.table.types.logical.DateType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.DoubleType; +import org.apache.flink.table.types.logical.FloatType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.NullType; +import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.StructuredType; +import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.table.types.logical.ZonedTimestampType; + +import java.sql.Types; + +/** + * Defines all SQL types with information needed for JDBC. + */ +public class FlinkSqlTypes { + + public static final FlinkSqlType BOOLEAN = new FlinkSqlType(Types.BOOLEAN, 1, 0, 5); + public static final FlinkSqlType TINYINT = new FlinkSqlType(Types.TINYINT, 3, 0, 4); + public static final FlinkSqlType SMALLINT = new FlinkSqlType(Types.SMALLINT, 5, 0, 6); + public static final FlinkSqlType INT = new FlinkSqlType(Types.INTEGER, 10, 0, 11); + public static final FlinkSqlType BIGINT = new FlinkSqlType(Types.BIGINT, 19, 0, 20); + public static final FlinkSqlType FLOAT = new FlinkSqlType(Types.FLOAT, 7, 7, 24); + public static final FlinkSqlType DOUBLE = new FlinkSqlType(Types.DOUBLE, 15, 15, 25); + public static final FlinkSqlType DATE = new FlinkSqlType(Types.DATE, 10, 0, 10); + public static final FlinkSqlType NULL = new FlinkSqlType(Types.NULL, 0, 0, 4); + public static final FlinkSqlType ARRAY = new FlinkSqlType(Types.ARRAY, Integer.MAX_VALUE, 0, Integer.MAX_VALUE); + public static final FlinkSqlType STRUCT = new FlinkSqlType(Types.STRUCT, Integer.MAX_VALUE, 0, Integer.MAX_VALUE); + public static final FlinkSqlType OTHER = new FlinkSqlType(Types.OTHER, Integer.MAX_VALUE, 0, Integer.MAX_VALUE); + + public static FlinkSqlType createDecimalType(DecimalType type) { + int precision = type.getPrecision(); + int scale = type.getScale(); + return new FlinkSqlType(Types.DECIMAL, precision, scale, precision); + } + + public static FlinkSqlType createCharType(CharType type) { + int length = type.getLength(); + return new FlinkSqlType(Types.CHAR, length, 0, length); + } + + public static FlinkSqlType createVarCharType(VarCharType type) { + int length = type.getLength(); + return new FlinkSqlType(Types.VARCHAR, length, 0, length); + } + + public static FlinkSqlType createTimeType(TimeType type) { + int scale = type.getPrecision(); + if (scale > 0) { + return new FlinkSqlType(Types.TIME, 9 + scale, scale, 9 + scale); + } else { + return new FlinkSqlType(Types.TIME, 8, 0, 8); + } + } + + public static FlinkSqlType createTimestampType(TimestampType type) { + int scale = type.getPrecision(); + if (scale > 0) { + return new FlinkSqlType(Types.TIMESTAMP, 20 + scale, scale, 20 + scale); + } else { + return new FlinkSqlType(Types.TIMESTAMP, 19, 0, 19); + } + } + + public static FlinkSqlType createZonedTimstampType(ZonedTimestampType type) { + int scale = type.getPrecision(); + if (scale > 0) { + return new FlinkSqlType(Types.TIMESTAMP_WITH_TIMEZONE, 27 + scale, scale, 27 + scale); + } else { + return new FlinkSqlType(Types.TIMESTAMP_WITH_TIMEZONE, 26, 0, 26); + } + } + + public static FlinkSqlType createBinaryType(BinaryType type) { + int length = type.getLength(); + return new FlinkSqlType(Types.BINARY, length, 0, length); + } + + public static FlinkSqlType createVarBinaryType(VarBinaryType type) { + int length = type.getLength(); + return new FlinkSqlType(Types.VARBINARY, length, 0, length); + } + + public static boolean isNumeric(FlinkSqlType type) { + switch (type.getSqlType()) { + case Types.TINYINT: + case Types.SMALLINT: + case Types.INTEGER: + case Types.BIGINT: + case Types.FLOAT: + case Types.DOUBLE: + case Types.DECIMAL: + return true; + default: + return false; + } + } + + public static boolean isChar(FlinkSqlType type) { + switch (type.getSqlType()) { + case Types.CHAR: + case Types.VARCHAR: + return true; + default: + return false; + } + } + + public static FlinkSqlType getType(LogicalType type) { + if (type instanceof BooleanType) { + return BOOLEAN; + } else if (type instanceof TinyIntType) { + return TINYINT; + } else if (type instanceof SmallIntType) { + return SMALLINT; + } else if (type instanceof IntType) { + return INT; + } else if (type instanceof BigIntType) { + return BIGINT; + } else if (type instanceof FloatType) { + return FLOAT; + } else if (type instanceof DoubleType) { + return DOUBLE; + } else if (type instanceof DecimalType) { + return createDecimalType((DecimalType) type); + } else if (type instanceof CharType) { + return createCharType((CharType) type); + } else if (type instanceof VarCharType) { + return createVarCharType((VarCharType) type); + } else if (type instanceof DateType) { + return DATE; + } else if (type instanceof TimeType) { + return createTimeType((TimeType) type); + } else if (type instanceof TimestampType) { + return createTimestampType((TimestampType) type); + } else if (type instanceof BinaryType) { + return createBinaryType((BinaryType) type); + } else if (type instanceof VarBinaryType) { + return createVarBinaryType((VarBinaryType) type); + } else if (type instanceof NullType) { + return NULL; + } else if (type instanceof StructuredType) { + return STRUCT; + } else if (type instanceof ArrayType) { + return ARRAY; + } else if (type instanceof ZonedTimestampType) { + return createZonedTimstampType((ZonedTimestampType) type); + } else { + return OTHER; + } + } +} diff --git a/src/main/resources/META-INF/services/java.sql.Driver b/src/main/resources/META-INF/services/java.sql.Driver new file mode 100644 index 0000000..cafb67c --- /dev/null +++ b/src/main/resources/META-INF/services/java.sql.Driver @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +com.ververica.flink.table.jdbc.FlinkDriver diff --git a/src/test/java/com/ververica/flink/table/jdbc/FlinkConnectionTest.java b/src/test/java/com/ververica/flink/table/jdbc/FlinkConnectionTest.java new file mode 100644 index 0000000..f9784db --- /dev/null +++ b/src/test/java/com/ververica/flink/table/jdbc/FlinkConnectionTest.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.ververica.flink.table.jdbc; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.net.InetSocketAddress; +import java.sql.Connection; +import java.sql.SQLException; +import java.sql.Statement; + +/** + * Tests for {@link FlinkConnection}. + */ +public class FlinkConnectionTest { + + private static FlinkJdbcDriverTestingGateway gateway; + private static Connection connection; + + @BeforeClass + public static void beforeClass() throws Exception { + gateway = new FlinkJdbcDriverTestingGateway(); + gateway.start(); + + InetSocketAddress addr = gateway.getServerAddress(); + Assert.assertNotNull(addr); + connection = new FlinkConnection("jdbc:flink://localhost:" + addr.getPort() + "?planner=blink"); + Assert.assertTrue(connection.isValid(0)); + + Statement statement = connection.createStatement(); + statement.execute("USE CATALOG cat1"); + statement.execute("CREATE DATABASE db12"); + statement.execute("USE CATALOG cat2"); + statement.execute("CREATE DATABASE db22"); + statement.close(); + } + + @AfterClass + public static void afterClass() throws Exception { + connection.close(); + gateway.stop(); + } + + @Test + public void testGetSetCatalog() throws SQLException { + connection.setCatalog("cat1"); + Assert.assertEquals("cat1", connection.getCatalog()); + connection.setCatalog("cat2"); + Assert.assertEquals("cat2", connection.getCatalog()); + } + + @Test + public void testGetSetDatabase() throws SQLException { + connection.setCatalog("cat1"); + Assert.assertEquals("db11", connection.getSchema()); + connection.setSchema("db12"); + Assert.assertEquals("db12", connection.getSchema()); + connection.setCatalog("cat2"); + Assert.assertEquals("db21", connection.getSchema()); + connection.setSchema("db22"); + Assert.assertEquals("db22", connection.getSchema()); + } +} diff --git a/src/test/java/com/ververica/flink/table/jdbc/FlinkDatabaseMetaDataTest.java b/src/test/java/com/ververica/flink/table/jdbc/FlinkDatabaseMetaDataTest.java new file mode 100644 index 0000000..07e3a7f --- /dev/null +++ b/src/test/java/com/ververica/flink/table/jdbc/FlinkDatabaseMetaDataTest.java @@ -0,0 +1,186 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.ververica.flink.table.jdbc; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Test; + +import java.io.File; +import java.net.InetSocketAddress; +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; + +/** + * Tests for {@link FlinkDatabaseMetaData}. + */ +public class FlinkDatabaseMetaDataTest { + + private static FlinkJdbcDriverTestingGateway gateway; + private static Connection connection; + + @BeforeClass + public static void beforeClass() throws Exception { + gateway = new FlinkJdbcDriverTestingGateway(); + gateway.start(); + + InetSocketAddress addr = gateway.getServerAddress(); + Assert.assertNotNull(addr); + connection = new FlinkConnection("jdbc:flink://localhost:" + addr.getPort() + "?planner=blink"); + Assert.assertTrue(connection.isValid(0)); + + Statement statement = connection.createStatement(); + statement.execute("USE CATALOG cat1"); + statement.execute("CREATE DATABASE db12"); + statement.execute("USE CATALOG cat2"); + statement.execute("CREATE DATABASE db22"); + + createTmpTable("default_catalog", "default_database", "tab001", statement); + createTmpTable("default_catalog", "default_database", "tab002", statement); + createTmpTable("cat1", "db11", "tab111", statement); + createTmpTable("cat1", "db11", "tab112", statement); + createTmpTable("cat1", "db12", "tab121", statement); + createTmpTable("cat2", "db21", "tab211", statement); + createTmpTable("cat2", "db22", "tab221", statement); + createTmpTable("cat2", "db22", "tab222", statement); + + runStatementInCatalogAndDatabase( + "cat1", "db12", "CREATE VIEW view122 AS SELECT * FROM tab121", statement); + runStatementInCatalogAndDatabase( + "cat2", "db21", "CREATE VIEW view212 AS SELECT * FROM tab211", statement); + + statement.close(); + } + + @AfterClass + public static void afterClass() throws Exception { + connection.close(); + gateway.stop(); + } + + @Test + public void testGetCatalogs() throws SQLException { + DatabaseMetaData meta = connection.getMetaData(); + + String[][] expected = new String[][] { + new String[]{"cat1"}, + new String[]{"cat2"}, + new String[]{"default_catalog"}}; + compareStringResults(expected, meta.getCatalogs()); + } + + @Test + public void testGetSchemas() throws SQLException { + DatabaseMetaData meta = connection.getMetaData(); + + String[][] expected1 = new String[][] { + new String[]{"db11", "cat1"}, + new String[]{"db12", "cat1"}, + new String[]{"db21", "cat2"}, + new String[]{"db22", "cat2"}, + new String[]{"default_database", "default_catalog"}}; + compareStringResults(expected1, meta.getSchemas()); + + String[][] expected2 = new String[][] { + new String[]{"db12", "cat1"}, + new String[]{"db22", "cat2"}}; + compareStringResults(expected2, meta.getSchemas(null, "d%2")); + + String[][] expected3 = new String[][] { + new String[]{"db21", "cat2"}}; + compareStringResults(expected3, meta.getSchemas("cat2", "d__1")); + } + + // view in SQL gateway is not bounded to a certain database, this is a gateway bug + @Ignore + @Test + public void testGetTables() throws SQLException { + DatabaseMetaData meta = connection.getMetaData(); + + String[][] expected1 = new String[][] { + new String[]{"cat1", "db11", "tab111", "TABLE"}, + new String[]{"cat1", "db11", "tab112", "TABLE"}, + new String[]{"cat1", "db12", "tab121", "TABLE"}, + new String[]{"cat1", "db12", "view122", "VIEW"}}; + compareStringResults(expected1, meta.getTables("cat1", null, null, null)); + + String[][] expected2 = new String[][] { + new String[]{"cat2", "db11", "tab111", "TABLE"}, + new String[]{"cat2", "db12", "tab121", "TABLE"}}; + compareStringResults(expected2, meta.getTables("cat2", null, "t%1", new String[]{"TABLE"})); + + String[][] expected3 = new String[][] { + new String[]{"cat1", "db12", "view122", "VIEW"}}; + compareStringResults(expected2, meta.getTables("cat2", "d__2", "%2", new String[]{"VIEW"})); + } + + @Test + public void testGetColumns() throws SQLException { + DatabaseMetaData meta = connection.getMetaData(); + + String[][] expected1 = new String[][] { + new String[]{"cat1", "db11", "tab112", "fa"}, + new String[]{"cat2", "db22", "tab222", "fa"}, + new String[]{"default_catalog", "default_database", "tab002", "fa"}}; + compareStringResults(expected1, meta.getColumns(null, null, "t%2", "_a")); + + String[][] expected2 = new String[][] { + new String[]{"cat2", "db21", "tab211", "fb"}}; + compareStringResults(expected2, meta.getColumns("cat2", "%1", "t%1", "fb")); + } + + private void compareStringResults(String[][] expected, ResultSet rs) throws SQLException { + for (String[] row : expected) { + Assert.assertTrue(rs.next()); + for (int i = 0; i < row.length; i++) { + Assert.assertEquals(row[i], rs.getString(i + 1)); + } + } + Assert.assertFalse(rs.next()); + } + + private static void createTmpTable( + String catalog, String database, String table, Statement statement) throws Exception { + statement.execute("USE CATALOG " + catalog); + statement.execute("USE " + database); + + File tmpFile = File.createTempFile("Flink-JDBC-test", ".csv"); + tmpFile.deleteOnExit(); + statement.execute("CREATE TABLE " + table + "(" + + " fa INT," + + " fb VARCHAR(100)" + + ") WITH (" + + " 'connector.type'='filesystem'," + + " 'connector.path'='file://" + tmpFile.getPath() + "'," + + " 'format.type' = 'csv')"); + } + + private static void runStatementInCatalogAndDatabase( + String catalog, String database, String stmt, Statement statement) throws SQLException { + statement.execute("USE CATALOG " + catalog); + statement.execute("USE " + database); + statement.execute(stmt); + } +} diff --git a/src/test/java/com/ververica/flink/table/jdbc/FlinkJdbcDriverTestingGateway.java b/src/test/java/com/ververica/flink/table/jdbc/FlinkJdbcDriverTestingGateway.java new file mode 100644 index 0000000..6cce894 --- /dev/null +++ b/src/test/java/com/ververica/flink/table/jdbc/FlinkJdbcDriverTestingGateway.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.ververica.flink.table.jdbc; + +import com.ververica.flink.table.gateway.SessionManager; +import com.ververica.flink.table.gateway.config.Environment; +import com.ververica.flink.table.gateway.context.DefaultContext; +import com.ververica.flink.table.gateway.rest.SqlGatewayEndpoint; + +import org.apache.flink.client.cli.DefaultCLI; +import org.apache.flink.client.deployment.DefaultClusterClientServiceLoader; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.configuration.WebOptions; +import org.apache.flink.runtime.rest.RestServerEndpointConfiguration; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.FileUtils; + +import java.io.File; +import java.net.InetSocketAddress; +import java.net.URL; +import java.util.Collections; +import java.util.Objects; + +/** + * A {@link SqlGatewayEndpoint} for the Flink JDBC driver test cases to connect to. + */ +public class FlinkJdbcDriverTestingGateway { + + private static final String DEFAULT_ENVIRONMENT_FILE = "default-env.yaml"; + private static final String TEST_DATA_FILE = "test-data.csv"; + + private static final int NUM_TMS = 2; + private static final int NUM_SLOTS_PER_TM = 2; + + private MiniClusterWithClientResource miniClusterWithClientResource; + private SqlGatewayEndpoint endpoint; + + public void start() throws Exception { + miniClusterWithClientResource = new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setConfiguration(getMiniClusterConfig()) + .setNumberTaskManagers(NUM_TMS) + .setNumberSlotsPerTaskManager(NUM_SLOTS_PER_TM) + .build()); + miniClusterWithClientResource.before(); + ClusterClient clusterClient = miniClusterWithClientResource.getClusterClient(); + + final URL envUrl = FlinkJdbcDriverTestingGateway.class.getClassLoader().getResource(DEFAULT_ENVIRONMENT_FILE); + Objects.requireNonNull(envUrl); + final URL dataUrl = FlinkJdbcDriverTestingGateway.class.getClassLoader().getResource(TEST_DATA_FILE); + Objects.requireNonNull(dataUrl); + String schema = FileUtils.readFileUtf8(new File(envUrl.getFile())) + .replace("$VAR_SOURCE_PATH", dataUrl.getPath()); + Environment env = Environment.parse(schema); + + DefaultContext defaultContext = new DefaultContext( + env, + Collections.emptyList(), + clusterClient.getFlinkConfiguration(), + new DefaultCLI(clusterClient.getFlinkConfiguration()), + new DefaultClusterClientServiceLoader()); + SessionManager sessionManager = new SessionManager(defaultContext); + + endpoint = new SqlGatewayEndpoint( + RestServerEndpointConfiguration.fromConfiguration(getEndpointConfig()), + sessionManager); + endpoint.start(); + } + + public InetSocketAddress getServerAddress() { + return endpoint.getServerAddress(); + } + + public void stop() throws Exception { + endpoint.close(); + miniClusterWithClientResource.after(); + } + + private static Configuration getMiniClusterConfig() { + Configuration config = new Configuration(); + config.set(TaskManagerOptions.MANAGED_MEMORY_SIZE, MemorySize.parse("4m")); + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); + config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, NUM_SLOTS_PER_TM); + config.setBoolean(WebOptions.SUBMIT_ENABLE, false); + return config; + } + + private static Configuration getEndpointConfig() { + Configuration config = new Configuration(); + config.setString(RestOptions.ADDRESS, "localhost"); + config.setString(RestOptions.BIND_PORT, "0-65535"); + return config; + } +} diff --git a/src/test/java/com/ververica/flink/table/jdbc/FlinkResultSetTest.java b/src/test/java/com/ververica/flink/table/jdbc/FlinkResultSetTest.java new file mode 100644 index 0000000..1042c15 --- /dev/null +++ b/src/test/java/com/ververica/flink/table/jdbc/FlinkResultSetTest.java @@ -0,0 +1,486 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.ververica.flink.table.jdbc; + +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.math.BigDecimal; +import java.net.InetSocketAddress; +import java.sql.Connection; +import java.sql.Date; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Time; +import java.sql.Timestamp; +import java.sql.Types; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZonedDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Calendar; +import java.util.List; +import java.util.TimeZone; + +/** + * Tests for {@link FlinkResultSet}. + */ +public class FlinkResultSetTest { + + private static FlinkJdbcDriverTestingGateway gateway; + private static Connection connection; + private Statement statement; + + @BeforeClass + public static void beforeClass() throws Exception { + gateway = new FlinkJdbcDriverTestingGateway(); + gateway.start(); + + InetSocketAddress addr = gateway.getServerAddress(); + Assert.assertNotNull(addr); + connection = new FlinkConnection("jdbc:flink://localhost:" + addr.getPort() + "?planner=blink"); + Assert.assertTrue(connection.isValid(0)); + } + + @AfterClass + public static void afterClass() throws Exception { + connection.close(); + gateway.stop(); + } + + @Before + public void before() throws SQLException { + statement = connection.createStatement(); + } + + @After + public void after() throws SQLException { + statement.close(); + } + + @Test + public void testObjectTypes() throws SQLException { + checkRepresentation("true", Types.BOOLEAN, true); + checkRepresentation("CAST('123' AS TINYINT)", Types.TINYINT, (byte) 123); + checkRepresentation("CAST('123' AS SMALLINT)", Types.SMALLINT, (short) 123); + checkRepresentation("123", Types.INTEGER, 123); + checkRepresentation("12300000000", Types.BIGINT, 12300000000L); + checkRepresentation("CAST('123.45' AS FLOAT)", Types.FLOAT, 123.45f); + checkRepresentation("1e-1", Types.DOUBLE, 0.1); + checkRepresentation("CAST('123.45' AS DECIMAL(5, 2))", Types.DECIMAL, BigDecimal.valueOf(123.45)); + checkRepresentation("CAST('hello' as VARCHAR(10))", Types.VARCHAR, "hello"); + checkRepresentation("CAST('foo' as CHAR(5))", Types.CHAR, "foo "); + checkRepresentation("CAST('2020-02-11' as DATE)", Types.DATE, LocalDate.of(2020, 2, 11)); + checkRepresentation("CAST('15:43:00.123' AS TIME(3))", Types.TIME, LocalTime.of(15, 43, 0, 123000000)); + checkRepresentation("CAST('2020-02-11 15:43:00.123' AS TIMESTAMP(3))", Types.TIMESTAMP, LocalDateTime.of(2020, 2, 11, 15, 43, 0, 123000000)); + + // TODO ExpressionReducer will throw exception + // checkRepresentation("1.0E0 / 0.0E0", Types.DOUBLE, Double.POSITIVE_INFINITY); + // checkRepresentation("0.0E0 / 0.0E0", Types.DOUBLE, Double.NaN); + } + + private void checkRepresentation(String expression, int expectedSqlType, Object expected) throws SQLException { + try (ResultSet rs = statement.executeQuery("SELECT " + expression)) { + ResultSetMetaData metadata = rs.getMetaData(); + Assert.assertEquals(1, metadata.getColumnCount()); + Assert.assertEquals(expectedSqlType, metadata.getColumnType(1)); + Assert.assertTrue(rs.next()); + Assert.assertEquals(expected, rs.getObject(1)); + Assert.assertFalse(rs.next()); + } + } + + @Test + public void testGetString() throws SQLException { + try (ResultSet rs = statement.executeQuery("SELECT " + + "CAST('str1' AS CHAR(4)) x, " + + "CAST('str2' AS VARCHAR(4)), " + + "CAST('str3' AS BINARY(4)), " + + "CAST('str4' AS VARBINARY(4)), " + + "CAST(NULL AS VARCHAR(4))")) { + Assert.assertTrue(rs.next()); + Assert.assertEquals("str1", rs.getString(1)); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals("str2", rs.getString(2)); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals("str3", rs.getString(3)); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals("str4", rs.getString(4)); + Assert.assertFalse(rs.wasNull()); + Assert.assertNull(rs.getString(5)); + Assert.assertTrue(rs.wasNull()); + Assert.assertEquals("str1", rs.getString("x")); + Assert.assertFalse(rs.wasNull()); + Assert.assertFalse(rs.next()); + } + } + + @Test + public void testGetBoolean() throws SQLException { + try (ResultSet rs = statement.executeQuery("SELECT " + + "true x, 0, 'hello', '0', CAST(NULL AS BOOLEAN)")) { + Assert.assertTrue(rs.next()); + Assert.assertTrue(rs.getBoolean(1)); + Assert.assertFalse(rs.wasNull()); + Assert.assertFalse(rs.getBoolean(2)); + Assert.assertFalse(rs.wasNull()); + Assert.assertTrue(rs.getBoolean(3)); + Assert.assertFalse(rs.wasNull()); + Assert.assertFalse(rs.getBoolean(4)); + Assert.assertFalse(rs.wasNull()); + Assert.assertNull(rs.getString(5)); + Assert.assertTrue(rs.wasNull()); + Assert.assertTrue(rs.getBoolean("x")); + Assert.assertFalse(rs.wasNull()); + Assert.assertFalse(rs.next()); + } + } + + @Test + public void testGetByte() throws SQLException { + try (ResultSet rs = statement.executeQuery("SELECT " + + "CAST(1 AS TINYINT) x, 2, '3', CAST(NULL AS TINYINT)")) { + Assert.assertTrue(rs.next()); + Assert.assertEquals((byte) 1, rs.getByte(1)); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals((byte) 2, rs.getByte(2)); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals((byte) 3, rs.getByte(3)); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals(0, rs.getByte(4)); + Assert.assertTrue(rs.wasNull()); + Assert.assertEquals((byte) 1, rs.getByte("x")); + Assert.assertFalse(rs.wasNull()); + Assert.assertFalse(rs.next()); + } + } + + @Test + public void testGetShort() throws SQLException { + try (ResultSet rs = statement.executeQuery("SELECT " + + "CAST(1 AS SMALLINT) x, 2, '3', CAST(NULL AS SMALLINT)")) { + Assert.assertTrue(rs.next()); + Assert.assertEquals((short) 1, rs.getShort(1)); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals((short) 2, rs.getShort(2)); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals((short) 3, rs.getShort(3)); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals(0, rs.getShort(4)); + Assert.assertTrue(rs.wasNull()); + Assert.assertEquals((short) 1, rs.getShort("x")); + Assert.assertFalse(rs.wasNull()); + Assert.assertFalse(rs.next()); + } + } + + @Test + public void testGetInt() throws SQLException { + try (ResultSet rs = statement.executeQuery("SELECT " + + "1 x, '2', CAST(NULL AS INT)")) { + Assert.assertTrue(rs.next()); + Assert.assertEquals(1, rs.getInt(1)); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals(2, rs.getInt(2)); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals(0, rs.getInt(3)); + Assert.assertTrue(rs.wasNull()); + Assert.assertEquals(1, rs.getInt("x")); + Assert.assertFalse(rs.wasNull()); + Assert.assertFalse(rs.next()); + } + } + + @Test + public void testGetLong() throws SQLException { + try (ResultSet rs = statement.executeQuery("SELECT " + + "CAST(1 AS BIGINT) x, 2, '3', CAST(NULL AS BIGINT)")) { + Assert.assertTrue(rs.next()); + Assert.assertEquals(1L, rs.getLong(1)); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals(2L, rs.getLong(2)); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals(3L, rs.getLong(3)); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals(0L, rs.getLong(4)); + Assert.assertTrue(rs.wasNull()); + Assert.assertEquals(1L, rs.getLong("x")); + Assert.assertFalse(rs.wasNull()); + Assert.assertFalse(rs.next()); + } + } + + @Test + public void testGetFloat() throws SQLException { + try (ResultSet rs = statement.executeQuery("SELECT " + + "CAST(0.2 AS FLOAT) x, 0.4, '0.8', CAST(NULL AS FLOAT)")) { + Assert.assertTrue(rs.next()); + Assert.assertEquals(0.2F, rs.getFloat(1), 0F); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals(0.4F, rs.getFloat(2), 0F); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals(0.8F, rs.getFloat(3), 0F); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals(0F, rs.getLong(4), 0F); + Assert.assertTrue(rs.wasNull()); + Assert.assertEquals(0.2F, rs.getFloat("x"), 0F); + Assert.assertFalse(rs.wasNull()); + Assert.assertFalse(rs.next()); + } + } + + @Test + public void testGetDouble() throws SQLException { + try (ResultSet rs = statement.executeQuery("SELECT " + + "CAST(0.2 AS DOUBLE) x, 0.4, '0.8', CAST(NULL AS DOUBLE)")) { + Assert.assertTrue(rs.next()); + Assert.assertEquals(0.2, rs.getDouble(1), 0D); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals(0.4, rs.getDouble(2), 0D); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals(0.8, rs.getDouble(3), 0D); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals(0D, rs.getDouble(4), 0D); + Assert.assertTrue(rs.wasNull()); + Assert.assertEquals(0.2, rs.getDouble("x"), 0D); + Assert.assertFalse(rs.wasNull()); + Assert.assertFalse(rs.next()); + } + } + + @Test + public void testGetBigDecimal() throws SQLException { + try (ResultSet rs = statement.executeQuery("SELECT " + + "CAST(123.45 AS DECIMAL(5, 2)) x, CAST(NULL AS DECIMAL(5, 2))")) { + Assert.assertTrue(rs.next()); + Assert.assertEquals(BigDecimal.valueOf(123.45), rs.getBigDecimal(1)); + Assert.assertFalse(rs.wasNull()); + Assert.assertNull(rs.getBigDecimal(2)); + Assert.assertTrue(rs.wasNull()); + Assert.assertEquals(BigDecimal.valueOf(123.45), rs.getBigDecimal("x")); + Assert.assertFalse(rs.wasNull()); + Assert.assertFalse(rs.next()); + } + } + + @Test + public void testGetBytes() throws SQLException { + try (ResultSet rs = statement.executeQuery("SELECT " + + "CAST('str1' AS BINARY(4)) x, " + + "CAST('str2' AS VARBINARY(4)), " + + "CAST('str3' AS CHAR(4)), " + + "CAST('str4' AS VARCHAR(4)), " + + "CAST(NULL AS BINARY(4))")) { + Assert.assertTrue(rs.next()); + Assert.assertArrayEquals("str1".getBytes(), rs.getBytes(1)); + Assert.assertFalse(rs.wasNull()); + Assert.assertArrayEquals("str2".getBytes(), rs.getBytes(2)); + Assert.assertFalse(rs.wasNull()); + Assert.assertArrayEquals("str3".getBytes(), rs.getBytes(3)); + Assert.assertFalse(rs.wasNull()); + Assert.assertArrayEquals("str4".getBytes(), rs.getBytes(4)); + Assert.assertFalse(rs.wasNull()); + Assert.assertNull(rs.getBytes(5)); + Assert.assertTrue(rs.wasNull()); + Assert.assertArrayEquals("str1".getBytes(), rs.getBytes("x")); + Assert.assertFalse(rs.wasNull()); + Assert.assertFalse(rs.next()); + } + } + + @Test + public void testGetDate() throws SQLException { + try (ResultSet rs = statement.executeQuery("SELECT " + + "CAST('2020-02-12' AS DATE) x, '2020-02-13', CAST(NULL AS DATE)")) { + Assert.assertTrue(rs.next()); + Assert.assertEquals(Date.valueOf("2020-02-12"), rs.getDate(1)); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals(Date.valueOf("2020-02-13"), rs.getDate(2)); + Assert.assertFalse(rs.wasNull()); + Assert.assertNull(rs.getBytes(3)); + Assert.assertTrue(rs.wasNull()); + Assert.assertEquals(Date.valueOf("2020-02-12"), rs.getDate("x")); + Assert.assertFalse(rs.wasNull()); + + TimeZone tz = TimeZone.getTimeZone("UTC"); + Assert.assertEquals( + new Date(ZonedDateTime.of( + LocalDateTime.of(2020, 2, 12, 0, 0, 0), + tz.toZoneId()).toInstant().toEpochMilli()), + rs.getDate("x", Calendar.getInstance(tz))); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals( + new Date(ZonedDateTime.of( + LocalDateTime.of(2020, 2, 13, 0, 0, 0), + tz.toZoneId()).toInstant().toEpochMilli()), + rs.getDate(2, Calendar.getInstance(tz))); + Assert.assertFalse(rs.wasNull()); + + Assert.assertFalse(rs.next()); + } + } + + @Test + public void testGetTime() throws SQLException { + try (ResultSet rs = statement.executeQuery("SELECT " + + "CAST('15:20:00' AS TIME) x, '16:20:00', CAST(NULL AS TIME)")) { + Assert.assertTrue(rs.next()); + Assert.assertEquals(Time.valueOf("15:20:00"), rs.getTime(1)); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals(Time.valueOf("16:20:00"), rs.getTime(2)); + Assert.assertFalse(rs.wasNull()); + Assert.assertNull(rs.getBytes(3)); + Assert.assertTrue(rs.wasNull()); + Assert.assertEquals(Time.valueOf("15:20:00"), rs.getTime("x")); + Assert.assertFalse(rs.wasNull()); + + TimeZone tz = TimeZone.getTimeZone("UTC"); + Assert.assertEquals( + new Time(ZonedDateTime.of( + LocalDateTime.of(1970, 1, 1, 15, 20, 0), + tz.toZoneId()).toInstant().toEpochMilli()), + rs.getTime("x", Calendar.getInstance(tz))); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals( + new Time(ZonedDateTime.of( + LocalDateTime.of(1970, 1, 1, 16, 20, 0), + tz.toZoneId()).toInstant().toEpochMilli()), + rs.getTime(2, Calendar.getInstance(tz))); + Assert.assertFalse(rs.wasNull()); + + Assert.assertFalse(rs.next()); + } + } + + @Test + public void testGetTimestamp() throws SQLException { + try (ResultSet rs = statement.executeQuery("SELECT " + + "CAST('2020-02-12 15:20:00' AS TIMESTAMP) x, '2020-02-13 16:20:00', CAST(NULL AS TIMESTAMP)")) { + Assert.assertTrue(rs.next()); + Assert.assertEquals(Timestamp.valueOf("2020-02-12 15:20:00"), rs.getTimestamp(1)); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals(Timestamp.valueOf("2020-02-13 16:20:00"), rs.getTimestamp(2)); + Assert.assertFalse(rs.wasNull()); + Assert.assertNull(rs.getBytes(3)); + Assert.assertTrue(rs.wasNull()); + Assert.assertEquals(Timestamp.valueOf("2020-02-12 15:20:00"), rs.getTimestamp("x")); + Assert.assertFalse(rs.wasNull()); + + TimeZone tz = TimeZone.getTimeZone("UTC"); + Assert.assertEquals( + new Timestamp(ZonedDateTime.of( + LocalDateTime.of(2020, 2, 12, 15, 20, 0), + tz.toZoneId()).toInstant().toEpochMilli()), + rs.getTimestamp("x", Calendar.getInstance(tz))); + Assert.assertFalse(rs.wasNull()); + Assert.assertEquals( + new Timestamp(ZonedDateTime.of( + LocalDateTime.of(2020, 2, 13, 16, 20, 0), + tz.toZoneId()).toInstant().toEpochMilli()), + rs.getTimestamp(2, Calendar.getInstance(tz))); + Assert.assertFalse(rs.wasNull()); + + Assert.assertFalse(rs.next()); + } + } + + @Test + public void testPositions() throws SQLException { + try (ResultSet rs = statement.executeQuery("SELECT * FROM myTable LIMIT 2")) { + Assert.assertTrue(rs.isBeforeFirst()); + Assert.assertFalse(rs.isFirst()); + Assert.assertFalse(rs.isAfterLast()); + Assert.assertEquals(0, rs.getRow()); + + Assert.assertTrue(rs.next()); + Assert.assertFalse(rs.isBeforeFirst()); + Assert.assertTrue(rs.isFirst()); + Assert.assertFalse(rs.isAfterLast()); + Assert.assertEquals(1, rs.getRow()); + + Assert.assertTrue(rs.next()); + Assert.assertFalse(rs.isBeforeFirst()); + Assert.assertFalse(rs.isFirst()); + Assert.assertFalse(rs.isAfterLast()); + Assert.assertEquals(2, rs.getRow()); + + Assert.assertFalse(rs.next()); + Assert.assertFalse(rs.isBeforeFirst()); + Assert.assertFalse(rs.isFirst()); + Assert.assertTrue(rs.isAfterLast()); + Assert.assertEquals(0, rs.getRow()); + } + } + + @Test + public void testFetchResultMultipleTimes() throws SQLException { + int[] expectedInt = new int[]{ + 22, 32, 32, 42, 42, 52}; + String[] expectedString = new String[]{ + "BBB Hi", "CCC World", "DDD Hello!!!!", "AAA Hello", "EEE Hi!!!!", "FFF World!!!!"}; + + statement.setFetchSize(2); + try (ResultSet rs = statement.executeQuery("SELECT * FROM myTable ORDER BY a, b")) { + for (int i = 0; i < expectedInt.length; i++) { + Assert.assertTrue(rs.next()); + Assert.assertEquals(i + 1, rs.getRow()); + Assert.assertEquals(expectedInt[i], rs.getInt(1)); + Assert.assertEquals(expectedString[i], rs.getString(2)); + } + Assert.assertFalse(rs.next()); + } + statement.setFetchSize(0); + } + + @Test + public void testInstantResult() throws SQLException { + String[] expected = new String[]{ + "default_catalog", "cat1", "cat2"}; + Arrays.sort(expected); + + List actualList = new ArrayList<>(); + try (ResultSet rs = statement.executeQuery("SHOW CATALOGS")) { + for (int i = 0; i < expected.length; i++) { + Assert.assertTrue(rs.next()); + actualList.add(rs.getString(1)); + } + } + String[] actual = actualList.toArray(new String[0]); + Arrays.sort(actual); + + Assert.assertArrayEquals(expected, actual); + } + + @Test + public void testEmptyResult() throws SQLException { + try (ResultSet rs = statement.executeQuery("SELECT * FROM myTable WHERE a = -1")) { + Assert.assertFalse(rs.next()); + Assert.assertEquals(0, rs.getRow()); + } + } +} diff --git a/src/test/java/com/ververica/flink/table/jdbc/FlinkStatementTest.java b/src/test/java/com/ververica/flink/table/jdbc/FlinkStatementTest.java new file mode 100644 index 0000000..105436f --- /dev/null +++ b/src/test/java/com/ververica/flink/table/jdbc/FlinkStatementTest.java @@ -0,0 +1,291 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.ververica.flink.table.jdbc; + +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.File; +import java.net.InetSocketAddress; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Tests for {@link FlinkStatement}. + * + *

NOTE: Please clean up newly added tables and databases after each test, + * set current catalog back to default_catalog and set current database back to default_database. + */ +public class FlinkStatementTest { + + private static FlinkJdbcDriverTestingGateway gateway; + private static Connection connection; + private Statement statement; + + @BeforeClass + public static void beforeClass() throws Exception { + gateway = new FlinkJdbcDriverTestingGateway(); + gateway.start(); + + InetSocketAddress addr = gateway.getServerAddress(); + Assert.assertNotNull(addr); + connection = new FlinkConnection("jdbc:flink://localhost:" + addr.getPort() + "?planner=blink"); + Assert.assertTrue(connection.isValid(0)); + } + + @AfterClass + public static void afterClass() throws Exception { + connection.close(); + gateway.stop(); + } + + @Before + public void before() throws SQLException { + statement = connection.createStatement(); + } + + @After + public void after() throws SQLException { + statement.close(); + } + + @Test + public void testExecuteQuery() throws SQLException { + try (ResultSet rs = statement.executeQuery("SELECT * FROM myTable ORDER BY a, b LIMIT 2")) { + Assert.assertTrue(rs.next()); + Assert.assertEquals(22, rs.getInt(1)); + Assert.assertEquals("BBB Hi", rs.getString(2)); + + Assert.assertTrue(rs.next()); + Assert.assertEquals(32, rs.getInt(1)); + Assert.assertEquals("CCC World", rs.getString(2)); + + Assert.assertFalse(rs.next()); + } + } + + @Test + public void testExecuteUpdate() throws Exception { + File tmpFile = File.createTempFile("flink-jdbc-driver-test", ".csv"); + tmpFile.deleteOnExit(); + + int createTableUpdateCount = statement.executeUpdate( + "CREATE TABLE testTable(" + + " fa INT," + + " fb VARCHAR(100)" + + ") WITH (" + + " 'connector.type'='filesystem'," + + " 'connector.path'='file://" + tmpFile.getPath() + "'," + + " 'format.type' = 'csv')"); + // CREATE TABLE is a DDL, according to JDBC Java doc it's update count is 0 + Assert.assertEquals(0, createTableUpdateCount); + + int insertUpdateCount = statement.executeUpdate( + "INSERT INTO testTable VALUES (1, 'stra'), (2, 'strb')"); + // TODO change this when gateway supports real update count + Assert.assertEquals(Statement.SUCCESS_NO_INFO, insertUpdateCount); + + try (ResultSet rs = statement.executeQuery("SELECT * FROM testTable ORDER BY fa")) { + Assert.assertTrue(rs.next()); + Assert.assertEquals(1, rs.getInt("fa")); + Assert.assertEquals("stra", rs.getString("fb")); + + Assert.assertTrue(rs.next()); + Assert.assertEquals(2, rs.getInt("fa")); + Assert.assertEquals("strb", rs.getString("fb")); + + Assert.assertFalse(rs.next()); + } + + int dropTableUpdateCount = statement.executeUpdate("DROP TABLE testTable"); + // DROP TABLE is a DDL, according to JDBC Java doc it's update count is 0 + Assert.assertEquals(0, dropTableUpdateCount); + } + + @Test + public void testMultipleStatements() throws Exception { + File tmpFile1 = File.createTempFile("flink-jdbc-driver-test", ".csv"); + File tmpFile2 = File.createTempFile("flink-jdbc-driver-test", ".csv"); + tmpFile1.deleteOnExit(); + tmpFile2.deleteOnExit(); + + boolean executeIsQuery = statement.execute("CREATE TABLE testTable1(" + + " fa INT," + + " fb VARCHAR(100)" + + ") WITH (" + + " 'connector.type'='filesystem'," + + " 'connector.path'='file://" + tmpFile1.getPath() + "'," + + " 'format.type' = 'csv');" + + "INSERT INTO testTable1 VALUES (1, 'stra'), (2, 'strb');" + + "SELECT * FROM testTable1 ORDER BY fa;" + + + "CREATE TABLE testTable2(" + + " fc INT," + + " fd VARCHAR(100)" + + ") WITH (" + + " 'connector.type'='filesystem'," + + " 'connector.path'='file://" + tmpFile2.getPath() + "'," + + " 'format.type' = 'csv');" + + "INSERT INTO testTable2(fc, fd) SELECT * FROM testTable1;" + + "SELECT * FROM testTable2 ORDER BY fc;" + + + "DROP TABLE testTable1;" + + "DROP TABLE testTable2;"); + + Assert.assertFalse(executeIsQuery); + // CREATE TABLE is a DDL, according to JDBC Java doc it's update count is 0 + Assert.assertEquals(0, statement.getUpdateCount()); + + Assert.assertFalse(statement.getMoreResults()); + // TODO change this when gateway supports real update count + Assert.assertEquals(Statement.SUCCESS_NO_INFO, statement.getUpdateCount()); + + Assert.assertTrue(statement.getMoreResults()); + ResultSet rs1 = statement.getResultSet(); + Assert.assertTrue(rs1.next()); + Assert.assertEquals(1, rs1.getInt("fa")); + Assert.assertEquals("stra", rs1.getString("fb")); + Assert.assertTrue(rs1.next()); + Assert.assertEquals(2, rs1.getInt("fa")); + Assert.assertEquals("strb", rs1.getString("fb")); + Assert.assertFalse(rs1.next()); + + Assert.assertFalse(statement.getMoreResults()); + // CREATE TABLE is a DDL, according to JDBC Java doc it's update count is 0 + Assert.assertEquals(0, statement.getUpdateCount()); + + Assert.assertFalse(statement.getMoreResults()); + // TODO change this when gateway supports real update count + Assert.assertEquals(Statement.SUCCESS_NO_INFO, statement.getUpdateCount()); + + Assert.assertTrue(statement.getMoreResults()); + ResultSet rs2 = statement.getResultSet(); + Assert.assertTrue(rs2.next()); + Assert.assertEquals(1, rs2.getInt("fc")); + Assert.assertEquals("stra", rs2.getString("fd")); + Assert.assertTrue(rs2.next()); + Assert.assertEquals(2, rs2.getInt("fc")); + Assert.assertEquals("strb", rs2.getString("fd")); + Assert.assertFalse(rs2.next()); + + Assert.assertFalse(statement.getMoreResults()); + // DROP TABLE is a DDL, according to JDBC Java doc it's update count is 0 + Assert.assertEquals(0, statement.getUpdateCount()); + + Assert.assertFalse(statement.getMoreResults()); + // DROP TABLE is a DDL, according to JDBC Java doc it's update count is 0 + Assert.assertEquals(0, statement.getUpdateCount()); + + Assert.assertFalse(statement.getMoreResults()); + Assert.assertEquals(-1, statement.getUpdateCount()); + } + + @Test + public void testShows() throws Exception { + compareStringResultsWithSorting( + new String[]{"default_catalog", "cat1", "cat2"}, statement.executeQuery("SHOW CATALOGS")); + + statement.execute("USE CATALOG cat1"); + statement.execute("CREATE DATABASE db12"); + compareStringResultsWithSorting( + new String[]{"db11", "db12"}, statement.executeQuery("SHOW DATABASES")); + + statement.execute("USE db11"); + compareStringResultsWithSorting(new String[]{"cat1"}, statement.executeQuery("SHOW CURRENT CATALOG")); + compareStringResultsWithSorting(new String[]{"db11"}, statement.executeQuery("SHOW CURRENT DATABASE")); + + File tmpFile1 = File.createTempFile("flink-jdbc-driver-test", ".csv"); + File tmpFile2 = File.createTempFile("flink-jdbc-driver-test", ".csv"); + tmpFile1.deleteOnExit(); + tmpFile2.deleteOnExit(); + + statement.executeUpdate("CREATE TABLE testTable1(" + + " fa INT," + + " fb VARCHAR(100)" + + ") WITH (" + + " 'connector.type'='filesystem'," + + " 'connector.path'='file://" + tmpFile1.getPath() + "'," + + " 'format.type' = 'csv');"); + statement.executeUpdate("CREATE TABLE testTable2(" + + " fc INT," + + " fd VARCHAR(100)" + + ") WITH (" + + " 'connector.type'='filesystem'," + + " 'connector.path'='file://" + tmpFile2.getPath() + "'," + + " 'format.type' = 'csv');"); + compareStringResultsWithSorting( + new String[]{"testTable1", "testTable2"}, statement.executeQuery("SHOW TABLES")); + + statement.executeUpdate("DROP TABLE testTable1"); + statement.executeUpdate("DROP TABLE testTable2"); + statement.executeUpdate("DROP DATABASE db12"); + statement.executeUpdate("USE CATALOG default_catalog"); + } + + @Test + public void testMaxRows() throws SQLException { + // max rows is smaller than actual result count + statement.setMaxRows(2); + try (ResultSet rs = statement.executeQuery("SELECT * FROM myTable ORDER BY a, b")) { + Assert.assertTrue(rs.next()); + Assert.assertEquals(22, rs.getInt(1)); + Assert.assertEquals("BBB Hi", rs.getString(2)); + Assert.assertTrue(rs.next()); + Assert.assertEquals(32, rs.getInt(1)); + Assert.assertEquals("CCC World", rs.getString(2)); + Assert.assertFalse(rs.next()); + } + + // max rows is larger than actual result count + statement.setMaxRows(5); + try (ResultSet rs = statement.executeQuery("SELECT * FROM myTable ORDER BY a, b LIMIT 2")) { + Assert.assertTrue(rs.next()); + Assert.assertEquals(22, rs.getInt(1)); + Assert.assertEquals("BBB Hi", rs.getString(2)); + Assert.assertTrue(rs.next()); + Assert.assertEquals(32, rs.getInt(1)); + Assert.assertEquals("CCC World", rs.getString(2)); + Assert.assertFalse(rs.next()); + } + } + + private void compareStringResultsWithSorting(String[] expected, ResultSet actualResultSet) throws SQLException { + Arrays.sort(expected); + + List actualList = new ArrayList<>(); + for (int i = 0; i < expected.length; i++) { + Assert.assertTrue(actualResultSet.next()); + actualList.add(actualResultSet.getString(1)); + } + String[] actual = actualList.toArray(new String[0]); + Arrays.sort(actual); + + Assert.assertArrayEquals(expected, actual); + } +} diff --git a/src/test/resources/default-env.yaml b/src/test/resources/default-env.yaml new file mode 100644 index 0000000..cc322f5 --- /dev/null +++ b/src/test/resources/default-env.yaml @@ -0,0 +1,61 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +tables: + - name: myTable + type: source-table + update-mode: append + schema: + - name: a + data-type: INT + - name: b + data-type: VARCHAR(100) + connector: + type: filesystem + path: "$VAR_SOURCE_PATH" + format: + type: csv + fields: + - name: a + data-type: INT + - name: b + data-type: VARCHAR(100) + line-delimiter: "\n" + comment-prefix: "#" + +functions: [] + +catalogs: + - name: cat1 + type: generic_in_memory + default-database: db11 + - name: cat2 + type: generic_in_memory + default-database: db21 + +execution: + planner: blink + type: batch + result-mode: table + current-catalog: default_catalog + current-database: default_database + +deployment: + response-timeout: 5000 + gateway-address: "" + gateway-port: 0 diff --git a/src/test/resources/test-data.csv b/src/test/resources/test-data.csv new file mode 100644 index 0000000..e5c4a5f --- /dev/null +++ b/src/test/resources/test-data.csv @@ -0,0 +1,23 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +42,AAA Hello +22,BBB Hi +32,CCC World +32,DDD Hello!!!! +42,EEE Hi!!!! +52,FFF World!!!! diff --git a/tools/maven/checkstyle.xml b/tools/maven/checkstyle.xml new file mode 100644 index 0000000..c7fd410 --- /dev/null +++ b/tools/maven/checkstyle.xml @@ -0,0 +1,571 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/tools/maven/suppressions.xml b/tools/maven/suppressions.xml new file mode 100644 index 0000000..4f80ec2 --- /dev/null +++ b/tools/maven/suppressions.xml @@ -0,0 +1,26 @@ + + + + + + +