├── .gitignore ├── LICENSE ├── README.md ├── pom.xml ├── src ├── main │ ├── java │ │ └── com │ │ │ └── ververica │ │ │ └── flink │ │ │ └── table │ │ │ └── jdbc │ │ │ ├── FlinkConnection.java │ │ │ ├── FlinkDatabaseMetaData.java │ │ │ ├── FlinkDriver.java │ │ │ ├── FlinkJdbcUtils.java │ │ │ ├── FlinkResultSet.java │ │ │ ├── FlinkResultSetMetaData.java │ │ │ ├── FlinkStatement.java │ │ │ ├── rest │ │ │ ├── RestUtils.java │ │ │ └── SessionClient.java │ │ │ ├── resulthandler │ │ │ ├── DefaultResultHandler.java │ │ │ ├── DescribeResultHandler.java │ │ │ ├── ResultHandler.java │ │ │ └── ResultHandlerFactory.java │ │ │ └── type │ │ │ ├── FlinkSqlType.java │ │ │ └── FlinkSqlTypes.java │ └── resources │ │ └── META-INF │ │ └── services │ │ └── java.sql.Driver └── test │ ├── java │ └── com │ │ └── ververica │ │ └── flink │ │ └── table │ │ └── jdbc │ │ ├── FlinkConnectionTest.java │ │ ├── FlinkDatabaseMetaDataTest.java │ │ ├── FlinkJdbcDriverTestingGateway.java │ │ ├── FlinkResultSetTest.java │ │ └── FlinkStatementTest.java │ └── resources │ ├── default-env.yaml │ └── test-data.csv └── tools └── maven ├── checkstyle.xml └── suppressions.xml /.gitignore: -------------------------------------------------------------------------------- 1 | # Compiled class file 2 | *.class 3 | 4 | # Log file 5 | *.log 6 | 7 | # BlueJ files 8 | *.ctxt 9 | 10 | # Mobile Tools for Java (J2ME) 11 | .mtj.tmp/ 12 | 13 | # Package Files # 14 | *.jar 15 | *.war 16 | *.nar 17 | *.ear 18 | *.zip 19 | *.tar.gz 20 | *.rar 21 | 22 | # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml 23 | hs_err_pid* 24 | 25 | # IntelliJ IDEA Files 26 | .idea 27 | *.iml 28 | 29 | # Maven Files 30 | target 31 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Apache License 2 | Version 2.0, January 2004 3 | http://www.apache.org/licenses/ 4 | 5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 6 | 7 | 1. Definitions. 8 | 9 | "License" shall mean the terms and conditions for use, reproduction, 10 | and distribution as defined by Sections 1 through 9 of this document. 11 | 12 | "Licensor" shall mean the copyright owner or entity authorized by 13 | the copyright owner that is granting the License. 14 | 15 | "Legal Entity" shall mean the union of the acting entity and all 16 | other entities that control, are controlled by, or are under common 17 | control with that entity. For the purposes of this definition, 18 | "control" means (i) the power, direct or indirect, to cause the 19 | direction or management of such entity, whether by contract or 20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 21 | outstanding shares, or (iii) beneficial ownership of such entity. 22 | 23 | "You" (or "Your") shall mean an individual or Legal Entity 24 | exercising permissions granted by this License. 25 | 26 | "Source" form shall mean the preferred form for making modifications, 27 | including but not limited to software source code, documentation 28 | source, and configuration files. 29 | 30 | "Object" form shall mean any form resulting from mechanical 31 | transformation or translation of a Source form, including but 32 | not limited to compiled object code, generated documentation, 33 | and conversions to other media types. 34 | 35 | "Work" shall mean the work of authorship, whether in Source or 36 | Object form, made available under the License, as indicated by a 37 | copyright notice that is included in or attached to the work 38 | (an example is provided in the Appendix below). 39 | 40 | "Derivative Works" shall mean any work, whether in Source or Object 41 | form, that is based on (or derived from) the Work and for which the 42 | editorial revisions, annotations, elaborations, or other modifications 43 | represent, as a whole, an original work of authorship. For the purposes 44 | of this License, Derivative Works shall not include works that remain 45 | separable from, or merely link (or bind by name) to the interfaces of, 46 | the Work and Derivative Works thereof. 47 | 48 | "Contribution" shall mean any work of authorship, including 49 | the original version of the Work and any modifications or additions 50 | to that Work or Derivative Works thereof, that is intentionally 51 | submitted to Licensor for inclusion in the Work by the copyright owner 52 | or by an individual or Legal Entity authorized to submit on behalf of 53 | the copyright owner. For the purposes of this definition, "submitted" 54 | means any form of electronic, verbal, or written communication sent 55 | to the Licensor or its representatives, including but not limited to 56 | communication on electronic mailing lists, source code control systems, 57 | and issue tracking systems that are managed by, or on behalf of, the 58 | Licensor for the purpose of discussing and improving the Work, but 59 | excluding communication that is conspicuously marked or otherwise 60 | designated in writing by the copyright owner as "Not a Contribution." 61 | 62 | "Contributor" shall mean Licensor and any individual or Legal Entity 63 | on behalf of whom a Contribution has been received by Licensor and 64 | subsequently incorporated within the Work. 65 | 66 | 2. Grant of Copyright License. Subject to the terms and conditions of 67 | this License, each Contributor hereby grants to You a perpetual, 68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 69 | copyright license to reproduce, prepare Derivative Works of, 70 | publicly display, publicly perform, sublicense, and distribute the 71 | Work and such Derivative Works in Source or Object form. 72 | 73 | 3. Grant of Patent License. Subject to the terms and conditions of 74 | this License, each Contributor hereby grants to You a perpetual, 75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 76 | (except as stated in this section) patent license to make, have made, 77 | use, offer to sell, sell, import, and otherwise transfer the Work, 78 | where such license applies only to those patent claims licensable 79 | by such Contributor that are necessarily infringed by their 80 | Contribution(s) alone or by combination of their Contribution(s) 81 | with the Work to which such Contribution(s) was submitted. If You 82 | institute patent litigation against any entity (including a 83 | cross-claim or counterclaim in a lawsuit) alleging that the Work 84 | or a Contribution incorporated within the Work constitutes direct 85 | or contributory patent infringement, then any patent licenses 86 | granted to You under this License for that Work shall terminate 87 | as of the date such litigation is filed. 88 | 89 | 4. Redistribution. You may reproduce and distribute copies of the 90 | Work or Derivative Works thereof in any medium, with or without 91 | modifications, and in Source or Object form, provided that You 92 | meet the following conditions: 93 | 94 | (a) You must give any other recipients of the Work or 95 | Derivative Works a copy of this License; and 96 | 97 | (b) You must cause any modified files to carry prominent notices 98 | stating that You changed the files; and 99 | 100 | (c) You must retain, in the Source form of any Derivative Works 101 | that You distribute, all copyright, patent, trademark, and 102 | attribution notices from the Source form of the Work, 103 | excluding those notices that do not pertain to any part of 104 | the Derivative Works; and 105 | 106 | (d) If the Work includes a "NOTICE" text file as part of its 107 | distribution, then any Derivative Works that You distribute must 108 | include a readable copy of the attribution notices contained 109 | within such NOTICE file, excluding those notices that do not 110 | pertain to any part of the Derivative Works, in at least one 111 | of the following places: within a NOTICE text file distributed 112 | as part of the Derivative Works; within the Source form or 113 | documentation, if provided along with the Derivative Works; or, 114 | within a display generated by the Derivative Works, if and 115 | wherever such third-party notices normally appear. The contents 116 | of the NOTICE file are for informational purposes only and 117 | do not modify the License. You may add Your own attribution 118 | notices within Derivative Works that You distribute, alongside 119 | or as an addendum to the NOTICE text from the Work, provided 120 | that such additional attribution notices cannot be construed 121 | as modifying the License. 122 | 123 | You may add Your own copyright statement to Your modifications and 124 | may provide additional or different license terms and conditions 125 | for use, reproduction, or distribution of Your modifications, or 126 | for any such Derivative Works as a whole, provided Your use, 127 | reproduction, and distribution of the Work otherwise complies with 128 | the conditions stated in this License. 129 | 130 | 5. Submission of Contributions. Unless You explicitly state otherwise, 131 | any Contribution intentionally submitted for inclusion in the Work 132 | by You to the Licensor shall be under the terms and conditions of 133 | this License, without any additional terms or conditions. 134 | Notwithstanding the above, nothing herein shall supersede or modify 135 | the terms of any separate license agreement you may have executed 136 | with Licensor regarding such Contributions. 137 | 138 | 6. Trademarks. This License does not grant permission to use the trade 139 | names, trademarks, service marks, or product names of the Licensor, 140 | except as required for reasonable and customary use in describing the 141 | origin of the Work and reproducing the content of the NOTICE file. 142 | 143 | 7. Disclaimer of Warranty. Unless required by applicable law or 144 | agreed to in writing, Licensor provides the Work (and each 145 | Contributor provides its Contributions) on an "AS IS" BASIS, 146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 147 | implied, including, without limitation, any warranties or conditions 148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 149 | PARTICULAR PURPOSE. You are solely responsible for determining the 150 | appropriateness of using or redistributing the Work and assume any 151 | risks associated with Your exercise of permissions under this License. 152 | 153 | 8. Limitation of Liability. In no event and under no legal theory, 154 | whether in tort (including negligence), contract, or otherwise, 155 | unless required by applicable law (such as deliberate and grossly 156 | negligent acts) or agreed to in writing, shall any Contributor be 157 | liable to You for damages, including any direct, indirect, special, 158 | incidental, or consequential damages of any character arising as a 159 | result of this License or out of the use or inability to use the 160 | Work (including but not limited to damages for loss of goodwill, 161 | work stoppage, computer failure or malfunction, or any and all 162 | other commercial damages or losses), even if such Contributor 163 | has been advised of the possibility of such damages. 164 | 165 | 9. Accepting Warranty or Additional Liability. While redistributing 166 | the Work or Derivative Works thereof, You may choose to offer, 167 | and charge a fee for, acceptance of support, warranty, indemnity, 168 | or other liability obligations and/or rights consistent with this 169 | License. However, in accepting such obligations, You may act only 170 | on Your own behalf and on Your sole responsibility, not on behalf 171 | of any other Contributor, and only if You agree to indemnify, 172 | defend, and hold each Contributor harmless for any liability 173 | incurred by, or claims asserted against, such Contributor by reason 174 | of your accepting any such warranty or additional liability. 175 | 176 | END OF TERMS AND CONDITIONS 177 | 178 | APPENDIX: How to apply the Apache License to your work. 179 | 180 | To apply the Apache License to your work, attach the following 181 | boilerplate notice, with the fields enclosed by brackets "[]" 182 | replaced with your own identifying information. (Don't include 183 | the brackets!) The text should be enclosed in the appropriate 184 | comment syntax for the file format. We also recommend that a 185 | file or class name and description of purpose be included on the 186 | same "printed page" as the copyright notice for easier 187 | identification within third-party archives. 188 | 189 | Copyright [yyyy] [name of copyright owner] 190 | 191 | Licensed under the Apache License, Version 2.0 (the "License"); 192 | you may not use this file except in compliance with the License. 193 | You may obtain a copy of the License at 194 | 195 | http://www.apache.org/licenses/LICENSE-2.0 196 | 197 | Unless required by applicable law or agreed to in writing, software 198 | distributed under the License is distributed on an "AS IS" BASIS, 199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 200 | See the License for the specific language governing permissions and 201 | limitations under the License. 202 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Flink JDBC Driver 2 | 3 | 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. 4 | 5 | This project is at an early stage. Feel free to file an issue if you meet any problems or have any suggestions. 6 | 7 | # Usage 8 | 9 | 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. 10 | 11 | ## Use with a JDBC Tool 12 | ### Use with Beeline 13 | 14 | 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). 15 | 16 | 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`. 17 | 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. 18 | ``` 19 | beeline> !connect jdbc:flink://localhost:8083?planner=blink 20 | ``` 21 | 3. Execute any statement you want. 22 | 23 | **Sample Commands** 24 | ``` 25 | Beeline version 2.2.0 by Apache Hive 26 | beeline> !connect jdbc:flink://localhost:8083?planner=blink 27 | Connecting to jdbc:flink://localhost:8083?planner=blink 28 | Enter username for jdbc:flink://localhost:8083?planner=blink: 29 | Enter password for jdbc:flink://localhost:8083?planner=blink: 30 | Connected to: Apache Flink (version 1.10.0) 31 | Driver: Flink Driver (version 0.1) 32 | Transaction isolation: TRANSACTION_REPEATABLE_READ 33 | 0: jdbc:flink://localhost:8083> CREATE TABLE T( 34 | . . . . . . . . . . . . . . . > a INT, 35 | . . . . . . . . . . . . . . . > b VARCHAR(10) 36 | . . . . . . . . . . . . . . . > ) WITH ( 37 | . . . . . . . . . . . . . . . > 'connector.type' = 'filesystem', 38 | . . . . . . . . . . . . . . . > 'connector.path' = 'file:///tmp/T.csv', 39 | . . . . . . . . . . . . . . . > 'format.type' = 'csv', 40 | . . . . . . . . . . . . . . . > 'format.derive-schema' = 'true' 41 | . . . . . . . . . . . . . . . > ); 42 | No rows affected (0.158 seconds) 43 | 0: jdbc:flink://localhost:8083> INSERT INTO T VALUES (1, 'Hi'), (2, 'Hello'); 44 | No rows affected (4.747 seconds) 45 | 0: jdbc:flink://localhost:8083> SELECT * FROM T; 46 | +----+--------+--+ 47 | | a | b | 48 | +----+--------+--+ 49 | | 1 | Hi | 50 | | 2 | Hello | 51 | +----+--------+--+ 52 | 2 rows selected (0.994 seconds) 53 | 0: jdbc:flink://localhost:8083> 54 | ``` 55 | 56 | ### Use with Tableau 57 | [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). 58 | 59 | 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. 60 | * Windows: `C:\Program Files\Tableau\Drivers` 61 | * Mac: `~/Library/Tableau/Drivers` 62 | * Linux: `/opt/tableau/tableau_driver/jdbc` 63 | 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. 64 | 3. Hit *Login* button and use Tableau as usual. 65 | 66 | ### Use with other JDBC Tools 67 | 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. 68 | 69 | ## Use with Java 70 | 71 | 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). 72 | 73 | 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. 74 | 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. 75 | 3. Execute any statement you want. 76 | 77 | **Sample.java** 78 | ```java 79 | import java.sql.Connection; 80 | import java.sql.DriverManager; 81 | import java.sql.ResultSet; 82 | import java.sql.Statement; 83 | 84 | public class Sample { 85 | public static void main(String[] args) throws Exception { 86 | Connection connection = DriverManager.getConnection("jdbc:flink://localhost:8083?planner=blink"); 87 | Statement statement = connection.createStatement(); 88 | 89 | statement.executeUpdate("CREATE TABLE T(\n" + 90 | " a INT,\n" + 91 | " b VARCHAR(10)\n" + 92 | ") WITH (\n" + 93 | " 'connector.type' = 'filesystem',\n" + 94 | " 'connector.path' = 'file:///tmp/T.csv',\n" + 95 | " 'format.type' = 'csv',\n" + 96 | " 'format.derive-schema' = 'true'\n" + 97 | ")"); 98 | statement.executeUpdate("INSERT INTO T VALUES (1, 'Hi'), (2, 'Hello')"); 99 | ResultSet rs = statement.executeQuery("SELECT * FROM T"); 100 | while (rs.next()) { 101 | System.out.println(rs.getInt(1) + ", " + rs.getString(2)); 102 | } 103 | 104 | statement.close(); 105 | connection.close(); 106 | } 107 | } 108 | ``` 109 | 110 | **Output** 111 | ``` 112 | 1, Hi 113 | 2, Hello 114 | ``` 115 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 18 | 21 | 22 | 4.0.0 23 | 24 | flink-jdbc-driver 25 | com.ververica 26 | flink-jdbc-driver 27 | 0.1-SNAPSHOT 28 | 29 | jar 30 | 31 | 32 | 1.10.0 33 | 0.1-SNAPSHOT 34 | 1.8 35 | 1.7.15 36 | 2.11 37 | 38 | 41 | ${java.version} 42 | ${java.version} 43 | 44 | 47 | 1C 48 | 50 | ${flink.forkCount} 51 | true 52 | 53 | 54 | 55 | 56 | 57 | 58 | org.apache.flink 59 | flink-runtime_${scala.binary.version} 60 | ${flink.version} 61 | 62 | 63 | org.apache.flink 64 | flink-table-common 65 | ${flink.version} 66 | 67 | 68 | com.ververica 69 | flink-sql-gateway 70 | ${flink.sql.gateway.version} 71 | 72 | 73 | org.slf4j 74 | slf4j-nop 75 | ${slf4j.version} 76 | runtime 77 | 78 | 79 | 80 | 81 | 82 | org.apache.flink 83 | flink-table-api-java 84 | ${flink.version} 85 | test 86 | 87 | 88 | org.apache.flink 89 | flink-table-planner-blink_${scala.binary.version} 90 | ${flink.version} 91 | test 92 | 93 | 94 | org.apache.flink 95 | flink-test-utils-junit 96 | ${flink.version} 97 | test 98 | 99 | 100 | org.apache.flink 101 | flink-test-utils_${scala.binary.version} 102 | ${flink.version} 103 | test 104 | 105 | 106 | 107 | 108 | 109 | 110 | org.apache.maven.plugins 111 | maven-surefire-plugin 112 | 2.22.1 113 | 114 | ${test.groups} 115 | ${flink.forkCount} 116 | ${flink.reuseForks} 117 | 118 | 0${surefire.forkNumber} 119 | ${test.scheduler.type} 120 | 121 | -Xms256m -Xmx2048m -Dmvn.forkNumber=${surefire.forkNumber} -XX:+UseG1GC 122 | 123 | 124 | 125 | 126 | default-test 127 | test 128 | 129 | test 130 | 131 | 132 | 133 | **/*Test.* 134 | 135 | 136 | 137 | 138 | 139 | integration-tests 140 | integration-test 141 | 142 | test 143 | 144 | 145 | 146 | **/*ITCase.* 147 | 148 | false 149 | 150 | 151 | 152 | 153 | 154 | org.apache.maven.plugins 155 | maven-shade-plugin 156 | 157 | 158 | shade-flink 159 | package 160 | 161 | shade 162 | 163 | 164 | false 165 | 166 | 167 | org.slf4j:slf4j-api 168 | org.slf4j:slf4j-nop 169 | org.apache.flink:flink-shaded-asm-7 170 | org.apache.flink:flink-shaded-jackson 171 | org.apache.flink:flink-shaded-netty 172 | org.apache.flink:flink-annotations 173 | org.apache.flink:flink-metrics-core 174 | org.apache.flink:flink-core 175 | org.apache.flink:flink-runtime_${scala.binary.version} 176 | org.apache.flink:flink-table-common 177 | com.ververica:flink-sql-gateway 178 | 179 | 180 | 181 | 182 | *:* 183 | 184 | ** 185 | 186 | 187 | 188 | org.apache.flink:flink-runtime_${scala.binary.version} 189 | 190 | org/apache/flink/runtime/rest/** 191 | org/apache/flink/runtime/net/** 192 | org/apache/flink/runtime/io/network/netty/** 193 | org/apache/flink/runtime/util/** 194 | 195 | 196 | 197 | com.ververica:flink-sql-gateway 198 | 199 | com/ververica/flink/table/gateway/rest/** 200 | 201 | 202 | 203 | 204 | 205 | 206 | 207 | 208 | 209 | org.apache.rat 210 | apache-rat-plugin 211 | 0.12 212 | false 213 | 214 | 215 | verify 216 | 217 | check 218 | 219 | 220 | 221 | 222 | false 223 | 0 224 | 225 | 241 | 242 | AL2 243 | Apache License 2.0 244 | 245 | 246 | Licensed to the Apache Software Foundation (ASF) under one 247 | 248 | 249 | 250 | 251 | 252 | Apache License 2.0 253 | 254 | 255 | 256 | 257 | **/.*/** 258 | **/*.prefs 259 | **/*.log 260 | 261 | **/README.md 262 | .github/** 263 | 264 | **/target/** 265 | 266 | **/.idea/** 267 | 268 | 269 | 270 | 271 | 272 | org.apache.maven.plugins 273 | maven-checkstyle-plugin 274 | 2.17 275 | 276 | 277 | com.puppycrawl.tools 278 | checkstyle 279 | 280 | 8.14 281 | 282 | 283 | 284 | 285 | validate 286 | validate 287 | 288 | check 289 | 290 | 291 | 292 | 293 | /tools/maven/suppressions.xml 294 | true 295 | /tools/maven/checkstyle.xml 296 | true 297 | true 298 | 299 | 300 | 301 | 302 | 303 | 304 | -------------------------------------------------------------------------------- /src/main/java/com/ververica/flink/table/jdbc/FlinkConnection.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.ververica.flink.table.jdbc; 20 | 21 | import com.ververica.flink.table.gateway.rest.message.StatementExecuteResponseBody; 22 | import com.ververica.flink.table.gateway.rest.result.ResultSet; 23 | import com.ververica.flink.table.jdbc.rest.RestUtils; 24 | import com.ververica.flink.table.jdbc.rest.SessionClient; 25 | 26 | import org.apache.flink.api.common.JobID; 27 | import org.apache.flink.types.Either; 28 | import org.apache.flink.util.Preconditions; 29 | 30 | import java.sql.Array; 31 | import java.sql.Blob; 32 | import java.sql.CallableStatement; 33 | import java.sql.Clob; 34 | import java.sql.Connection; 35 | import java.sql.DatabaseMetaData; 36 | import java.sql.NClob; 37 | import java.sql.PreparedStatement; 38 | import java.sql.SQLClientInfoException; 39 | import java.sql.SQLException; 40 | import java.sql.SQLFeatureNotSupportedException; 41 | import java.sql.SQLWarning; 42 | import java.sql.SQLXML; 43 | import java.sql.Savepoint; 44 | import java.sql.Statement; 45 | import java.sql.Struct; 46 | import java.util.Map; 47 | import java.util.Properties; 48 | import java.util.concurrent.Executor; 49 | 50 | /** 51 | * Flink JDBC connection. 52 | */ 53 | public class FlinkConnection implements Connection { 54 | 55 | private final SessionClient session; 56 | private boolean closed; 57 | 58 | public FlinkConnection(String url) throws Exception { 59 | this.closed = false; 60 | this.session = createSession(url.substring(FlinkDriver.URL_PREFIX.length())); 61 | } 62 | 63 | @Override 64 | public Statement createStatement() throws SQLException { 65 | return new FlinkStatement(session, this); 66 | } 67 | 68 | @Override 69 | public void close() throws SQLException { 70 | if (closed) { 71 | return; 72 | } 73 | 74 | try { 75 | session.close(); 76 | } catch (Exception e) { 77 | throw new SQLException(e); 78 | } 79 | closed = true; 80 | } 81 | 82 | @Override 83 | public boolean isClosed() throws SQLException { 84 | return closed; 85 | } 86 | 87 | @Override 88 | public void setCatalog(String catalog) throws SQLException { 89 | session.submitStatement("USE CATALOG " + catalog); 90 | } 91 | 92 | @Override 93 | public String getCatalog() throws SQLException { 94 | StatementExecuteResponseBody response; 95 | response = session.submitStatement("SHOW CURRENT CATALOG"); 96 | Preconditions.checkArgument( 97 | response.getResults().size() == 1, 98 | "SHOW CURRENT CATALOG should return exactly one result set. This is a bug."); 99 | 100 | Either jobIdOrResultSet = 101 | RestUtils.getEitherJobIdOrResultSet(response.getResults().get(0)); 102 | Preconditions.checkArgument( 103 | jobIdOrResultSet.isRight(), 104 | "SHOW CURRENT CATALOG should immediately return a result. This is a bug."); 105 | 106 | ResultSet resultSet = jobIdOrResultSet.right(); 107 | Preconditions.checkArgument( 108 | resultSet.getData().size() == 1, 109 | "SHOW CURRENT CATALOG should return exactly one row of result. This is a bug."); 110 | 111 | return resultSet.getData().get(0).toString(); 112 | } 113 | 114 | @Override 115 | public DatabaseMetaData getMetaData() throws SQLException { 116 | return new FlinkDatabaseMetaData(session, this); 117 | } 118 | 119 | @Override 120 | public void setSchema(String schema) throws SQLException { 121 | session.submitStatement("USE " + schema); 122 | } 123 | 124 | @Override 125 | public String getSchema() throws SQLException { 126 | StatementExecuteResponseBody response; 127 | response = session.submitStatement("SHOW CURRENT DATABASE"); 128 | Preconditions.checkArgument( 129 | response.getResults().size() == 1, 130 | "SHOW CURRENT DATABASE should return exactly one result set. This is a bug."); 131 | 132 | Either jobIdOrResultSet = 133 | RestUtils.getEitherJobIdOrResultSet(response.getResults().get(0)); 134 | Preconditions.checkArgument( 135 | jobIdOrResultSet.isRight(), 136 | "SHOW CURRENT DATABASE should immediately return a result. This is a bug."); 137 | 138 | ResultSet resultSet = jobIdOrResultSet.right(); 139 | Preconditions.checkArgument( 140 | resultSet.getData().size() == 1, 141 | "SHOW CURRENT DATABASE should return exactly one row of result. This is a bug."); 142 | 143 | return resultSet.getData().get(0).toString(); 144 | } 145 | 146 | @Override 147 | public PreparedStatement prepareStatement(String sql) throws SQLException { 148 | throw new SQLFeatureNotSupportedException("FlinkConnection#prepareStatement is not supported"); 149 | } 150 | 151 | @Override 152 | public CallableStatement prepareCall(String sql) throws SQLException { 153 | throw new SQLFeatureNotSupportedException("FlinkConnection#prepareCall is not supported"); 154 | } 155 | 156 | @Override 157 | public String nativeSQL(String sql) throws SQLException { 158 | throw new SQLFeatureNotSupportedException("FlinkConnection#nativeSQL is not supported"); 159 | } 160 | 161 | @Override 162 | public void setAutoCommit(boolean autoCommit) throws SQLException { 163 | // TODO 164 | // we currently do not support this, 165 | // but we can't throw a SQLException because we want to support beeline 166 | } 167 | 168 | @Override 169 | public boolean getAutoCommit() throws SQLException { 170 | // TODO 171 | // we currently do not support this, 172 | // but we can't throw a SQLException because we want to support beeline 173 | return true; 174 | } 175 | 176 | @Override 177 | public void commit() throws SQLException { 178 | throw new SQLFeatureNotSupportedException("FlinkConnection#commit is not supported"); 179 | } 180 | 181 | @Override 182 | public void rollback() throws SQLException { 183 | throw new SQLFeatureNotSupportedException("FlinkConnection#rollback is not supported"); 184 | } 185 | 186 | @Override 187 | public void setReadOnly(boolean readOnly) throws SQLException { 188 | throw new SQLFeatureNotSupportedException("FlinkConnection#setReadOnly is not supported"); 189 | } 190 | 191 | @Override 192 | public boolean isReadOnly() throws SQLException { 193 | throw new SQLFeatureNotSupportedException("FlinkConnection#isReadOnly is not supported"); 194 | } 195 | 196 | @Override 197 | public void setTransactionIsolation(int level) throws SQLException { 198 | // TODO 199 | // we currently do not support this, 200 | // but we can't throw a SQLException because we want to support beeline 201 | } 202 | 203 | @Override 204 | public int getTransactionIsolation() throws SQLException { 205 | // TODO 206 | // we currently do not support this, 207 | // but we can't throw a SQLException because we want to support beeline 208 | return Connection.TRANSACTION_NONE; 209 | } 210 | 211 | @Override 212 | public SQLWarning getWarnings() throws SQLException { 213 | // TODO 214 | // we currently do not support this, 215 | // but we can't throw a SQLException because we want to support beeline 216 | return null; 217 | } 218 | 219 | @Override 220 | public void clearWarnings() throws SQLException { 221 | throw new SQLFeatureNotSupportedException("FlinkConnection#clearWarnings is not supported"); 222 | } 223 | 224 | @Override 225 | public Statement createStatement(int resultSetType, int resultSetConcurrency) throws SQLException { 226 | throw new SQLFeatureNotSupportedException("FlinkConnection#createStatement is not supported"); 227 | } 228 | 229 | @Override 230 | public PreparedStatement prepareStatement(String sql, int resultSetType, int resultSetConcurrency) throws SQLException { 231 | throw new SQLFeatureNotSupportedException("FlinkConnection#prepareStatement is not supported"); 232 | } 233 | 234 | @Override 235 | public CallableStatement prepareCall(String sql, int resultSetType, int resultSetConcurrency) throws SQLException { 236 | throw new SQLFeatureNotSupportedException("FlinkConnection#prepareCall is not supported"); 237 | } 238 | 239 | @Override 240 | public Map> getTypeMap() throws SQLException { 241 | throw new SQLFeatureNotSupportedException("FlinkConnection#getTypeMap is not supported"); 242 | } 243 | 244 | @Override 245 | public void setTypeMap(Map> map) throws SQLException { 246 | throw new SQLFeatureNotSupportedException("FlinkConnection#setTypeMap is not supported"); 247 | } 248 | 249 | @Override 250 | public void setHoldability(int holdability) throws SQLException { 251 | throw new SQLFeatureNotSupportedException("FlinkConnection#setHoldability is not supported"); 252 | } 253 | 254 | @Override 255 | public int getHoldability() throws SQLException { 256 | throw new SQLFeatureNotSupportedException("FlinkConnection#getHoldability is not supported"); 257 | } 258 | 259 | @Override 260 | public Savepoint setSavepoint() throws SQLException { 261 | throw new SQLFeatureNotSupportedException("FlinkConnection#setSavepoint is not supported"); 262 | } 263 | 264 | @Override 265 | public Savepoint setSavepoint(String name) throws SQLException { 266 | throw new SQLFeatureNotSupportedException("FlinkConnection#setSavepoint is not supported"); 267 | } 268 | 269 | @Override 270 | public void rollback(Savepoint savepoint) throws SQLException { 271 | throw new SQLFeatureNotSupportedException("FlinkConnection#rollback is not supported"); 272 | } 273 | 274 | @Override 275 | public void releaseSavepoint(Savepoint savepoint) throws SQLException { 276 | throw new SQLFeatureNotSupportedException("FlinkConnection#releaseSavepoint is not supported"); 277 | } 278 | 279 | @Override 280 | public Statement createStatement(int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException { 281 | throw new SQLFeatureNotSupportedException("FlinkConnection#createStatement is not supported"); 282 | } 283 | 284 | @Override 285 | public PreparedStatement prepareStatement(String sql, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException { 286 | throw new SQLFeatureNotSupportedException("FlinkConnection#prepareStatement is not supported"); 287 | } 288 | 289 | @Override 290 | public CallableStatement prepareCall(String sql, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException { 291 | throw new SQLFeatureNotSupportedException("FlinkConnection#prepareCall is not supported"); 292 | } 293 | 294 | @Override 295 | public PreparedStatement prepareStatement(String sql, int autoGeneratedKeys) throws SQLException { 296 | throw new SQLFeatureNotSupportedException("FlinkConnection#prepareStatement is not supported"); 297 | } 298 | 299 | @Override 300 | public PreparedStatement prepareStatement(String sql, int[] columnIndexes) throws SQLException { 301 | throw new SQLFeatureNotSupportedException("FlinkConnection#prepareStatement is not supported"); 302 | } 303 | 304 | @Override 305 | public PreparedStatement prepareStatement(String sql, String[] columnNames) throws SQLException { 306 | throw new SQLFeatureNotSupportedException("FlinkConnection#prepareStatement is not supported"); 307 | } 308 | 309 | @Override 310 | public Clob createClob() throws SQLException { 311 | throw new SQLFeatureNotSupportedException("FlinkConnection#createClob is not supported"); 312 | } 313 | 314 | @Override 315 | public Blob createBlob() throws SQLException { 316 | throw new SQLFeatureNotSupportedException("FlinkConnection#createBlob is not supported"); 317 | } 318 | 319 | @Override 320 | public NClob createNClob() throws SQLException { 321 | throw new SQLFeatureNotSupportedException("FlinkConnection#createNClob is not supported"); 322 | } 323 | 324 | @Override 325 | public SQLXML createSQLXML() throws SQLException { 326 | throw new SQLFeatureNotSupportedException("FlinkConnection#createSQLXML is not supported"); 327 | } 328 | 329 | @Override 330 | public boolean isValid(int timeout) throws SQLException { 331 | // TODO support timeout 332 | if (timeout < 0) { 333 | throw new SQLException("Timeout must not be negative"); 334 | } 335 | 336 | try { 337 | session.sendHeartbeat(); 338 | return true; 339 | } catch (SQLException e) { 340 | return false; 341 | } 342 | } 343 | 344 | @Override 345 | public void setClientInfo(String name, String value) throws SQLClientInfoException { 346 | throw new SQLClientInfoException(); 347 | } 348 | 349 | @Override 350 | public void setClientInfo(Properties properties) throws SQLClientInfoException { 351 | throw new SQLClientInfoException(); 352 | } 353 | 354 | @Override 355 | public String getClientInfo(String name) throws SQLException { 356 | throw new SQLFeatureNotSupportedException("FlinkConnection#getClientInfo is not supported"); 357 | } 358 | 359 | @Override 360 | public Properties getClientInfo() throws SQLException { 361 | throw new SQLFeatureNotSupportedException("FlinkConnection#getClientInfo is not supported"); 362 | } 363 | 364 | @Override 365 | public Array createArrayOf(String typeName, Object[] elements) throws SQLException { 366 | throw new SQLFeatureNotSupportedException("FlinkConnection#createArrayOf is not supported"); 367 | } 368 | 369 | @Override 370 | public Struct createStruct(String typeName, Object[] attributes) throws SQLException { 371 | throw new SQLFeatureNotSupportedException("FlinkConnection#createStruct is not supported"); 372 | } 373 | 374 | @Override 375 | public void abort(Executor executor) throws SQLException { 376 | throw new SQLFeatureNotSupportedException("FlinkConnection#abort is not supported"); 377 | } 378 | 379 | @Override 380 | public void setNetworkTimeout(Executor executor, int milliseconds) throws SQLException { 381 | throw new SQLFeatureNotSupportedException("FlinkConnection#setNetworkTimeout is not supported"); 382 | } 383 | 384 | @Override 385 | public int getNetworkTimeout() throws SQLException { 386 | throw new SQLFeatureNotSupportedException("FlinkConnection#getNetworkTimeout is not supported"); 387 | } 388 | 389 | @Override 390 | public T unwrap(Class iface) throws SQLException { 391 | throw new SQLFeatureNotSupportedException("FlinkConnection#unwrap is not supported"); 392 | } 393 | 394 | @Override 395 | public boolean isWrapperFor(Class iface) throws SQLException { 396 | throw new SQLFeatureNotSupportedException("FlinkConnection#isWrapperFor is not supported"); 397 | } 398 | 399 | private UrlInfo parseUrl(String url) { 400 | String neededParams = "These url parameters are needed: planner"; 401 | 402 | String host; 403 | int port; 404 | String planner = null; 405 | 406 | int argumentStart = url.indexOf('?'); 407 | if (argumentStart < 0) { 408 | throw new IllegalArgumentException(neededParams); 409 | } else { 410 | int colonPos = url.indexOf(':'); 411 | if (colonPos < 0) { 412 | throw new IllegalArgumentException("Cannot read port from string " + url); 413 | } else { 414 | host = url.substring(0, colonPos); 415 | try { 416 | port = Integer.valueOf(url.substring(colonPos + 1, argumentStart)); 417 | } catch (NumberFormatException e) { 418 | throw new IllegalArgumentException("Invalid port format"); 419 | } 420 | } 421 | } 422 | 423 | for (String kv : url.substring(argumentStart + 1).split("&")) { 424 | int equalPos = kv.indexOf('='); 425 | if (equalPos < 0) { 426 | throw new IllegalArgumentException("Invalid url parameter kv pair " + kv); 427 | } 428 | 429 | String key = kv.substring(0, equalPos); 430 | String value = kv.substring(equalPos + 1); 431 | 432 | if (key.equals("planner")) { 433 | planner = value; 434 | } else { 435 | throw new IllegalArgumentException("Unknown url parameter key " + key); 436 | } 437 | } 438 | 439 | if (planner == null) { 440 | throw new IllegalArgumentException(neededParams); 441 | } 442 | 443 | return new UrlInfo(host, port, planner); 444 | } 445 | 446 | private SessionClient createSession(String url) throws Exception { 447 | UrlInfo urlInfo = parseUrl(url); 448 | return new SessionClient(urlInfo.host, urlInfo.port, "Flink-JDBC", urlInfo.planner, "batch", "Flink-JDBC-Connection-IO"); 449 | } 450 | 451 | /** 452 | * Contents of Flink JDBC url. 453 | */ 454 | private static class UrlInfo { 455 | final String host; 456 | final int port; 457 | final String planner; 458 | 459 | UrlInfo(String host, int port, String planner) { 460 | this.host = host; 461 | this.port = port; 462 | this.planner = planner; 463 | } 464 | } 465 | } 466 | -------------------------------------------------------------------------------- /src/main/java/com/ververica/flink/table/jdbc/FlinkDriver.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.ververica.flink.table.jdbc; 20 | 21 | import java.sql.Connection; 22 | import java.sql.Driver; 23 | import java.sql.DriverPropertyInfo; 24 | import java.sql.SQLException; 25 | import java.sql.SQLFeatureNotSupportedException; 26 | import java.util.Properties; 27 | import java.util.logging.Logger; 28 | 29 | /** 30 | * Flink JDBC driver. 31 | */ 32 | public class FlinkDriver implements Driver { 33 | 34 | public static final String URL_PREFIX = "jdbc:flink://"; 35 | 36 | static { 37 | try { 38 | java.sql.DriverManager.registerDriver(new FlinkDriver()); 39 | } catch (SQLException e) { 40 | throw new RuntimeException(e); 41 | } 42 | } 43 | 44 | @Override 45 | public Connection connect(String url, Properties info) throws SQLException { 46 | try { 47 | return acceptsURL(url) ? new FlinkConnection(url) : null; 48 | } catch (Exception e) { 49 | throw new SQLException(e); 50 | } 51 | } 52 | 53 | @Override 54 | public boolean acceptsURL(String url) throws SQLException { 55 | return url.startsWith(URL_PREFIX); 56 | } 57 | 58 | @Override 59 | public DriverPropertyInfo[] getPropertyInfo(String url, Properties info) throws SQLException { 60 | return new DriverPropertyInfo[0]; 61 | } 62 | 63 | @Override 64 | public int getMajorVersion() { 65 | return Integer.valueOf(FlinkDatabaseMetaData.DRIVER_VERSION.split("\\.")[0]); 66 | } 67 | 68 | @Override 69 | public int getMinorVersion() { 70 | return Integer.valueOf(FlinkDatabaseMetaData.DRIVER_VERSION.split("\\.")[1]); 71 | } 72 | 73 | @Override 74 | public boolean jdbcCompliant() { 75 | return false; 76 | } 77 | 78 | @Override 79 | public Logger getParentLogger() throws SQLFeatureNotSupportedException { 80 | throw new SQLFeatureNotSupportedException("FlinkDriver#getParentLogger is not supported"); 81 | } 82 | } 83 | -------------------------------------------------------------------------------- /src/main/java/com/ververica/flink/table/jdbc/FlinkJdbcUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.ververica.flink.table.jdbc; 20 | 21 | import java.util.regex.Pattern; 22 | 23 | /** 24 | * Util class for Flink JDBC. 25 | */ 26 | public class FlinkJdbcUtils { 27 | 28 | public static Pattern sqlPatternToJavaPattern(String sqlPattern) { 29 | return Pattern.compile(sqlPattern 30 | .replace("%", ".*") 31 | .replace("_", ".?")); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /src/main/java/com/ververica/flink/table/jdbc/FlinkResultSetMetaData.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.ververica.flink.table.jdbc; 20 | 21 | import com.ververica.flink.table.gateway.rest.result.ColumnInfo; 22 | import com.ververica.flink.table.jdbc.type.FlinkSqlTypes; 23 | 24 | import java.sql.ResultSetMetaData; 25 | import java.sql.SQLException; 26 | import java.sql.SQLFeatureNotSupportedException; 27 | import java.util.List; 28 | 29 | /** 30 | * Fink JDBC result set meta data. 31 | */ 32 | public class FlinkResultSetMetaData implements ResultSetMetaData { 33 | 34 | private final List columns; 35 | 36 | public FlinkResultSetMetaData(List columns) { 37 | this.columns = columns; 38 | } 39 | 40 | @Override 41 | public int getColumnCount() throws SQLException { 42 | return columns.size(); 43 | } 44 | 45 | @Override 46 | public boolean isAutoIncrement(int column) throws SQLException { 47 | throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#isAutoIncrement is not supported"); 48 | } 49 | 50 | @Override 51 | public boolean isCaseSensitive(int column) throws SQLException { 52 | throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#isCaseSensitive is not supported"); 53 | } 54 | 55 | @Override 56 | public boolean isSearchable(int column) throws SQLException { 57 | checkIndexBound(column); 58 | return true; 59 | } 60 | 61 | @Override 62 | public boolean isCurrency(int column) throws SQLException { 63 | throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#isCurrency is not supported"); 64 | } 65 | 66 | @Override 67 | public int isNullable(int column) throws SQLException { 68 | checkIndexBound(column); 69 | return columns.get(column - 1).getLogicalType().isNullable() ? 70 | ResultSetMetaData.columnNullable : 71 | ResultSetMetaData.columnNoNulls; 72 | } 73 | 74 | @Override 75 | public boolean isSigned(int column) throws SQLException { 76 | throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#isSigned is not supported"); 77 | } 78 | 79 | @Override 80 | public int getColumnDisplaySize(int column) throws SQLException { 81 | checkIndexBound(column); 82 | return FlinkSqlTypes.getType(columns.get(column - 1).getLogicalType()).getDisplaySize(); 83 | } 84 | 85 | @Override 86 | public String getColumnLabel(int column) throws SQLException { 87 | return getColumnName(column); 88 | } 89 | 90 | @Override 91 | public String getColumnName(int column) throws SQLException { 92 | checkIndexBound(column); 93 | return columns.get(column - 1).getName(); 94 | } 95 | 96 | @Override 97 | public String getSchemaName(int column) throws SQLException { 98 | throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#getSchemaName is not supported"); 99 | } 100 | 101 | @Override 102 | public int getPrecision(int column) throws SQLException { 103 | checkIndexBound(column); 104 | return FlinkSqlTypes.getType(columns.get(column - 1).getLogicalType()).getPrecision(); 105 | } 106 | 107 | @Override 108 | public int getScale(int column) throws SQLException { 109 | checkIndexBound(column); 110 | return FlinkSqlTypes.getType(columns.get(column - 1).getLogicalType()).getScale(); 111 | } 112 | 113 | @Override 114 | public String getTableName(int column) throws SQLException { 115 | throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#getTableName is not supported"); 116 | } 117 | 118 | @Override 119 | public String getCatalogName(int column) throws SQLException { 120 | throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#getCatalogName is not supported"); 121 | } 122 | 123 | @Override 124 | public int getColumnType(int column) throws SQLException { 125 | checkIndexBound(column); 126 | return FlinkSqlTypes.getType(columns.get(column - 1).getLogicalType()).getSqlType(); 127 | } 128 | 129 | @Override 130 | public String getColumnTypeName(int column) throws SQLException { 131 | checkIndexBound(column); 132 | return columns.get(column - 1).getType(); 133 | } 134 | 135 | @Override 136 | public boolean isReadOnly(int column) throws SQLException { 137 | throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#isReadOnly is not supported"); 138 | } 139 | 140 | @Override 141 | public boolean isWritable(int column) throws SQLException { 142 | throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#isWritable is not supported"); 143 | } 144 | 145 | @Override 146 | public boolean isDefinitelyWritable(int column) throws SQLException { 147 | throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#isDefinitelyWritable is not supported"); 148 | } 149 | 150 | @Override 151 | public String getColumnClassName(int column) throws SQLException { 152 | checkIndexBound(column); 153 | return columns.get(column - 1).getLogicalType().getDefaultConversion().getName(); 154 | } 155 | 156 | @Override 157 | public T unwrap(Class iface) throws SQLException { 158 | throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#unwrap is not supported"); 159 | } 160 | 161 | @Override 162 | public boolean isWrapperFor(Class iface) throws SQLException { 163 | throw new SQLFeatureNotSupportedException("FlinkResultSetMetaData#isWrapperFor is not supported"); 164 | } 165 | 166 | private void checkIndexBound(int column) throws SQLException { 167 | int columnNum = columns.size(); 168 | if (column <= 0) { 169 | throw new SQLException("Column index must be positive."); 170 | } 171 | if (column > columnNum) { 172 | throw new SQLException( 173 | "Column index " + column + " out of bound. There are only " + columnNum + " columns."); 174 | } 175 | } 176 | } 177 | -------------------------------------------------------------------------------- /src/main/java/com/ververica/flink/table/jdbc/FlinkStatement.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.ververica.flink.table.jdbc; 20 | 21 | import com.ververica.flink.table.gateway.rest.message.StatementExecuteResponseBody; 22 | import com.ververica.flink.table.jdbc.rest.RestUtils; 23 | import com.ververica.flink.table.jdbc.rest.SessionClient; 24 | import com.ververica.flink.table.jdbc.resulthandler.ResultHandlerFactory; 25 | 26 | import org.apache.flink.api.common.JobID; 27 | import org.apache.flink.types.Either; 28 | import org.apache.flink.util.Preconditions; 29 | 30 | import java.sql.Connection; 31 | import java.sql.ResultSet; 32 | import java.sql.SQLException; 33 | import java.sql.SQLFeatureNotSupportedException; 34 | import java.sql.SQLWarning; 35 | import java.sql.Statement; 36 | import java.util.Arrays; 37 | import java.util.List; 38 | import java.util.concurrent.atomic.AtomicReference; 39 | import java.util.concurrent.locks.ReadWriteLock; 40 | import java.util.concurrent.locks.ReentrantReadWriteLock; 41 | 42 | /** 43 | * Flink JDBC statement. 44 | */ 45 | public class FlinkStatement implements Statement { 46 | 47 | private static final List QUERY_COMMANDS = Arrays.asList( 48 | "SELECT", 49 | "SHOW_MODULES", 50 | "SHOW_CATALOGS", 51 | "SHOW_CURRENT_CATALOG", 52 | "SHOW_DATABASES", 53 | "SHOW_CURRENT_DATABASE", 54 | "SHOW_TABLES", 55 | "SHOW_FUNCTIONS", 56 | "DESCRIBE", 57 | "EXPLAIN"); 58 | 59 | private final SessionClient session; 60 | private final FlinkConnection connection; 61 | 62 | private final AtomicReference currentStatements; 63 | 64 | private long maxRows; 65 | private int queryTimeout; 66 | private int fetchSize; 67 | 68 | private boolean closed; 69 | 70 | public FlinkStatement(SessionClient session, FlinkConnection connection) { 71 | this.session = session; 72 | this.connection = connection; 73 | 74 | this.currentStatements = new AtomicReference<>(); 75 | 76 | this.maxRows = 0; 77 | this.queryTimeout = 0; 78 | this.fetchSize = 0; 79 | 80 | this.closed = false; 81 | } 82 | 83 | @Override 84 | public ResultSet executeQuery(String sql) throws SQLException { 85 | boolean isQuery = execute(sql); 86 | if (!isQuery) { 87 | throw new SQLException(sql + " is not a query"); 88 | } 89 | return getResultSet(); 90 | } 91 | 92 | @Override 93 | public int executeUpdate(String sql) throws SQLException { 94 | return (int) executeLargeUpdate(sql); 95 | } 96 | 97 | @Override 98 | public void close() throws SQLException { 99 | if (closed) { 100 | return; 101 | } 102 | 103 | cancel(); 104 | closed = true; 105 | } 106 | 107 | @Override 108 | public int getMaxFieldSize() throws SQLException { 109 | throw new SQLFeatureNotSupportedException("FlinkStatement#getMaxFieldSize is not supported"); 110 | } 111 | 112 | @Override 113 | public void setMaxFieldSize(int max) throws SQLException { 114 | throw new SQLFeatureNotSupportedException("FlinkStatement#setMaxFieldSize is not supported"); 115 | } 116 | 117 | @Override 118 | public int getMaxRows() throws SQLException { 119 | return (int) getLargeMaxRows(); 120 | } 121 | 122 | @Override 123 | public void setMaxRows(int max) throws SQLException { 124 | setLargeMaxRows(max); 125 | } 126 | 127 | @Override 128 | public void setEscapeProcessing(boolean enable) throws SQLException { 129 | throw new SQLFeatureNotSupportedException("FlinkStatement#setEscapeProcessing is not supported"); 130 | } 131 | 132 | @Override 133 | public int getQueryTimeout() throws SQLException { 134 | checkClosed(); 135 | return queryTimeout; 136 | } 137 | 138 | @Override 139 | public void setQueryTimeout(int seconds) throws SQLException { 140 | checkClosed(); 141 | if (seconds < 0) { 142 | throw new SQLException("Query timeout must not be negative."); 143 | } 144 | 145 | queryTimeout = seconds; 146 | } 147 | 148 | @Override 149 | public void cancel() throws SQLException { 150 | checkClosed(); 151 | 152 | AtomicStatements statements = currentStatements.get(); 153 | if (statements == null) { 154 | // do nothing 155 | return; 156 | } 157 | statements.cancel(); 158 | currentStatements.set(null); 159 | } 160 | 161 | @Override 162 | public SQLWarning getWarnings() throws SQLException { 163 | // TODO 164 | // we currently do not support this, 165 | // but we can't throw a SQLException because we want to support beeline 166 | return null; 167 | } 168 | 169 | @Override 170 | public void clearWarnings() throws SQLException { 171 | // TODO 172 | // we currently do not support this, 173 | // but we can't throw a SQLException because we want to support beeline 174 | } 175 | 176 | @Override 177 | public void setCursorName(String name) throws SQLException { 178 | throw new SQLFeatureNotSupportedException("FlinkStatement#setCursorName is not supported"); 179 | } 180 | 181 | @Override 182 | public boolean execute(String sql) throws SQLException { 183 | cancel(); 184 | AtomicStatements statements = new AtomicResultSetStatements(sql); 185 | statements.runNext(); 186 | currentStatements.set(statements); 187 | return statements.isQuery(); 188 | } 189 | 190 | @Override 191 | public ResultSet getResultSet() throws SQLException { 192 | checkClosed(); 193 | checkHasStatements(); 194 | 195 | AtomicStatements statements = currentStatements.get(); 196 | if (!statements.isQuery()) { 197 | throw new SQLException("Current result is not a result set. Please call getUpdateCount() instead."); 198 | } 199 | 200 | Object ret = statements.getCurrentResult(); 201 | if (ret instanceof ResultSet) { 202 | return (ResultSet) ret; 203 | } else { 204 | throw new SQLException("Current result is not a result set."); 205 | } 206 | } 207 | 208 | @Override 209 | public int getUpdateCount() throws SQLException { 210 | return (int) getLargeUpdateCount(); 211 | } 212 | 213 | @Override 214 | public boolean getMoreResults() throws SQLException { 215 | checkClosed(); 216 | checkHasStatements(); 217 | AtomicStatements statements = currentStatements.get(); 218 | statements.runNext(); 219 | return statements.isQuery(); 220 | } 221 | 222 | @Override 223 | public void setFetchDirection(int direction) throws SQLException { 224 | throw new SQLFeatureNotSupportedException("FlinkStatement#setFetchDirection is not supported"); 225 | } 226 | 227 | @Override 228 | public int getFetchDirection() throws SQLException { 229 | throw new SQLFeatureNotSupportedException("FlinkStatement#getFetchDirection is not supported"); 230 | } 231 | 232 | @Override 233 | public void setFetchSize(int rows) throws SQLException { 234 | checkClosed(); 235 | if (rows < 0) { 236 | throw new SQLException("Fetch size must not be negative."); 237 | } 238 | 239 | fetchSize = rows; 240 | } 241 | 242 | @Override 243 | public int getFetchSize() throws SQLException { 244 | checkClosed(); 245 | return fetchSize; 246 | } 247 | 248 | @Override 249 | public int getResultSetConcurrency() throws SQLException { 250 | throw new SQLFeatureNotSupportedException("FlinkStatement#getResultSetConcurrency is not supported"); 251 | } 252 | 253 | @Override 254 | public int getResultSetType() throws SQLException { 255 | return ResultSet.TYPE_FORWARD_ONLY; 256 | } 257 | 258 | @Override 259 | public void addBatch(String sql) throws SQLException { 260 | throw new SQLFeatureNotSupportedException("FlinkStatement#addBatch is not supported"); 261 | } 262 | 263 | @Override 264 | public void clearBatch() throws SQLException { 265 | throw new SQLFeatureNotSupportedException("FlinkStatement#clearBatch is not supported"); 266 | } 267 | 268 | @Override 269 | public int[] executeBatch() throws SQLException { 270 | long[] result = executeLargeBatch(); 271 | int[] ret = new int[result.length]; 272 | for (int i = 0; i < result.length; i++) { 273 | ret[i] = (int) result[i]; 274 | } 275 | return ret; 276 | } 277 | 278 | @Override 279 | public Connection getConnection() throws SQLException { 280 | return connection; 281 | } 282 | 283 | @Override 284 | public boolean getMoreResults(int current) throws SQLException { 285 | throw new SQLFeatureNotSupportedException("FlinkStatement#getMoreResults is not supported"); 286 | } 287 | 288 | @Override 289 | public ResultSet getGeneratedKeys() throws SQLException { 290 | throw new SQLFeatureNotSupportedException("FlinkStatement#getGeneratedKeys is not supported"); 291 | } 292 | 293 | @Override 294 | public int executeUpdate(String sql, int autoGeneratedKeys) throws SQLException { 295 | return (int) executeLargeUpdate(sql, autoGeneratedKeys); 296 | } 297 | 298 | @Override 299 | public int executeUpdate(String sql, int[] columnIndexes) throws SQLException { 300 | return (int) executeLargeUpdate(sql, columnIndexes); 301 | } 302 | 303 | @Override 304 | public int executeUpdate(String sql, String[] columnNames) throws SQLException { 305 | return (int) executeLargeUpdate(sql, columnNames); 306 | } 307 | 308 | @Override 309 | public boolean execute(String sql, int autoGeneratedKeys) throws SQLException { 310 | throw new SQLFeatureNotSupportedException("FlinkStatement#execute is not supported"); 311 | } 312 | 313 | @Override 314 | public boolean execute(String sql, int[] columnIndexes) throws SQLException { 315 | throw new SQLFeatureNotSupportedException("FlinkStatement#execute is not supported"); 316 | } 317 | 318 | @Override 319 | public boolean execute(String sql, String[] columnNames) throws SQLException { 320 | throw new SQLFeatureNotSupportedException("FlinkStatement#execute is not supported"); 321 | } 322 | 323 | @Override 324 | public int getResultSetHoldability() throws SQLException { 325 | throw new SQLFeatureNotSupportedException("FlinkStatement#execute is not supported"); 326 | } 327 | 328 | @Override 329 | public boolean isClosed() throws SQLException { 330 | return closed; 331 | } 332 | 333 | @Override 334 | public void setPoolable(boolean poolable) throws SQLException { 335 | throw new SQLFeatureNotSupportedException("FlinkStatement#setPoolable is not supported"); 336 | } 337 | 338 | @Override 339 | public boolean isPoolable() throws SQLException { 340 | throw new SQLFeatureNotSupportedException("FlinkStatement#isPoolable is not supported"); 341 | } 342 | 343 | @Override 344 | public void closeOnCompletion() throws SQLException { 345 | throw new SQLFeatureNotSupportedException("FlinkStatement#closeOnCompletion is not supported"); 346 | } 347 | 348 | @Override 349 | public boolean isCloseOnCompletion() throws SQLException { 350 | throw new SQLFeatureNotSupportedException("FlinkStatement#isCloseOnCompletion is not supported"); 351 | } 352 | 353 | @Override 354 | public long getLargeUpdateCount() throws SQLException { 355 | checkClosed(); 356 | checkHasStatements(); 357 | 358 | AtomicStatements statements = currentStatements.get(); 359 | if (statements.isQuery()) { 360 | throw new SQLException("Current result is not an update count. Please call getResultSet() instead."); 361 | } 362 | 363 | if (statements.afterLastStatement()) { 364 | // According to the java doc of getMoreResults() 365 | return -1L; 366 | } else { 367 | Object ret = statements.getCurrentResult(); 368 | if (ret instanceof ResultSet) { 369 | ResultSet rs = (ResultSet) ret; 370 | if (rs.next()) { 371 | try { 372 | return rs.getLong(1); 373 | } catch (SQLException e) { 374 | throw new SQLException("Current result is not an update count."); 375 | } 376 | } else { 377 | throw new SQLException("Current result is not an update count."); 378 | } 379 | } else { 380 | throw new SQLException("Current result is not an update count."); 381 | } 382 | } 383 | } 384 | 385 | @Override 386 | public void setLargeMaxRows(long max) throws SQLException { 387 | checkClosed(); 388 | if (max < 0) { 389 | throw new SQLException("Max rows must not be negative."); 390 | } 391 | 392 | maxRows = max; 393 | } 394 | 395 | @Override 396 | public long getLargeMaxRows() throws SQLException { 397 | checkClosed(); 398 | return maxRows; 399 | } 400 | 401 | @Override 402 | public long[] executeLargeBatch() throws SQLException { 403 | throw new SQLFeatureNotSupportedException("FlinkStatement#executeLargeBatch is not supported"); 404 | } 405 | 406 | @Override 407 | public long executeLargeUpdate(String sql) throws SQLException { 408 | boolean isQuery = execute(sql); 409 | if (isQuery) { 410 | throw new SQLException(sql + " is not an update statement"); 411 | } 412 | return getLargeUpdateCount(); 413 | } 414 | 415 | @Override 416 | public long executeLargeUpdate(String sql, int autoGeneratedKeys) throws SQLException { 417 | throw new SQLFeatureNotSupportedException("FlinkStatement#executeLargeUpdate is not supported"); 418 | } 419 | 420 | @Override 421 | public long executeLargeUpdate(String sql, int[] columnIndexes) throws SQLException { 422 | throw new SQLFeatureNotSupportedException("FlinkStatement#executeLargeUpdate is not supported"); 423 | } 424 | 425 | @Override 426 | public long executeLargeUpdate(String sql, String[] columnNames) throws SQLException { 427 | throw new SQLFeatureNotSupportedException("FlinkStatement#executeLargeUpdate is not supported"); 428 | } 429 | 430 | @Override 431 | public T unwrap(Class iface) throws SQLException { 432 | throw new SQLFeatureNotSupportedException("FlinkStatement#unwrap is not supported"); 433 | } 434 | 435 | @Override 436 | public boolean isWrapperFor(Class iface) throws SQLException { 437 | throw new SQLFeatureNotSupportedException("FlinkStatement#isWrapperFor is not supported"); 438 | } 439 | 440 | private void checkClosed() throws SQLException { 441 | if (closed) { 442 | throw new SQLException("This result set is already closed"); 443 | } 444 | } 445 | 446 | private void checkHasStatements() throws SQLException { 447 | if (currentStatements.get() == null) { 448 | throw new SQLException("No statement is running"); 449 | } 450 | } 451 | 452 | /** 453 | * A group of statements executed in order, 454 | * with atomic read results / change current statement interface. 455 | * 456 | * @param statement result type 457 | */ 458 | private interface AtomicStatements { 459 | 460 | boolean runNext() throws SQLException; 461 | 462 | R getCurrentResult(); 463 | 464 | boolean isQuery(); 465 | 466 | boolean afterLastStatement(); 467 | 468 | void cancel() throws SQLException; 469 | } 470 | 471 | /** 472 | * A group of statements executed in order, 473 | * with atomic read results / change current statement interface. 474 | * 475 | *

These statements produce {@link ResultSet} as results. 476 | */ 477 | private class AtomicResultSetStatements implements AtomicStatements { 478 | 479 | private final String[] statements; 480 | private int lastExecutedIdx; 481 | 482 | private Either jobIdOrResultSet; 483 | private ResultSet currentResultSet; 484 | private boolean isQuery; 485 | 486 | private ReadWriteLock lock; 487 | 488 | AtomicResultSetStatements(String stmt) { 489 | this.statements = stmt.split(";"); 490 | this.lastExecutedIdx = -1; 491 | 492 | this.lock = new ReentrantReadWriteLock(); 493 | } 494 | 495 | @Override 496 | public boolean runNext() throws SQLException { 497 | lock.writeLock().lock(); 498 | if (lastExecutedIdx < statements.length) { 499 | lastExecutedIdx++; 500 | } 501 | if (lastExecutedIdx >= statements.length) { 502 | // According to the java doc of getMoreResults() 503 | isQuery = false; 504 | lock.writeLock().unlock(); 505 | return false; 506 | } 507 | String sql = statements[lastExecutedIdx]; 508 | 509 | try { 510 | StatementExecuteResponseBody response; 511 | response = queryTimeout > 0 ? 512 | session.submitStatement(sql, queryTimeout * 1000L) : 513 | session.submitStatement(sql); 514 | 515 | Preconditions.checkState( 516 | response.getResults().size() == 1 && response.getStatementTypes().size() == 1, 517 | "Statement " + sql + " should produce exactly 1 result set. This is a bug."); 518 | jobIdOrResultSet = RestUtils.getEitherJobIdOrResultSet(response.getResults().get(0)); 519 | currentResultSet = new FlinkResultSet( 520 | session, 521 | jobIdOrResultSet, 522 | ResultHandlerFactory.getResultHandlerByStatementType(response.getStatementTypes().get(0)), 523 | maxRows, 524 | FlinkStatement.this); 525 | currentResultSet.setFetchSize(fetchSize); 526 | isQuery = QUERY_COMMANDS.contains(response.getStatementTypes().get(0)); 527 | return true; 528 | } finally { 529 | lock.writeLock().unlock(); 530 | } 531 | } 532 | 533 | @Override 534 | public boolean isQuery() { 535 | lock.readLock().lock(); 536 | boolean ret = isQuery; 537 | lock.readLock().unlock(); 538 | return ret; 539 | } 540 | 541 | @Override 542 | public boolean afterLastStatement() { 543 | lock.readLock().lock(); 544 | boolean ret = lastExecutedIdx >= statements.length; 545 | lock.readLock().unlock(); 546 | return ret; 547 | } 548 | 549 | @Override 550 | public ResultSet getCurrentResult() { 551 | lock.readLock().lock(); 552 | ResultSet ret = currentResultSet; 553 | lock.readLock().unlock(); 554 | return ret; 555 | } 556 | 557 | @Override 558 | public void cancel() throws SQLException { 559 | lock.writeLock().lock(); 560 | currentResultSet.close(); 561 | lock.writeLock().unlock(); 562 | } 563 | } 564 | } 565 | -------------------------------------------------------------------------------- /src/main/java/com/ververica/flink/table/jdbc/rest/RestUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.ververica.flink.table.jdbc.rest; 20 | 21 | import com.ververica.flink.table.gateway.rest.result.ConstantNames; 22 | import com.ververica.flink.table.gateway.rest.result.ResultSet; 23 | 24 | import org.apache.flink.api.common.JobID; 25 | import org.apache.flink.types.Either; 26 | 27 | /** 28 | * Utility class to handle REST data structures. 29 | */ 30 | public class RestUtils { 31 | 32 | public static JobID getJobID(ResultSet resultSet) { 33 | if (resultSet.getColumns().size() != 1) { 34 | throw new IllegalArgumentException("Should contain only one column. This is a bug."); 35 | } 36 | if (resultSet.getColumns().get(0).getName().equals(ConstantNames.JOB_ID)) { 37 | String jobId = (String) resultSet.getData().get(0).getField(0); 38 | return JobID.fromHexString(jobId); 39 | } else { 40 | throw new IllegalArgumentException("Column name should be " + ConstantNames.JOB_ID + ". This is a bug."); 41 | } 42 | } 43 | 44 | public static Either getEitherJobIdOrResultSet(ResultSet resultSet) { 45 | if (resultSet.getColumns().size() == 1 && resultSet.getColumns().get(0).getName() 46 | .equals(ConstantNames.JOB_ID)) { 47 | String jobId = (String) resultSet.getData().get(0).getField(0); 48 | return Either.Left(JobID.fromHexString(jobId)); 49 | } else { 50 | return Either.Right(resultSet); 51 | } 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /src/main/java/com/ververica/flink/table/jdbc/rest/SessionClient.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.ververica.flink.table.jdbc.rest; 20 | 21 | import com.ververica.flink.table.gateway.rest.handler.GetInfoHeaders; 22 | import com.ververica.flink.table.gateway.rest.handler.JobCancelHeaders; 23 | import com.ververica.flink.table.gateway.rest.handler.ResultFetchHeaders; 24 | import com.ververica.flink.table.gateway.rest.handler.SessionCloseHeaders; 25 | import com.ververica.flink.table.gateway.rest.handler.SessionCreateHeaders; 26 | import com.ververica.flink.table.gateway.rest.handler.SessionHeartbeatHeaders; 27 | import com.ververica.flink.table.gateway.rest.handler.StatementExecuteHeaders; 28 | import com.ververica.flink.table.gateway.rest.message.GetInfoResponseBody; 29 | import com.ververica.flink.table.gateway.rest.message.ResultFetchMessageParameters; 30 | import com.ververica.flink.table.gateway.rest.message.ResultFetchRequestBody; 31 | import com.ververica.flink.table.gateway.rest.message.ResultFetchResponseBody; 32 | import com.ververica.flink.table.gateway.rest.message.SessionCreateRequestBody; 33 | import com.ververica.flink.table.gateway.rest.message.SessionJobMessageParameters; 34 | import com.ververica.flink.table.gateway.rest.message.SessionMessageParameters; 35 | import com.ververica.flink.table.gateway.rest.message.StatementExecuteRequestBody; 36 | import com.ververica.flink.table.gateway.rest.message.StatementExecuteResponseBody; 37 | 38 | import org.apache.flink.api.common.JobID; 39 | import org.apache.flink.api.common.time.Time; 40 | import org.apache.flink.configuration.Configuration; 41 | import org.apache.flink.runtime.rest.RestClient; 42 | import org.apache.flink.runtime.rest.RestClientConfiguration; 43 | import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; 44 | import org.apache.flink.runtime.rest.messages.EmptyRequestBody; 45 | import org.apache.flink.runtime.util.ExecutorThreadFactory; 46 | import org.apache.flink.util.ExecutorUtils; 47 | 48 | import java.sql.SQLException; 49 | import java.util.Collections; 50 | import java.util.concurrent.ExecutorService; 51 | import java.util.concurrent.Executors; 52 | import java.util.concurrent.TimeUnit; 53 | 54 | /** 55 | * A client to connect to Flink SQL gateway. 56 | */ 57 | public class SessionClient { 58 | 59 | private final String serverHost; 60 | private final int serverPort; 61 | private final String sessionName; 62 | private final String planner; 63 | private final String executionType; 64 | private final RestClient restClient; 65 | 66 | private final ExecutorService executor; 67 | private volatile String sessionId; 68 | private volatile boolean isClosed = false; 69 | 70 | public SessionClient( 71 | String serverHost, 72 | int serverPort, 73 | String sessionName, 74 | String planner, 75 | String executionType, 76 | String threadName) 77 | throws Exception { 78 | this.serverHost = serverHost; 79 | this.serverPort = serverPort; 80 | this.sessionName = sessionName; 81 | this.planner = planner; 82 | this.executionType = executionType; 83 | this.executor = Executors.newFixedThreadPool(4, new ExecutorThreadFactory(threadName)); 84 | this.restClient = new RestClient(RestClientConfiguration.fromConfiguration(new Configuration()), executor); 85 | 86 | connectInternal(); 87 | } 88 | 89 | public String getServerHost() { 90 | return serverHost; 91 | } 92 | 93 | public int getServerPort() { 94 | return serverPort; 95 | } 96 | 97 | public String getPlanner() { 98 | return planner; 99 | } 100 | 101 | private void connectInternal() throws Exception { 102 | this.sessionId = restClient.sendRequest( 103 | serverHost, 104 | serverPort, 105 | SessionCreateHeaders.getInstance(), 106 | EmptyMessageParameters.getInstance(), 107 | new SessionCreateRequestBody(sessionName, planner, executionType, Collections.emptyMap())) 108 | .get().getSessionId(); 109 | } 110 | 111 | public synchronized void close() throws Exception { 112 | if (isClosed) { 113 | return; 114 | } 115 | isClosed = true; 116 | try { 117 | restClient.sendRequest( 118 | serverHost, 119 | serverPort, 120 | SessionCloseHeaders.getInstance(), 121 | new SessionMessageParameters(sessionId), 122 | EmptyRequestBody.getInstance()).get(); 123 | } finally { 124 | restClient.shutdown(Time.seconds(5)); 125 | ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, executor); 126 | } 127 | } 128 | 129 | public synchronized void sendHeartbeat() throws SQLException { 130 | checkState(); 131 | try { 132 | restClient.sendRequest( 133 | serverHost, 134 | serverPort, 135 | SessionHeartbeatHeaders.getInstance(), 136 | new SessionMessageParameters(sessionId), 137 | EmptyRequestBody.getInstance()) 138 | .get(); 139 | } catch (Exception e) { 140 | throw new SQLException("Failed to send heartbeat to server", e); 141 | } 142 | } 143 | 144 | public StatementExecuteResponseBody submitStatement(String stmt) throws SQLException { 145 | return submitStatement(stmt, Long.MAX_VALUE); 146 | } 147 | 148 | public synchronized StatementExecuteResponseBody submitStatement(String stmt, long executionTimeoutMillis) 149 | throws SQLException { 150 | checkState(); 151 | try { 152 | return restClient.sendRequest( 153 | serverHost, 154 | serverPort, 155 | StatementExecuteHeaders.getInstance(), 156 | new SessionMessageParameters(sessionId), 157 | new StatementExecuteRequestBody(stmt, executionTimeoutMillis)) 158 | .get(); 159 | } catch (Exception e) { 160 | throw new SQLException("Failed to submit statement `" + stmt + "` to server", e); 161 | } 162 | } 163 | 164 | public synchronized void cancelJob(JobID jobId) throws SQLException { 165 | checkState(); 166 | try { 167 | restClient.sendRequest( 168 | serverHost, 169 | serverPort, 170 | JobCancelHeaders.getInstance(), 171 | new SessionJobMessageParameters(sessionId, jobId), 172 | EmptyRequestBody.getInstance()) 173 | .get(); 174 | } catch (Exception e) { 175 | throw new SQLException("Failed to cancel job " + jobId.toString(), e); 176 | } 177 | } 178 | 179 | public synchronized ResultFetchResponseBody fetchResult(JobID jobId, long token) throws SQLException { 180 | return fetchResult(jobId, token, null); 181 | } 182 | 183 | public synchronized ResultFetchResponseBody fetchResult( 184 | JobID jobId, long token, Integer fetchSize) throws SQLException { 185 | checkState(); 186 | try { 187 | return restClient.sendRequest( 188 | serverHost, 189 | serverPort, 190 | ResultFetchHeaders.getInstance(), 191 | new ResultFetchMessageParameters(sessionId, jobId, token), 192 | new ResultFetchRequestBody(fetchSize)) 193 | .get(); 194 | } catch (Exception e) { 195 | throw new SQLException( 196 | "Failed to fetch result for job " + jobId.toString() + 197 | " (token = " + token + ", fetchSize = " + fetchSize + ")", 198 | e.getCause()); 199 | } 200 | } 201 | 202 | public GetInfoResponseBody getInfo() throws SQLException { 203 | checkState(); 204 | try { 205 | return restClient.sendRequest( 206 | serverHost, 207 | serverPort, 208 | GetInfoHeaders.getInstance(), 209 | EmptyMessageParameters.getInstance(), 210 | EmptyRequestBody.getInstance()) 211 | .get(); 212 | } catch (Exception e) { 213 | throw new SQLException("Failed to get server info", e); 214 | } 215 | } 216 | 217 | private void checkState() { 218 | if (isClosed) { 219 | throw new IllegalStateException("Session is already closed."); 220 | } 221 | } 222 | } 223 | -------------------------------------------------------------------------------- /src/main/java/com/ververica/flink/table/jdbc/resulthandler/DefaultResultHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.ververica.flink.table.jdbc.resulthandler; 20 | 21 | import com.ververica.flink.table.gateway.rest.result.ResultSet; 22 | 23 | /** 24 | * A result handler that directly forwards the {@link ResultSet} produced by the REST API. 25 | */ 26 | public class DefaultResultHandler implements ResultHandler { 27 | 28 | @Override 29 | public ResultSet handleResult(ResultSet raw) { 30 | return raw; 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /src/main/java/com/ververica/flink/table/jdbc/resulthandler/DescribeResultHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.ververica.flink.table.jdbc.resulthandler; 20 | 21 | import com.ververica.flink.table.gateway.rest.result.ColumnInfo; 22 | import com.ververica.flink.table.gateway.rest.result.ConstantNames; 23 | import com.ververica.flink.table.gateway.rest.result.ResultSet; 24 | import com.ververica.flink.table.gateway.rest.result.TableSchemaUtil; 25 | 26 | import org.apache.flink.table.api.TableColumn; 27 | import org.apache.flink.table.api.TableSchema; 28 | import org.apache.flink.table.types.logical.BooleanType; 29 | import org.apache.flink.table.types.logical.LogicalType; 30 | import org.apache.flink.table.types.logical.VarCharType; 31 | import org.apache.flink.types.Row; 32 | import org.apache.flink.util.Preconditions; 33 | 34 | import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; 35 | 36 | import java.util.ArrayList; 37 | import java.util.Arrays; 38 | import java.util.Collections; 39 | import java.util.List; 40 | 41 | /** 42 | * A result handler that change the {@link ResultSet} produced by the describe statement of REST API 43 | * to a form that can be printed to screen. 44 | */ 45 | public class DescribeResultHandler implements ResultHandler { 46 | 47 | @Override 48 | public ResultSet handleResult(ResultSet raw) { 49 | List rawColumnInfos = raw.getColumns(); 50 | Preconditions.checkArgument( 51 | rawColumnInfos.size() == 1 && 52 | rawColumnInfos.get(0).getName().equals(ConstantNames.SCHEMA) && 53 | rawColumnInfos.get(0).getLogicalType() instanceof VarCharType, 54 | "Invalid DESCRIBE result schema"); 55 | Preconditions.checkArgument( 56 | raw.getData().size() == 1, 57 | "DESCRIBE result should contain exactly 1 json string record"); 58 | 59 | List newColumnInfos = Arrays.asList( 60 | new ColumnInfo("column_name", rawColumnInfos.get(0).getType()), 61 | new ColumnInfo("column_type", rawColumnInfos.get(0).getType()), 62 | ColumnInfo.create("nullable", new BooleanType(false)), 63 | ColumnInfo.create("primary_key", new BooleanType(false))); 64 | 65 | Row rawRow = raw.getData().get(0); 66 | String json = rawRow.getField(0).toString(); 67 | TableSchema schema; 68 | try { 69 | schema = TableSchemaUtil.readTableSchemaFromJson(json); 70 | } catch (JsonProcessingException e) { 71 | throw new RuntimeException("Failed to parse json to table schema", e); 72 | } 73 | List primaryKeys; 74 | if (schema.getPrimaryKey().isPresent()) { 75 | primaryKeys = schema.getPrimaryKey().get().getColumns(); 76 | } else { 77 | primaryKeys = Collections.emptyList(); 78 | } 79 | 80 | List newRows = new ArrayList<>(); 81 | for (TableColumn column : schema.getTableColumns()) { 82 | String name = column.getName(); 83 | LogicalType type = column.getType().getLogicalType(); 84 | newRows.add(Row.of(name, type.toString(), type.isNullable(), primaryKeys.contains(name))); 85 | } 86 | 87 | return new ResultSet(newColumnInfos, newRows); 88 | } 89 | } 90 | -------------------------------------------------------------------------------- /src/main/java/com/ververica/flink/table/jdbc/resulthandler/ResultHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.ververica.flink.table.jdbc.resulthandler; 20 | 21 | import com.ververica.flink.table.gateway.rest.result.ResultSet; 22 | 23 | /** 24 | * An interface which change {@link ResultSet}s directly returned by REST API to the form we want. 25 | */ 26 | public interface ResultHandler { 27 | 28 | ResultSet handleResult(ResultSet raw); 29 | } 30 | -------------------------------------------------------------------------------- /src/main/java/com/ververica/flink/table/jdbc/resulthandler/ResultHandlerFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.ververica.flink.table.jdbc.resulthandler; 20 | 21 | /** 22 | * Factory to create {@link ResultHandler}s. 23 | */ 24 | public class ResultHandlerFactory { 25 | 26 | public static ResultHandler getDefaultResultHandler() { 27 | return new DefaultResultHandler(); 28 | } 29 | 30 | public static ResultHandler getResultHandlerByStatementType(String statementType) { 31 | if ("DESCRIBE".equals(statementType)) { 32 | return new DescribeResultHandler(); 33 | } else { 34 | return getDefaultResultHandler(); 35 | } 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/com/ververica/flink/table/jdbc/type/FlinkSqlType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.ververica.flink.table.jdbc.type; 20 | 21 | /** 22 | * A data structure which records the information JDBC needed for SQL types. 23 | */ 24 | public class FlinkSqlType { 25 | 26 | private final int sqlType; 27 | private final int precision; 28 | private final int scale; 29 | private final int displaySize; 30 | 31 | FlinkSqlType(int sqlType, int precision, int scale, int displaySize) { 32 | this.sqlType = sqlType; 33 | this.precision = precision; 34 | this.scale = scale; 35 | this.displaySize = displaySize; 36 | } 37 | 38 | public int getSqlType() { 39 | return sqlType; 40 | } 41 | 42 | public int getPrecision() { 43 | return precision; 44 | } 45 | 46 | public int getScale() { 47 | return scale; 48 | } 49 | 50 | public int getDisplaySize() { 51 | return displaySize; 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /src/main/java/com/ververica/flink/table/jdbc/type/FlinkSqlTypes.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.ververica.flink.table.jdbc.type; 20 | 21 | import org.apache.flink.table.types.logical.ArrayType; 22 | import org.apache.flink.table.types.logical.BigIntType; 23 | import org.apache.flink.table.types.logical.BinaryType; 24 | import org.apache.flink.table.types.logical.BooleanType; 25 | import org.apache.flink.table.types.logical.CharType; 26 | import org.apache.flink.table.types.logical.DateType; 27 | import org.apache.flink.table.types.logical.DecimalType; 28 | import org.apache.flink.table.types.logical.DoubleType; 29 | import org.apache.flink.table.types.logical.FloatType; 30 | import org.apache.flink.table.types.logical.IntType; 31 | import org.apache.flink.table.types.logical.LogicalType; 32 | import org.apache.flink.table.types.logical.NullType; 33 | import org.apache.flink.table.types.logical.SmallIntType; 34 | import org.apache.flink.table.types.logical.StructuredType; 35 | import org.apache.flink.table.types.logical.TimeType; 36 | import org.apache.flink.table.types.logical.TimestampType; 37 | import org.apache.flink.table.types.logical.TinyIntType; 38 | import org.apache.flink.table.types.logical.VarBinaryType; 39 | import org.apache.flink.table.types.logical.VarCharType; 40 | import org.apache.flink.table.types.logical.ZonedTimestampType; 41 | 42 | import java.sql.Types; 43 | 44 | /** 45 | * Defines all SQL types with information needed for JDBC. 46 | */ 47 | public class FlinkSqlTypes { 48 | 49 | public static final FlinkSqlType BOOLEAN = new FlinkSqlType(Types.BOOLEAN, 1, 0, 5); 50 | public static final FlinkSqlType TINYINT = new FlinkSqlType(Types.TINYINT, 3, 0, 4); 51 | public static final FlinkSqlType SMALLINT = new FlinkSqlType(Types.SMALLINT, 5, 0, 6); 52 | public static final FlinkSqlType INT = new FlinkSqlType(Types.INTEGER, 10, 0, 11); 53 | public static final FlinkSqlType BIGINT = new FlinkSqlType(Types.BIGINT, 19, 0, 20); 54 | public static final FlinkSqlType FLOAT = new FlinkSqlType(Types.FLOAT, 7, 7, 24); 55 | public static final FlinkSqlType DOUBLE = new FlinkSqlType(Types.DOUBLE, 15, 15, 25); 56 | public static final FlinkSqlType DATE = new FlinkSqlType(Types.DATE, 10, 0, 10); 57 | public static final FlinkSqlType NULL = new FlinkSqlType(Types.NULL, 0, 0, 4); 58 | public static final FlinkSqlType ARRAY = new FlinkSqlType(Types.ARRAY, Integer.MAX_VALUE, 0, Integer.MAX_VALUE); 59 | public static final FlinkSqlType STRUCT = new FlinkSqlType(Types.STRUCT, Integer.MAX_VALUE, 0, Integer.MAX_VALUE); 60 | public static final FlinkSqlType OTHER = new FlinkSqlType(Types.OTHER, Integer.MAX_VALUE, 0, Integer.MAX_VALUE); 61 | 62 | public static FlinkSqlType createDecimalType(DecimalType type) { 63 | int precision = type.getPrecision(); 64 | int scale = type.getScale(); 65 | return new FlinkSqlType(Types.DECIMAL, precision, scale, precision); 66 | } 67 | 68 | public static FlinkSqlType createCharType(CharType type) { 69 | int length = type.getLength(); 70 | return new FlinkSqlType(Types.CHAR, length, 0, length); 71 | } 72 | 73 | public static FlinkSqlType createVarCharType(VarCharType type) { 74 | int length = type.getLength(); 75 | return new FlinkSqlType(Types.VARCHAR, length, 0, length); 76 | } 77 | 78 | public static FlinkSqlType createTimeType(TimeType type) { 79 | int scale = type.getPrecision(); 80 | if (scale > 0) { 81 | return new FlinkSqlType(Types.TIME, 9 + scale, scale, 9 + scale); 82 | } else { 83 | return new FlinkSqlType(Types.TIME, 8, 0, 8); 84 | } 85 | } 86 | 87 | public static FlinkSqlType createTimestampType(TimestampType type) { 88 | int scale = type.getPrecision(); 89 | if (scale > 0) { 90 | return new FlinkSqlType(Types.TIMESTAMP, 20 + scale, scale, 20 + scale); 91 | } else { 92 | return new FlinkSqlType(Types.TIMESTAMP, 19, 0, 19); 93 | } 94 | } 95 | 96 | public static FlinkSqlType createZonedTimstampType(ZonedTimestampType type) { 97 | int scale = type.getPrecision(); 98 | if (scale > 0) { 99 | return new FlinkSqlType(Types.TIMESTAMP_WITH_TIMEZONE, 27 + scale, scale, 27 + scale); 100 | } else { 101 | return new FlinkSqlType(Types.TIMESTAMP_WITH_TIMEZONE, 26, 0, 26); 102 | } 103 | } 104 | 105 | public static FlinkSqlType createBinaryType(BinaryType type) { 106 | int length = type.getLength(); 107 | return new FlinkSqlType(Types.BINARY, length, 0, length); 108 | } 109 | 110 | public static FlinkSqlType createVarBinaryType(VarBinaryType type) { 111 | int length = type.getLength(); 112 | return new FlinkSqlType(Types.VARBINARY, length, 0, length); 113 | } 114 | 115 | public static boolean isNumeric(FlinkSqlType type) { 116 | switch (type.getSqlType()) { 117 | case Types.TINYINT: 118 | case Types.SMALLINT: 119 | case Types.INTEGER: 120 | case Types.BIGINT: 121 | case Types.FLOAT: 122 | case Types.DOUBLE: 123 | case Types.DECIMAL: 124 | return true; 125 | default: 126 | return false; 127 | } 128 | } 129 | 130 | public static boolean isChar(FlinkSqlType type) { 131 | switch (type.getSqlType()) { 132 | case Types.CHAR: 133 | case Types.VARCHAR: 134 | return true; 135 | default: 136 | return false; 137 | } 138 | } 139 | 140 | public static FlinkSqlType getType(LogicalType type) { 141 | if (type instanceof BooleanType) { 142 | return BOOLEAN; 143 | } else if (type instanceof TinyIntType) { 144 | return TINYINT; 145 | } else if (type instanceof SmallIntType) { 146 | return SMALLINT; 147 | } else if (type instanceof IntType) { 148 | return INT; 149 | } else if (type instanceof BigIntType) { 150 | return BIGINT; 151 | } else if (type instanceof FloatType) { 152 | return FLOAT; 153 | } else if (type instanceof DoubleType) { 154 | return DOUBLE; 155 | } else if (type instanceof DecimalType) { 156 | return createDecimalType((DecimalType) type); 157 | } else if (type instanceof CharType) { 158 | return createCharType((CharType) type); 159 | } else if (type instanceof VarCharType) { 160 | return createVarCharType((VarCharType) type); 161 | } else if (type instanceof DateType) { 162 | return DATE; 163 | } else if (type instanceof TimeType) { 164 | return createTimeType((TimeType) type); 165 | } else if (type instanceof TimestampType) { 166 | return createTimestampType((TimestampType) type); 167 | } else if (type instanceof BinaryType) { 168 | return createBinaryType((BinaryType) type); 169 | } else if (type instanceof VarBinaryType) { 170 | return createVarBinaryType((VarBinaryType) type); 171 | } else if (type instanceof NullType) { 172 | return NULL; 173 | } else if (type instanceof StructuredType) { 174 | return STRUCT; 175 | } else if (type instanceof ArrayType) { 176 | return ARRAY; 177 | } else if (type instanceof ZonedTimestampType) { 178 | return createZonedTimstampType((ZonedTimestampType) type); 179 | } else { 180 | return OTHER; 181 | } 182 | } 183 | } 184 | -------------------------------------------------------------------------------- /src/main/resources/META-INF/services/java.sql.Driver: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | com.ververica.flink.table.jdbc.FlinkDriver 17 | -------------------------------------------------------------------------------- /src/test/java/com/ververica/flink/table/jdbc/FlinkConnectionTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package com.ververica.flink.table.jdbc; 21 | 22 | import org.junit.AfterClass; 23 | import org.junit.Assert; 24 | import org.junit.BeforeClass; 25 | import org.junit.Test; 26 | 27 | import java.net.InetSocketAddress; 28 | import java.sql.Connection; 29 | import java.sql.SQLException; 30 | import java.sql.Statement; 31 | 32 | /** 33 | * Tests for {@link FlinkConnection}. 34 | */ 35 | public class FlinkConnectionTest { 36 | 37 | private static FlinkJdbcDriverTestingGateway gateway; 38 | private static Connection connection; 39 | 40 | @BeforeClass 41 | public static void beforeClass() throws Exception { 42 | gateway = new FlinkJdbcDriverTestingGateway(); 43 | gateway.start(); 44 | 45 | InetSocketAddress addr = gateway.getServerAddress(); 46 | Assert.assertNotNull(addr); 47 | connection = new FlinkConnection("jdbc:flink://localhost:" + addr.getPort() + "?planner=blink"); 48 | Assert.assertTrue(connection.isValid(0)); 49 | 50 | Statement statement = connection.createStatement(); 51 | statement.execute("USE CATALOG cat1"); 52 | statement.execute("CREATE DATABASE db12"); 53 | statement.execute("USE CATALOG cat2"); 54 | statement.execute("CREATE DATABASE db22"); 55 | statement.close(); 56 | } 57 | 58 | @AfterClass 59 | public static void afterClass() throws Exception { 60 | connection.close(); 61 | gateway.stop(); 62 | } 63 | 64 | @Test 65 | public void testGetSetCatalog() throws SQLException { 66 | connection.setCatalog("cat1"); 67 | Assert.assertEquals("cat1", connection.getCatalog()); 68 | connection.setCatalog("cat2"); 69 | Assert.assertEquals("cat2", connection.getCatalog()); 70 | } 71 | 72 | @Test 73 | public void testGetSetDatabase() throws SQLException { 74 | connection.setCatalog("cat1"); 75 | Assert.assertEquals("db11", connection.getSchema()); 76 | connection.setSchema("db12"); 77 | Assert.assertEquals("db12", connection.getSchema()); 78 | connection.setCatalog("cat2"); 79 | Assert.assertEquals("db21", connection.getSchema()); 80 | connection.setSchema("db22"); 81 | Assert.assertEquals("db22", connection.getSchema()); 82 | } 83 | } 84 | -------------------------------------------------------------------------------- /src/test/java/com/ververica/flink/table/jdbc/FlinkDatabaseMetaDataTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package com.ververica.flink.table.jdbc; 21 | 22 | import org.junit.AfterClass; 23 | import org.junit.Assert; 24 | import org.junit.BeforeClass; 25 | import org.junit.Ignore; 26 | import org.junit.Test; 27 | 28 | import java.io.File; 29 | import java.net.InetSocketAddress; 30 | import java.sql.Connection; 31 | import java.sql.DatabaseMetaData; 32 | import java.sql.ResultSet; 33 | import java.sql.SQLException; 34 | import java.sql.Statement; 35 | 36 | /** 37 | * Tests for {@link FlinkDatabaseMetaData}. 38 | */ 39 | public class FlinkDatabaseMetaDataTest { 40 | 41 | private static FlinkJdbcDriverTestingGateway gateway; 42 | private static Connection connection; 43 | 44 | @BeforeClass 45 | public static void beforeClass() throws Exception { 46 | gateway = new FlinkJdbcDriverTestingGateway(); 47 | gateway.start(); 48 | 49 | InetSocketAddress addr = gateway.getServerAddress(); 50 | Assert.assertNotNull(addr); 51 | connection = new FlinkConnection("jdbc:flink://localhost:" + addr.getPort() + "?planner=blink"); 52 | Assert.assertTrue(connection.isValid(0)); 53 | 54 | Statement statement = connection.createStatement(); 55 | statement.execute("USE CATALOG cat1"); 56 | statement.execute("CREATE DATABASE db12"); 57 | statement.execute("USE CATALOG cat2"); 58 | statement.execute("CREATE DATABASE db22"); 59 | 60 | createTmpTable("default_catalog", "default_database", "tab001", statement); 61 | createTmpTable("default_catalog", "default_database", "tab002", statement); 62 | createTmpTable("cat1", "db11", "tab111", statement); 63 | createTmpTable("cat1", "db11", "tab112", statement); 64 | createTmpTable("cat1", "db12", "tab121", statement); 65 | createTmpTable("cat2", "db21", "tab211", statement); 66 | createTmpTable("cat2", "db22", "tab221", statement); 67 | createTmpTable("cat2", "db22", "tab222", statement); 68 | 69 | runStatementInCatalogAndDatabase( 70 | "cat1", "db12", "CREATE VIEW view122 AS SELECT * FROM tab121", statement); 71 | runStatementInCatalogAndDatabase( 72 | "cat2", "db21", "CREATE VIEW view212 AS SELECT * FROM tab211", statement); 73 | 74 | statement.close(); 75 | } 76 | 77 | @AfterClass 78 | public static void afterClass() throws Exception { 79 | connection.close(); 80 | gateway.stop(); 81 | } 82 | 83 | @Test 84 | public void testGetCatalogs() throws SQLException { 85 | DatabaseMetaData meta = connection.getMetaData(); 86 | 87 | String[][] expected = new String[][] { 88 | new String[]{"cat1"}, 89 | new String[]{"cat2"}, 90 | new String[]{"default_catalog"}}; 91 | compareStringResults(expected, meta.getCatalogs()); 92 | } 93 | 94 | @Test 95 | public void testGetSchemas() throws SQLException { 96 | DatabaseMetaData meta = connection.getMetaData(); 97 | 98 | String[][] expected1 = new String[][] { 99 | new String[]{"db11", "cat1"}, 100 | new String[]{"db12", "cat1"}, 101 | new String[]{"db21", "cat2"}, 102 | new String[]{"db22", "cat2"}, 103 | new String[]{"default_database", "default_catalog"}}; 104 | compareStringResults(expected1, meta.getSchemas()); 105 | 106 | String[][] expected2 = new String[][] { 107 | new String[]{"db12", "cat1"}, 108 | new String[]{"db22", "cat2"}}; 109 | compareStringResults(expected2, meta.getSchemas(null, "d%2")); 110 | 111 | String[][] expected3 = new String[][] { 112 | new String[]{"db21", "cat2"}}; 113 | compareStringResults(expected3, meta.getSchemas("cat2", "d__1")); 114 | } 115 | 116 | // view in SQL gateway is not bounded to a certain database, this is a gateway bug 117 | @Ignore 118 | @Test 119 | public void testGetTables() throws SQLException { 120 | DatabaseMetaData meta = connection.getMetaData(); 121 | 122 | String[][] expected1 = new String[][] { 123 | new String[]{"cat1", "db11", "tab111", "TABLE"}, 124 | new String[]{"cat1", "db11", "tab112", "TABLE"}, 125 | new String[]{"cat1", "db12", "tab121", "TABLE"}, 126 | new String[]{"cat1", "db12", "view122", "VIEW"}}; 127 | compareStringResults(expected1, meta.getTables("cat1", null, null, null)); 128 | 129 | String[][] expected2 = new String[][] { 130 | new String[]{"cat2", "db11", "tab111", "TABLE"}, 131 | new String[]{"cat2", "db12", "tab121", "TABLE"}}; 132 | compareStringResults(expected2, meta.getTables("cat2", null, "t%1", new String[]{"TABLE"})); 133 | 134 | String[][] expected3 = new String[][] { 135 | new String[]{"cat1", "db12", "view122", "VIEW"}}; 136 | compareStringResults(expected2, meta.getTables("cat2", "d__2", "%2", new String[]{"VIEW"})); 137 | } 138 | 139 | @Test 140 | public void testGetColumns() throws SQLException { 141 | DatabaseMetaData meta = connection.getMetaData(); 142 | 143 | String[][] expected1 = new String[][] { 144 | new String[]{"cat1", "db11", "tab112", "fa"}, 145 | new String[]{"cat2", "db22", "tab222", "fa"}, 146 | new String[]{"default_catalog", "default_database", "tab002", "fa"}}; 147 | compareStringResults(expected1, meta.getColumns(null, null, "t%2", "_a")); 148 | 149 | String[][] expected2 = new String[][] { 150 | new String[]{"cat2", "db21", "tab211", "fb"}}; 151 | compareStringResults(expected2, meta.getColumns("cat2", "%1", "t%1", "fb")); 152 | } 153 | 154 | private void compareStringResults(String[][] expected, ResultSet rs) throws SQLException { 155 | for (String[] row : expected) { 156 | Assert.assertTrue(rs.next()); 157 | for (int i = 0; i < row.length; i++) { 158 | Assert.assertEquals(row[i], rs.getString(i + 1)); 159 | } 160 | } 161 | Assert.assertFalse(rs.next()); 162 | } 163 | 164 | private static void createTmpTable( 165 | String catalog, String database, String table, Statement statement) throws Exception { 166 | statement.execute("USE CATALOG " + catalog); 167 | statement.execute("USE " + database); 168 | 169 | File tmpFile = File.createTempFile("Flink-JDBC-test", ".csv"); 170 | tmpFile.deleteOnExit(); 171 | statement.execute("CREATE TABLE " + table + "(" + 172 | " fa INT," + 173 | " fb VARCHAR(100)" + 174 | ") WITH (" + 175 | " 'connector.type'='filesystem'," + 176 | " 'connector.path'='file://" + tmpFile.getPath() + "'," + 177 | " 'format.type' = 'csv')"); 178 | } 179 | 180 | private static void runStatementInCatalogAndDatabase( 181 | String catalog, String database, String stmt, Statement statement) throws SQLException { 182 | statement.execute("USE CATALOG " + catalog); 183 | statement.execute("USE " + database); 184 | statement.execute(stmt); 185 | } 186 | } 187 | -------------------------------------------------------------------------------- /src/test/java/com/ververica/flink/table/jdbc/FlinkJdbcDriverTestingGateway.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package com.ververica.flink.table.jdbc; 21 | 22 | import com.ververica.flink.table.gateway.SessionManager; 23 | import com.ververica.flink.table.gateway.config.Environment; 24 | import com.ververica.flink.table.gateway.context.DefaultContext; 25 | import com.ververica.flink.table.gateway.rest.SqlGatewayEndpoint; 26 | 27 | import org.apache.flink.client.cli.DefaultCLI; 28 | import org.apache.flink.client.deployment.DefaultClusterClientServiceLoader; 29 | import org.apache.flink.client.program.ClusterClient; 30 | import org.apache.flink.configuration.ConfigConstants; 31 | import org.apache.flink.configuration.Configuration; 32 | import org.apache.flink.configuration.MemorySize; 33 | import org.apache.flink.configuration.RestOptions; 34 | import org.apache.flink.configuration.TaskManagerOptions; 35 | import org.apache.flink.configuration.WebOptions; 36 | import org.apache.flink.runtime.rest.RestServerEndpointConfiguration; 37 | import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; 38 | import org.apache.flink.test.util.MiniClusterWithClientResource; 39 | import org.apache.flink.util.FileUtils; 40 | 41 | import java.io.File; 42 | import java.net.InetSocketAddress; 43 | import java.net.URL; 44 | import java.util.Collections; 45 | import java.util.Objects; 46 | 47 | /** 48 | * A {@link SqlGatewayEndpoint} for the Flink JDBC driver test cases to connect to. 49 | */ 50 | public class FlinkJdbcDriverTestingGateway { 51 | 52 | private static final String DEFAULT_ENVIRONMENT_FILE = "default-env.yaml"; 53 | private static final String TEST_DATA_FILE = "test-data.csv"; 54 | 55 | private static final int NUM_TMS = 2; 56 | private static final int NUM_SLOTS_PER_TM = 2; 57 | 58 | private MiniClusterWithClientResource miniClusterWithClientResource; 59 | private SqlGatewayEndpoint endpoint; 60 | 61 | public void start() throws Exception { 62 | miniClusterWithClientResource = new MiniClusterWithClientResource( 63 | new MiniClusterResourceConfiguration.Builder() 64 | .setConfiguration(getMiniClusterConfig()) 65 | .setNumberTaskManagers(NUM_TMS) 66 | .setNumberSlotsPerTaskManager(NUM_SLOTS_PER_TM) 67 | .build()); 68 | miniClusterWithClientResource.before(); 69 | ClusterClient clusterClient = miniClusterWithClientResource.getClusterClient(); 70 | 71 | final URL envUrl = FlinkJdbcDriverTestingGateway.class.getClassLoader().getResource(DEFAULT_ENVIRONMENT_FILE); 72 | Objects.requireNonNull(envUrl); 73 | final URL dataUrl = FlinkJdbcDriverTestingGateway.class.getClassLoader().getResource(TEST_DATA_FILE); 74 | Objects.requireNonNull(dataUrl); 75 | String schema = FileUtils.readFileUtf8(new File(envUrl.getFile())) 76 | .replace("$VAR_SOURCE_PATH", dataUrl.getPath()); 77 | Environment env = Environment.parse(schema); 78 | 79 | DefaultContext defaultContext = new DefaultContext( 80 | env, 81 | Collections.emptyList(), 82 | clusterClient.getFlinkConfiguration(), 83 | new DefaultCLI(clusterClient.getFlinkConfiguration()), 84 | new DefaultClusterClientServiceLoader()); 85 | SessionManager sessionManager = new SessionManager(defaultContext); 86 | 87 | endpoint = new SqlGatewayEndpoint( 88 | RestServerEndpointConfiguration.fromConfiguration(getEndpointConfig()), 89 | sessionManager); 90 | endpoint.start(); 91 | } 92 | 93 | public InetSocketAddress getServerAddress() { 94 | return endpoint.getServerAddress(); 95 | } 96 | 97 | public void stop() throws Exception { 98 | endpoint.close(); 99 | miniClusterWithClientResource.after(); 100 | } 101 | 102 | private static Configuration getMiniClusterConfig() { 103 | Configuration config = new Configuration(); 104 | config.set(TaskManagerOptions.MANAGED_MEMORY_SIZE, MemorySize.parse("4m")); 105 | config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); 106 | config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, NUM_SLOTS_PER_TM); 107 | config.setBoolean(WebOptions.SUBMIT_ENABLE, false); 108 | return config; 109 | } 110 | 111 | private static Configuration getEndpointConfig() { 112 | Configuration config = new Configuration(); 113 | config.setString(RestOptions.ADDRESS, "localhost"); 114 | config.setString(RestOptions.BIND_PORT, "0-65535"); 115 | return config; 116 | } 117 | } 118 | -------------------------------------------------------------------------------- /src/test/java/com/ververica/flink/table/jdbc/FlinkResultSetTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package com.ververica.flink.table.jdbc; 21 | 22 | import org.junit.After; 23 | import org.junit.AfterClass; 24 | import org.junit.Assert; 25 | import org.junit.Before; 26 | import org.junit.BeforeClass; 27 | import org.junit.Test; 28 | 29 | import java.math.BigDecimal; 30 | import java.net.InetSocketAddress; 31 | import java.sql.Connection; 32 | import java.sql.Date; 33 | import java.sql.ResultSet; 34 | import java.sql.ResultSetMetaData; 35 | import java.sql.SQLException; 36 | import java.sql.Statement; 37 | import java.sql.Time; 38 | import java.sql.Timestamp; 39 | import java.sql.Types; 40 | import java.time.LocalDate; 41 | import java.time.LocalDateTime; 42 | import java.time.LocalTime; 43 | import java.time.ZonedDateTime; 44 | import java.util.ArrayList; 45 | import java.util.Arrays; 46 | import java.util.Calendar; 47 | import java.util.List; 48 | import java.util.TimeZone; 49 | 50 | /** 51 | * Tests for {@link FlinkResultSet}. 52 | */ 53 | public class FlinkResultSetTest { 54 | 55 | private static FlinkJdbcDriverTestingGateway gateway; 56 | private static Connection connection; 57 | private Statement statement; 58 | 59 | @BeforeClass 60 | public static void beforeClass() throws Exception { 61 | gateway = new FlinkJdbcDriverTestingGateway(); 62 | gateway.start(); 63 | 64 | InetSocketAddress addr = gateway.getServerAddress(); 65 | Assert.assertNotNull(addr); 66 | connection = new FlinkConnection("jdbc:flink://localhost:" + addr.getPort() + "?planner=blink"); 67 | Assert.assertTrue(connection.isValid(0)); 68 | } 69 | 70 | @AfterClass 71 | public static void afterClass() throws Exception { 72 | connection.close(); 73 | gateway.stop(); 74 | } 75 | 76 | @Before 77 | public void before() throws SQLException { 78 | statement = connection.createStatement(); 79 | } 80 | 81 | @After 82 | public void after() throws SQLException { 83 | statement.close(); 84 | } 85 | 86 | @Test 87 | public void testObjectTypes() throws SQLException { 88 | checkRepresentation("true", Types.BOOLEAN, true); 89 | checkRepresentation("CAST('123' AS TINYINT)", Types.TINYINT, (byte) 123); 90 | checkRepresentation("CAST('123' AS SMALLINT)", Types.SMALLINT, (short) 123); 91 | checkRepresentation("123", Types.INTEGER, 123); 92 | checkRepresentation("12300000000", Types.BIGINT, 12300000000L); 93 | checkRepresentation("CAST('123.45' AS FLOAT)", Types.FLOAT, 123.45f); 94 | checkRepresentation("1e-1", Types.DOUBLE, 0.1); 95 | checkRepresentation("CAST('123.45' AS DECIMAL(5, 2))", Types.DECIMAL, BigDecimal.valueOf(123.45)); 96 | checkRepresentation("CAST('hello' as VARCHAR(10))", Types.VARCHAR, "hello"); 97 | checkRepresentation("CAST('foo' as CHAR(5))", Types.CHAR, "foo "); 98 | checkRepresentation("CAST('2020-02-11' as DATE)", Types.DATE, LocalDate.of(2020, 2, 11)); 99 | checkRepresentation("CAST('15:43:00.123' AS TIME(3))", Types.TIME, LocalTime.of(15, 43, 0, 123000000)); 100 | checkRepresentation("CAST('2020-02-11 15:43:00.123' AS TIMESTAMP(3))", Types.TIMESTAMP, LocalDateTime.of(2020, 2, 11, 15, 43, 0, 123000000)); 101 | 102 | // TODO ExpressionReducer will throw exception 103 | // checkRepresentation("1.0E0 / 0.0E0", Types.DOUBLE, Double.POSITIVE_INFINITY); 104 | // checkRepresentation("0.0E0 / 0.0E0", Types.DOUBLE, Double.NaN); 105 | } 106 | 107 | private void checkRepresentation(String expression, int expectedSqlType, Object expected) throws SQLException { 108 | try (ResultSet rs = statement.executeQuery("SELECT " + expression)) { 109 | ResultSetMetaData metadata = rs.getMetaData(); 110 | Assert.assertEquals(1, metadata.getColumnCount()); 111 | Assert.assertEquals(expectedSqlType, metadata.getColumnType(1)); 112 | Assert.assertTrue(rs.next()); 113 | Assert.assertEquals(expected, rs.getObject(1)); 114 | Assert.assertFalse(rs.next()); 115 | } 116 | } 117 | 118 | @Test 119 | public void testGetString() throws SQLException { 120 | try (ResultSet rs = statement.executeQuery("SELECT " + 121 | "CAST('str1' AS CHAR(4)) x, " + 122 | "CAST('str2' AS VARCHAR(4)), " + 123 | "CAST('str3' AS BINARY(4)), " + 124 | "CAST('str4' AS VARBINARY(4)), " + 125 | "CAST(NULL AS VARCHAR(4))")) { 126 | Assert.assertTrue(rs.next()); 127 | Assert.assertEquals("str1", rs.getString(1)); 128 | Assert.assertFalse(rs.wasNull()); 129 | Assert.assertEquals("str2", rs.getString(2)); 130 | Assert.assertFalse(rs.wasNull()); 131 | Assert.assertEquals("str3", rs.getString(3)); 132 | Assert.assertFalse(rs.wasNull()); 133 | Assert.assertEquals("str4", rs.getString(4)); 134 | Assert.assertFalse(rs.wasNull()); 135 | Assert.assertNull(rs.getString(5)); 136 | Assert.assertTrue(rs.wasNull()); 137 | Assert.assertEquals("str1", rs.getString("x")); 138 | Assert.assertFalse(rs.wasNull()); 139 | Assert.assertFalse(rs.next()); 140 | } 141 | } 142 | 143 | @Test 144 | public void testGetBoolean() throws SQLException { 145 | try (ResultSet rs = statement.executeQuery("SELECT " + 146 | "true x, 0, 'hello', '0', CAST(NULL AS BOOLEAN)")) { 147 | Assert.assertTrue(rs.next()); 148 | Assert.assertTrue(rs.getBoolean(1)); 149 | Assert.assertFalse(rs.wasNull()); 150 | Assert.assertFalse(rs.getBoolean(2)); 151 | Assert.assertFalse(rs.wasNull()); 152 | Assert.assertTrue(rs.getBoolean(3)); 153 | Assert.assertFalse(rs.wasNull()); 154 | Assert.assertFalse(rs.getBoolean(4)); 155 | Assert.assertFalse(rs.wasNull()); 156 | Assert.assertNull(rs.getString(5)); 157 | Assert.assertTrue(rs.wasNull()); 158 | Assert.assertTrue(rs.getBoolean("x")); 159 | Assert.assertFalse(rs.wasNull()); 160 | Assert.assertFalse(rs.next()); 161 | } 162 | } 163 | 164 | @Test 165 | public void testGetByte() throws SQLException { 166 | try (ResultSet rs = statement.executeQuery("SELECT " + 167 | "CAST(1 AS TINYINT) x, 2, '3', CAST(NULL AS TINYINT)")) { 168 | Assert.assertTrue(rs.next()); 169 | Assert.assertEquals((byte) 1, rs.getByte(1)); 170 | Assert.assertFalse(rs.wasNull()); 171 | Assert.assertEquals((byte) 2, rs.getByte(2)); 172 | Assert.assertFalse(rs.wasNull()); 173 | Assert.assertEquals((byte) 3, rs.getByte(3)); 174 | Assert.assertFalse(rs.wasNull()); 175 | Assert.assertEquals(0, rs.getByte(4)); 176 | Assert.assertTrue(rs.wasNull()); 177 | Assert.assertEquals((byte) 1, rs.getByte("x")); 178 | Assert.assertFalse(rs.wasNull()); 179 | Assert.assertFalse(rs.next()); 180 | } 181 | } 182 | 183 | @Test 184 | public void testGetShort() throws SQLException { 185 | try (ResultSet rs = statement.executeQuery("SELECT " + 186 | "CAST(1 AS SMALLINT) x, 2, '3', CAST(NULL AS SMALLINT)")) { 187 | Assert.assertTrue(rs.next()); 188 | Assert.assertEquals((short) 1, rs.getShort(1)); 189 | Assert.assertFalse(rs.wasNull()); 190 | Assert.assertEquals((short) 2, rs.getShort(2)); 191 | Assert.assertFalse(rs.wasNull()); 192 | Assert.assertEquals((short) 3, rs.getShort(3)); 193 | Assert.assertFalse(rs.wasNull()); 194 | Assert.assertEquals(0, rs.getShort(4)); 195 | Assert.assertTrue(rs.wasNull()); 196 | Assert.assertEquals((short) 1, rs.getShort("x")); 197 | Assert.assertFalse(rs.wasNull()); 198 | Assert.assertFalse(rs.next()); 199 | } 200 | } 201 | 202 | @Test 203 | public void testGetInt() throws SQLException { 204 | try (ResultSet rs = statement.executeQuery("SELECT " + 205 | "1 x, '2', CAST(NULL AS INT)")) { 206 | Assert.assertTrue(rs.next()); 207 | Assert.assertEquals(1, rs.getInt(1)); 208 | Assert.assertFalse(rs.wasNull()); 209 | Assert.assertEquals(2, rs.getInt(2)); 210 | Assert.assertFalse(rs.wasNull()); 211 | Assert.assertEquals(0, rs.getInt(3)); 212 | Assert.assertTrue(rs.wasNull()); 213 | Assert.assertEquals(1, rs.getInt("x")); 214 | Assert.assertFalse(rs.wasNull()); 215 | Assert.assertFalse(rs.next()); 216 | } 217 | } 218 | 219 | @Test 220 | public void testGetLong() throws SQLException { 221 | try (ResultSet rs = statement.executeQuery("SELECT " + 222 | "CAST(1 AS BIGINT) x, 2, '3', CAST(NULL AS BIGINT)")) { 223 | Assert.assertTrue(rs.next()); 224 | Assert.assertEquals(1L, rs.getLong(1)); 225 | Assert.assertFalse(rs.wasNull()); 226 | Assert.assertEquals(2L, rs.getLong(2)); 227 | Assert.assertFalse(rs.wasNull()); 228 | Assert.assertEquals(3L, rs.getLong(3)); 229 | Assert.assertFalse(rs.wasNull()); 230 | Assert.assertEquals(0L, rs.getLong(4)); 231 | Assert.assertTrue(rs.wasNull()); 232 | Assert.assertEquals(1L, rs.getLong("x")); 233 | Assert.assertFalse(rs.wasNull()); 234 | Assert.assertFalse(rs.next()); 235 | } 236 | } 237 | 238 | @Test 239 | public void testGetFloat() throws SQLException { 240 | try (ResultSet rs = statement.executeQuery("SELECT " + 241 | "CAST(0.2 AS FLOAT) x, 0.4, '0.8', CAST(NULL AS FLOAT)")) { 242 | Assert.assertTrue(rs.next()); 243 | Assert.assertEquals(0.2F, rs.getFloat(1), 0F); 244 | Assert.assertFalse(rs.wasNull()); 245 | Assert.assertEquals(0.4F, rs.getFloat(2), 0F); 246 | Assert.assertFalse(rs.wasNull()); 247 | Assert.assertEquals(0.8F, rs.getFloat(3), 0F); 248 | Assert.assertFalse(rs.wasNull()); 249 | Assert.assertEquals(0F, rs.getLong(4), 0F); 250 | Assert.assertTrue(rs.wasNull()); 251 | Assert.assertEquals(0.2F, rs.getFloat("x"), 0F); 252 | Assert.assertFalse(rs.wasNull()); 253 | Assert.assertFalse(rs.next()); 254 | } 255 | } 256 | 257 | @Test 258 | public void testGetDouble() throws SQLException { 259 | try (ResultSet rs = statement.executeQuery("SELECT " + 260 | "CAST(0.2 AS DOUBLE) x, 0.4, '0.8', CAST(NULL AS DOUBLE)")) { 261 | Assert.assertTrue(rs.next()); 262 | Assert.assertEquals(0.2, rs.getDouble(1), 0D); 263 | Assert.assertFalse(rs.wasNull()); 264 | Assert.assertEquals(0.4, rs.getDouble(2), 0D); 265 | Assert.assertFalse(rs.wasNull()); 266 | Assert.assertEquals(0.8, rs.getDouble(3), 0D); 267 | Assert.assertFalse(rs.wasNull()); 268 | Assert.assertEquals(0D, rs.getDouble(4), 0D); 269 | Assert.assertTrue(rs.wasNull()); 270 | Assert.assertEquals(0.2, rs.getDouble("x"), 0D); 271 | Assert.assertFalse(rs.wasNull()); 272 | Assert.assertFalse(rs.next()); 273 | } 274 | } 275 | 276 | @Test 277 | public void testGetBigDecimal() throws SQLException { 278 | try (ResultSet rs = statement.executeQuery("SELECT " + 279 | "CAST(123.45 AS DECIMAL(5, 2)) x, CAST(NULL AS DECIMAL(5, 2))")) { 280 | Assert.assertTrue(rs.next()); 281 | Assert.assertEquals(BigDecimal.valueOf(123.45), rs.getBigDecimal(1)); 282 | Assert.assertFalse(rs.wasNull()); 283 | Assert.assertNull(rs.getBigDecimal(2)); 284 | Assert.assertTrue(rs.wasNull()); 285 | Assert.assertEquals(BigDecimal.valueOf(123.45), rs.getBigDecimal("x")); 286 | Assert.assertFalse(rs.wasNull()); 287 | Assert.assertFalse(rs.next()); 288 | } 289 | } 290 | 291 | @Test 292 | public void testGetBytes() throws SQLException { 293 | try (ResultSet rs = statement.executeQuery("SELECT " + 294 | "CAST('str1' AS BINARY(4)) x, " + 295 | "CAST('str2' AS VARBINARY(4)), " + 296 | "CAST('str3' AS CHAR(4)), " + 297 | "CAST('str4' AS VARCHAR(4)), " + 298 | "CAST(NULL AS BINARY(4))")) { 299 | Assert.assertTrue(rs.next()); 300 | Assert.assertArrayEquals("str1".getBytes(), rs.getBytes(1)); 301 | Assert.assertFalse(rs.wasNull()); 302 | Assert.assertArrayEquals("str2".getBytes(), rs.getBytes(2)); 303 | Assert.assertFalse(rs.wasNull()); 304 | Assert.assertArrayEquals("str3".getBytes(), rs.getBytes(3)); 305 | Assert.assertFalse(rs.wasNull()); 306 | Assert.assertArrayEquals("str4".getBytes(), rs.getBytes(4)); 307 | Assert.assertFalse(rs.wasNull()); 308 | Assert.assertNull(rs.getBytes(5)); 309 | Assert.assertTrue(rs.wasNull()); 310 | Assert.assertArrayEquals("str1".getBytes(), rs.getBytes("x")); 311 | Assert.assertFalse(rs.wasNull()); 312 | Assert.assertFalse(rs.next()); 313 | } 314 | } 315 | 316 | @Test 317 | public void testGetDate() throws SQLException { 318 | try (ResultSet rs = statement.executeQuery("SELECT " + 319 | "CAST('2020-02-12' AS DATE) x, '2020-02-13', CAST(NULL AS DATE)")) { 320 | Assert.assertTrue(rs.next()); 321 | Assert.assertEquals(Date.valueOf("2020-02-12"), rs.getDate(1)); 322 | Assert.assertFalse(rs.wasNull()); 323 | Assert.assertEquals(Date.valueOf("2020-02-13"), rs.getDate(2)); 324 | Assert.assertFalse(rs.wasNull()); 325 | Assert.assertNull(rs.getBytes(3)); 326 | Assert.assertTrue(rs.wasNull()); 327 | Assert.assertEquals(Date.valueOf("2020-02-12"), rs.getDate("x")); 328 | Assert.assertFalse(rs.wasNull()); 329 | 330 | TimeZone tz = TimeZone.getTimeZone("UTC"); 331 | Assert.assertEquals( 332 | new Date(ZonedDateTime.of( 333 | LocalDateTime.of(2020, 2, 12, 0, 0, 0), 334 | tz.toZoneId()).toInstant().toEpochMilli()), 335 | rs.getDate("x", Calendar.getInstance(tz))); 336 | Assert.assertFalse(rs.wasNull()); 337 | Assert.assertEquals( 338 | new Date(ZonedDateTime.of( 339 | LocalDateTime.of(2020, 2, 13, 0, 0, 0), 340 | tz.toZoneId()).toInstant().toEpochMilli()), 341 | rs.getDate(2, Calendar.getInstance(tz))); 342 | Assert.assertFalse(rs.wasNull()); 343 | 344 | Assert.assertFalse(rs.next()); 345 | } 346 | } 347 | 348 | @Test 349 | public void testGetTime() throws SQLException { 350 | try (ResultSet rs = statement.executeQuery("SELECT " + 351 | "CAST('15:20:00' AS TIME) x, '16:20:00', CAST(NULL AS TIME)")) { 352 | Assert.assertTrue(rs.next()); 353 | Assert.assertEquals(Time.valueOf("15:20:00"), rs.getTime(1)); 354 | Assert.assertFalse(rs.wasNull()); 355 | Assert.assertEquals(Time.valueOf("16:20:00"), rs.getTime(2)); 356 | Assert.assertFalse(rs.wasNull()); 357 | Assert.assertNull(rs.getBytes(3)); 358 | Assert.assertTrue(rs.wasNull()); 359 | Assert.assertEquals(Time.valueOf("15:20:00"), rs.getTime("x")); 360 | Assert.assertFalse(rs.wasNull()); 361 | 362 | TimeZone tz = TimeZone.getTimeZone("UTC"); 363 | Assert.assertEquals( 364 | new Time(ZonedDateTime.of( 365 | LocalDateTime.of(1970, 1, 1, 15, 20, 0), 366 | tz.toZoneId()).toInstant().toEpochMilli()), 367 | rs.getTime("x", Calendar.getInstance(tz))); 368 | Assert.assertFalse(rs.wasNull()); 369 | Assert.assertEquals( 370 | new Time(ZonedDateTime.of( 371 | LocalDateTime.of(1970, 1, 1, 16, 20, 0), 372 | tz.toZoneId()).toInstant().toEpochMilli()), 373 | rs.getTime(2, Calendar.getInstance(tz))); 374 | Assert.assertFalse(rs.wasNull()); 375 | 376 | Assert.assertFalse(rs.next()); 377 | } 378 | } 379 | 380 | @Test 381 | public void testGetTimestamp() throws SQLException { 382 | try (ResultSet rs = statement.executeQuery("SELECT " + 383 | "CAST('2020-02-12 15:20:00' AS TIMESTAMP) x, '2020-02-13 16:20:00', CAST(NULL AS TIMESTAMP)")) { 384 | Assert.assertTrue(rs.next()); 385 | Assert.assertEquals(Timestamp.valueOf("2020-02-12 15:20:00"), rs.getTimestamp(1)); 386 | Assert.assertFalse(rs.wasNull()); 387 | Assert.assertEquals(Timestamp.valueOf("2020-02-13 16:20:00"), rs.getTimestamp(2)); 388 | Assert.assertFalse(rs.wasNull()); 389 | Assert.assertNull(rs.getBytes(3)); 390 | Assert.assertTrue(rs.wasNull()); 391 | Assert.assertEquals(Timestamp.valueOf("2020-02-12 15:20:00"), rs.getTimestamp("x")); 392 | Assert.assertFalse(rs.wasNull()); 393 | 394 | TimeZone tz = TimeZone.getTimeZone("UTC"); 395 | Assert.assertEquals( 396 | new Timestamp(ZonedDateTime.of( 397 | LocalDateTime.of(2020, 2, 12, 15, 20, 0), 398 | tz.toZoneId()).toInstant().toEpochMilli()), 399 | rs.getTimestamp("x", Calendar.getInstance(tz))); 400 | Assert.assertFalse(rs.wasNull()); 401 | Assert.assertEquals( 402 | new Timestamp(ZonedDateTime.of( 403 | LocalDateTime.of(2020, 2, 13, 16, 20, 0), 404 | tz.toZoneId()).toInstant().toEpochMilli()), 405 | rs.getTimestamp(2, Calendar.getInstance(tz))); 406 | Assert.assertFalse(rs.wasNull()); 407 | 408 | Assert.assertFalse(rs.next()); 409 | } 410 | } 411 | 412 | @Test 413 | public void testPositions() throws SQLException { 414 | try (ResultSet rs = statement.executeQuery("SELECT * FROM myTable LIMIT 2")) { 415 | Assert.assertTrue(rs.isBeforeFirst()); 416 | Assert.assertFalse(rs.isFirst()); 417 | Assert.assertFalse(rs.isAfterLast()); 418 | Assert.assertEquals(0, rs.getRow()); 419 | 420 | Assert.assertTrue(rs.next()); 421 | Assert.assertFalse(rs.isBeforeFirst()); 422 | Assert.assertTrue(rs.isFirst()); 423 | Assert.assertFalse(rs.isAfterLast()); 424 | Assert.assertEquals(1, rs.getRow()); 425 | 426 | Assert.assertTrue(rs.next()); 427 | Assert.assertFalse(rs.isBeforeFirst()); 428 | Assert.assertFalse(rs.isFirst()); 429 | Assert.assertFalse(rs.isAfterLast()); 430 | Assert.assertEquals(2, rs.getRow()); 431 | 432 | Assert.assertFalse(rs.next()); 433 | Assert.assertFalse(rs.isBeforeFirst()); 434 | Assert.assertFalse(rs.isFirst()); 435 | Assert.assertTrue(rs.isAfterLast()); 436 | Assert.assertEquals(0, rs.getRow()); 437 | } 438 | } 439 | 440 | @Test 441 | public void testFetchResultMultipleTimes() throws SQLException { 442 | int[] expectedInt = new int[]{ 443 | 22, 32, 32, 42, 42, 52}; 444 | String[] expectedString = new String[]{ 445 | "BBB Hi", "CCC World", "DDD Hello!!!!", "AAA Hello", "EEE Hi!!!!", "FFF World!!!!"}; 446 | 447 | statement.setFetchSize(2); 448 | try (ResultSet rs = statement.executeQuery("SELECT * FROM myTable ORDER BY a, b")) { 449 | for (int i = 0; i < expectedInt.length; i++) { 450 | Assert.assertTrue(rs.next()); 451 | Assert.assertEquals(i + 1, rs.getRow()); 452 | Assert.assertEquals(expectedInt[i], rs.getInt(1)); 453 | Assert.assertEquals(expectedString[i], rs.getString(2)); 454 | } 455 | Assert.assertFalse(rs.next()); 456 | } 457 | statement.setFetchSize(0); 458 | } 459 | 460 | @Test 461 | public void testInstantResult() throws SQLException { 462 | String[] expected = new String[]{ 463 | "default_catalog", "cat1", "cat2"}; 464 | Arrays.sort(expected); 465 | 466 | List actualList = new ArrayList<>(); 467 | try (ResultSet rs = statement.executeQuery("SHOW CATALOGS")) { 468 | for (int i = 0; i < expected.length; i++) { 469 | Assert.assertTrue(rs.next()); 470 | actualList.add(rs.getString(1)); 471 | } 472 | } 473 | String[] actual = actualList.toArray(new String[0]); 474 | Arrays.sort(actual); 475 | 476 | Assert.assertArrayEquals(expected, actual); 477 | } 478 | 479 | @Test 480 | public void testEmptyResult() throws SQLException { 481 | try (ResultSet rs = statement.executeQuery("SELECT * FROM myTable WHERE a = -1")) { 482 | Assert.assertFalse(rs.next()); 483 | Assert.assertEquals(0, rs.getRow()); 484 | } 485 | } 486 | } 487 | -------------------------------------------------------------------------------- /src/test/java/com/ververica/flink/table/jdbc/FlinkStatementTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package com.ververica.flink.table.jdbc; 21 | 22 | import org.junit.After; 23 | import org.junit.AfterClass; 24 | import org.junit.Assert; 25 | import org.junit.Before; 26 | import org.junit.BeforeClass; 27 | import org.junit.Test; 28 | 29 | import java.io.File; 30 | import java.net.InetSocketAddress; 31 | import java.sql.Connection; 32 | import java.sql.ResultSet; 33 | import java.sql.SQLException; 34 | import java.sql.Statement; 35 | import java.util.ArrayList; 36 | import java.util.Arrays; 37 | import java.util.List; 38 | 39 | /** 40 | * Tests for {@link FlinkStatement}. 41 | * 42 | *

NOTE: Please clean up newly added tables and databases after each test, 43 | * set current catalog back to default_catalog and set current database back to default_database. 44 | */ 45 | public class FlinkStatementTest { 46 | 47 | private static FlinkJdbcDriverTestingGateway gateway; 48 | private static Connection connection; 49 | private Statement statement; 50 | 51 | @BeforeClass 52 | public static void beforeClass() throws Exception { 53 | gateway = new FlinkJdbcDriverTestingGateway(); 54 | gateway.start(); 55 | 56 | InetSocketAddress addr = gateway.getServerAddress(); 57 | Assert.assertNotNull(addr); 58 | connection = new FlinkConnection("jdbc:flink://localhost:" + addr.getPort() + "?planner=blink"); 59 | Assert.assertTrue(connection.isValid(0)); 60 | } 61 | 62 | @AfterClass 63 | public static void afterClass() throws Exception { 64 | connection.close(); 65 | gateway.stop(); 66 | } 67 | 68 | @Before 69 | public void before() throws SQLException { 70 | statement = connection.createStatement(); 71 | } 72 | 73 | @After 74 | public void after() throws SQLException { 75 | statement.close(); 76 | } 77 | 78 | @Test 79 | public void testExecuteQuery() throws SQLException { 80 | try (ResultSet rs = statement.executeQuery("SELECT * FROM myTable ORDER BY a, b LIMIT 2")) { 81 | Assert.assertTrue(rs.next()); 82 | Assert.assertEquals(22, rs.getInt(1)); 83 | Assert.assertEquals("BBB Hi", rs.getString(2)); 84 | 85 | Assert.assertTrue(rs.next()); 86 | Assert.assertEquals(32, rs.getInt(1)); 87 | Assert.assertEquals("CCC World", rs.getString(2)); 88 | 89 | Assert.assertFalse(rs.next()); 90 | } 91 | } 92 | 93 | @Test 94 | public void testExecuteUpdate() throws Exception { 95 | File tmpFile = File.createTempFile("flink-jdbc-driver-test", ".csv"); 96 | tmpFile.deleteOnExit(); 97 | 98 | int createTableUpdateCount = statement.executeUpdate( 99 | "CREATE TABLE testTable(" + 100 | " fa INT," + 101 | " fb VARCHAR(100)" + 102 | ") WITH (" + 103 | " 'connector.type'='filesystem'," + 104 | " 'connector.path'='file://" + tmpFile.getPath() + "'," + 105 | " 'format.type' = 'csv')"); 106 | // CREATE TABLE is a DDL, according to JDBC Java doc it's update count is 0 107 | Assert.assertEquals(0, createTableUpdateCount); 108 | 109 | int insertUpdateCount = statement.executeUpdate( 110 | "INSERT INTO testTable VALUES (1, 'stra'), (2, 'strb')"); 111 | // TODO change this when gateway supports real update count 112 | Assert.assertEquals(Statement.SUCCESS_NO_INFO, insertUpdateCount); 113 | 114 | try (ResultSet rs = statement.executeQuery("SELECT * FROM testTable ORDER BY fa")) { 115 | Assert.assertTrue(rs.next()); 116 | Assert.assertEquals(1, rs.getInt("fa")); 117 | Assert.assertEquals("stra", rs.getString("fb")); 118 | 119 | Assert.assertTrue(rs.next()); 120 | Assert.assertEquals(2, rs.getInt("fa")); 121 | Assert.assertEquals("strb", rs.getString("fb")); 122 | 123 | Assert.assertFalse(rs.next()); 124 | } 125 | 126 | int dropTableUpdateCount = statement.executeUpdate("DROP TABLE testTable"); 127 | // DROP TABLE is a DDL, according to JDBC Java doc it's update count is 0 128 | Assert.assertEquals(0, dropTableUpdateCount); 129 | } 130 | 131 | @Test 132 | public void testMultipleStatements() throws Exception { 133 | File tmpFile1 = File.createTempFile("flink-jdbc-driver-test", ".csv"); 134 | File tmpFile2 = File.createTempFile("flink-jdbc-driver-test", ".csv"); 135 | tmpFile1.deleteOnExit(); 136 | tmpFile2.deleteOnExit(); 137 | 138 | boolean executeIsQuery = statement.execute("CREATE TABLE testTable1(" + 139 | " fa INT," + 140 | " fb VARCHAR(100)" + 141 | ") WITH (" + 142 | " 'connector.type'='filesystem'," + 143 | " 'connector.path'='file://" + tmpFile1.getPath() + "'," + 144 | " 'format.type' = 'csv');" + 145 | "INSERT INTO testTable1 VALUES (1, 'stra'), (2, 'strb');" + 146 | "SELECT * FROM testTable1 ORDER BY fa;" + 147 | 148 | "CREATE TABLE testTable2(" + 149 | " fc INT," + 150 | " fd VARCHAR(100)" + 151 | ") WITH (" + 152 | " 'connector.type'='filesystem'," + 153 | " 'connector.path'='file://" + tmpFile2.getPath() + "'," + 154 | " 'format.type' = 'csv');" + 155 | "INSERT INTO testTable2(fc, fd) SELECT * FROM testTable1;" + 156 | "SELECT * FROM testTable2 ORDER BY fc;" + 157 | 158 | "DROP TABLE testTable1;" + 159 | "DROP TABLE testTable2;"); 160 | 161 | Assert.assertFalse(executeIsQuery); 162 | // CREATE TABLE is a DDL, according to JDBC Java doc it's update count is 0 163 | Assert.assertEquals(0, statement.getUpdateCount()); 164 | 165 | Assert.assertFalse(statement.getMoreResults()); 166 | // TODO change this when gateway supports real update count 167 | Assert.assertEquals(Statement.SUCCESS_NO_INFO, statement.getUpdateCount()); 168 | 169 | Assert.assertTrue(statement.getMoreResults()); 170 | ResultSet rs1 = statement.getResultSet(); 171 | Assert.assertTrue(rs1.next()); 172 | Assert.assertEquals(1, rs1.getInt("fa")); 173 | Assert.assertEquals("stra", rs1.getString("fb")); 174 | Assert.assertTrue(rs1.next()); 175 | Assert.assertEquals(2, rs1.getInt("fa")); 176 | Assert.assertEquals("strb", rs1.getString("fb")); 177 | Assert.assertFalse(rs1.next()); 178 | 179 | Assert.assertFalse(statement.getMoreResults()); 180 | // CREATE TABLE is a DDL, according to JDBC Java doc it's update count is 0 181 | Assert.assertEquals(0, statement.getUpdateCount()); 182 | 183 | Assert.assertFalse(statement.getMoreResults()); 184 | // TODO change this when gateway supports real update count 185 | Assert.assertEquals(Statement.SUCCESS_NO_INFO, statement.getUpdateCount()); 186 | 187 | Assert.assertTrue(statement.getMoreResults()); 188 | ResultSet rs2 = statement.getResultSet(); 189 | Assert.assertTrue(rs2.next()); 190 | Assert.assertEquals(1, rs2.getInt("fc")); 191 | Assert.assertEquals("stra", rs2.getString("fd")); 192 | Assert.assertTrue(rs2.next()); 193 | Assert.assertEquals(2, rs2.getInt("fc")); 194 | Assert.assertEquals("strb", rs2.getString("fd")); 195 | Assert.assertFalse(rs2.next()); 196 | 197 | Assert.assertFalse(statement.getMoreResults()); 198 | // DROP TABLE is a DDL, according to JDBC Java doc it's update count is 0 199 | Assert.assertEquals(0, statement.getUpdateCount()); 200 | 201 | Assert.assertFalse(statement.getMoreResults()); 202 | // DROP TABLE is a DDL, according to JDBC Java doc it's update count is 0 203 | Assert.assertEquals(0, statement.getUpdateCount()); 204 | 205 | Assert.assertFalse(statement.getMoreResults()); 206 | Assert.assertEquals(-1, statement.getUpdateCount()); 207 | } 208 | 209 | @Test 210 | public void testShows() throws Exception { 211 | compareStringResultsWithSorting( 212 | new String[]{"default_catalog", "cat1", "cat2"}, statement.executeQuery("SHOW CATALOGS")); 213 | 214 | statement.execute("USE CATALOG cat1"); 215 | statement.execute("CREATE DATABASE db12"); 216 | compareStringResultsWithSorting( 217 | new String[]{"db11", "db12"}, statement.executeQuery("SHOW DATABASES")); 218 | 219 | statement.execute("USE db11"); 220 | compareStringResultsWithSorting(new String[]{"cat1"}, statement.executeQuery("SHOW CURRENT CATALOG")); 221 | compareStringResultsWithSorting(new String[]{"db11"}, statement.executeQuery("SHOW CURRENT DATABASE")); 222 | 223 | File tmpFile1 = File.createTempFile("flink-jdbc-driver-test", ".csv"); 224 | File tmpFile2 = File.createTempFile("flink-jdbc-driver-test", ".csv"); 225 | tmpFile1.deleteOnExit(); 226 | tmpFile2.deleteOnExit(); 227 | 228 | statement.executeUpdate("CREATE TABLE testTable1(" + 229 | " fa INT," + 230 | " fb VARCHAR(100)" + 231 | ") WITH (" + 232 | " 'connector.type'='filesystem'," + 233 | " 'connector.path'='file://" + tmpFile1.getPath() + "'," + 234 | " 'format.type' = 'csv');"); 235 | statement.executeUpdate("CREATE TABLE testTable2(" + 236 | " fc INT," + 237 | " fd VARCHAR(100)" + 238 | ") WITH (" + 239 | " 'connector.type'='filesystem'," + 240 | " 'connector.path'='file://" + tmpFile2.getPath() + "'," + 241 | " 'format.type' = 'csv');"); 242 | compareStringResultsWithSorting( 243 | new String[]{"testTable1", "testTable2"}, statement.executeQuery("SHOW TABLES")); 244 | 245 | statement.executeUpdate("DROP TABLE testTable1"); 246 | statement.executeUpdate("DROP TABLE testTable2"); 247 | statement.executeUpdate("DROP DATABASE db12"); 248 | statement.executeUpdate("USE CATALOG default_catalog"); 249 | } 250 | 251 | @Test 252 | public void testMaxRows() throws SQLException { 253 | // max rows is smaller than actual result count 254 | statement.setMaxRows(2); 255 | try (ResultSet rs = statement.executeQuery("SELECT * FROM myTable ORDER BY a, b")) { 256 | Assert.assertTrue(rs.next()); 257 | Assert.assertEquals(22, rs.getInt(1)); 258 | Assert.assertEquals("BBB Hi", rs.getString(2)); 259 | Assert.assertTrue(rs.next()); 260 | Assert.assertEquals(32, rs.getInt(1)); 261 | Assert.assertEquals("CCC World", rs.getString(2)); 262 | Assert.assertFalse(rs.next()); 263 | } 264 | 265 | // max rows is larger than actual result count 266 | statement.setMaxRows(5); 267 | try (ResultSet rs = statement.executeQuery("SELECT * FROM myTable ORDER BY a, b LIMIT 2")) { 268 | Assert.assertTrue(rs.next()); 269 | Assert.assertEquals(22, rs.getInt(1)); 270 | Assert.assertEquals("BBB Hi", rs.getString(2)); 271 | Assert.assertTrue(rs.next()); 272 | Assert.assertEquals(32, rs.getInt(1)); 273 | Assert.assertEquals("CCC World", rs.getString(2)); 274 | Assert.assertFalse(rs.next()); 275 | } 276 | } 277 | 278 | private void compareStringResultsWithSorting(String[] expected, ResultSet actualResultSet) throws SQLException { 279 | Arrays.sort(expected); 280 | 281 | List actualList = new ArrayList<>(); 282 | for (int i = 0; i < expected.length; i++) { 283 | Assert.assertTrue(actualResultSet.next()); 284 | actualList.add(actualResultSet.getString(1)); 285 | } 286 | String[] actual = actualList.toArray(new String[0]); 287 | Arrays.sort(actual); 288 | 289 | Assert.assertArrayEquals(expected, actual); 290 | } 291 | } 292 | -------------------------------------------------------------------------------- /src/test/resources/default-env.yaml: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | tables: 20 | - name: myTable 21 | type: source-table 22 | update-mode: append 23 | schema: 24 | - name: a 25 | data-type: INT 26 | - name: b 27 | data-type: VARCHAR(100) 28 | connector: 29 | type: filesystem 30 | path: "$VAR_SOURCE_PATH" 31 | format: 32 | type: csv 33 | fields: 34 | - name: a 35 | data-type: INT 36 | - name: b 37 | data-type: VARCHAR(100) 38 | line-delimiter: "\n" 39 | comment-prefix: "#" 40 | 41 | functions: [] 42 | 43 | catalogs: 44 | - name: cat1 45 | type: generic_in_memory 46 | default-database: db11 47 | - name: cat2 48 | type: generic_in_memory 49 | default-database: db21 50 | 51 | execution: 52 | planner: blink 53 | type: batch 54 | result-mode: table 55 | current-catalog: default_catalog 56 | current-database: default_database 57 | 58 | deployment: 59 | response-timeout: 5000 60 | gateway-address: "" 61 | gateway-port: 0 62 | -------------------------------------------------------------------------------- /src/test/resources/test-data.csv: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 42,AAA Hello 19 | 22,BBB Hi 20 | 32,CCC World 21 | 32,DDD Hello!!!! 22 | 42,EEE Hi!!!! 23 | 52,FFF World!!!! 24 | -------------------------------------------------------------------------------- /tools/maven/checkstyle.xml: -------------------------------------------------------------------------------- 1 | 2 | 20 | 23 | 24 | 31 | 32 | 33 | 34 | 35 | 36 | 37 | 38 | 39 | 40 | 41 | 42 | 43 | 44 | 45 | 46 | 47 | 48 | 49 | 50 | 51 | 52 | 53 | 54 | 55 | 56 | 57 | 58 | 59 | 60 | 61 | 62 | 63 | 64 | 65 | 66 | 67 | 68 | 69 | 70 | 71 | 72 | 73 | 74 | 79 | 80 | 81 | 82 | 83 | 84 | 85 | 86 | 87 | 88 | 89 | 90 | 91 | 92 | 93 | 94 | 99 | 100 | 101 | 102 | 104 | 105 | 106 | 107 | 108 | 109 | 110 | 111 | 112 | 113 | 114 | 115 | 116 | 117 | 118 | 119 | 120 | 121 | 122 | 123 | 124 | 125 | 126 | 131 | 132 | 133 | 134 | 135 | 136 | 137 | 138 | 139 | 140 | 141 | 142 | 143 | 144 | 145 | 146 | 147 | 148 | 149 | 150 | 151 | 152 | 153 | 154 | 155 | 156 | 157 | 158 | 159 | 160 | 161 | 162 | 163 | 164 | 165 | 166 | 167 | 168 | 169 | 170 | 171 | 172 | 173 | 174 | 175 | 176 | 177 | 178 | 179 | 180 | 181 | 182 | 183 | 184 | 185 | 190 | 191 | 192 | 193 | 194 | 196 | 197 | 198 | 199 | 200 | 201 | 202 | 203 | 204 | 205 | 206 | 208 | 209 | 210 | 211 | 212 | 213 | 214 | 215 | 216 | 217 | 218 | 219 | 224 | 225 | 226 | 227 | 231 | 232 | 233 | 234 | 235 | 238 | 239 | 240 | 241 | 242 | 243 | 244 | 245 | 246 | 248 | 249 | 250 | 255 | 256 | 257 | 258 | 259 | 260 | 261 | 262 | 263 | 264 | 265 | 266 | 267 | 268 | 269 | 270 | 271 | 272 | 273 | 274 | 275 | 276 | 277 | 278 | 279 | 280 | 281 | 282 | 283 | 284 | 285 | 290 | 291 | 292 | 293 | 294 | 296 | 299 | 300 | 301 | 302 | 303 | 304 | 306 | 307 | 308 | 309 | 310 | 311 | 313 | 314 | 315 | 316 | 317 | 318 | 319 | 321 | 322 | 323 | 324 | 325 | 327 | 328 | 329 | 330 | 331 | 332 | 333 | 334 | 335 | 336 | 337 | 338 | 339 | 340 | 341 | 342 | 343 | 344 | 345 | 346 | 347 | 348 | 349 | 350 | 351 | 352 | 353 | 354 | 355 | 357 | 358 | 359 | 360 | 361 | 363 | 364 | 365 | 366 | 367 | 369 | 370 | 371 | 372 | 375 | 376 | 377 | 378 | 379 | 380 | 381 | 382 | 383 | 384 | 385 | 386 | 387 | 388 | 389 | 390 | 395 | 396 | 397 | 398 | 399 | 400 | 401 | 402 | 403 | 404 | 405 | 406 | 407 | 408 | 409 | 410 | 411 | 412 | 413 | 414 | 415 | 416 | 417 | 418 | 419 | 420 | 421 | 422 | 423 | 424 | 425 | 426 | 427 | 428 | 429 | 430 | 431 | 432 | 433 | 434 | 435 | 436 | 437 | 438 | 439 | 440 | 441 | 442 | 443 | 444 | 445 | 446 | 447 | 448 | 449 | 454 | 456 | 457 | 458 | 459 | 460 | 461 | 462 | 463 | 464 | 465 | 466 | 467 | 468 | 469 | 470 | 471 | 472 | 477 | 478 | 479 | 484 | 485 | 486 | 487 | 488 | 493 | 494 | 495 | 499 | 500 | 501 | 504 | 505 | 506 | 507 | 511 | 518 | 519 | 520 | 521 | 522 | 525 | 526 | 527 | 528 | 529 | 532 | 534 | 535 | 536 | 537 | 538 | 539 | 542 | 543 | 544 | 545 | 546 | 547 | 548 | 549 | 550 | 551 | 552 | 553 | 554 | 555 | 556 | 557 | 558 | 559 | 560 | 561 | 562 | 563 | 564 | 567 | 568 | 569 | 570 | 571 | 572 | -------------------------------------------------------------------------------- /tools/maven/suppressions.xml: -------------------------------------------------------------------------------- 1 | 2 | 20 | 21 | 24 | 25 | 26 | 27 | --------------------------------------------------------------------------------