├── .gitignore ├── .travis.yml ├── LICENSE ├── README.md ├── _config.yml ├── pom.xml ├── scalastyle-config.xml └── src ├── main ├── resources │ └── META-INF │ │ └── services │ │ └── org.apache.spark.sql.sources.DataSourceRegister └── scala │ └── org │ └── apache │ └── spark │ └── sql │ └── execution │ └── datasources │ └── greenplum │ ├── DefaultSource.scala │ ├── DefaultSource15.scala │ ├── GreenplumOptions.scala │ ├── GreenplumRelation.scala │ ├── GreenplumUtils.scala │ └── PartitionCopyFailureException.scala └── test └── scala └── org └── apache └── spark └── sql └── execution └── datasources └── greenplum ├── GreenplumOptionsSuite.scala └── GreenplumUtilsSuite.scala /.gitignore: -------------------------------------------------------------------------------- 1 | *#*# 2 | *.#* 3 | *.iml 4 | *.ipr 5 | *.iws 6 | *.pyc 7 | *.pyo 8 | *.swp 9 | *~ 10 | .DS_Store 11 | .cache 12 | .classpath 13 | .ensime 14 | .ensime_cache/ 15 | .ensime_lucene 16 | .generated-mima* 17 | .idea/ 18 | .idea_modules/ 19 | .project 20 | .pydevproject 21 | .scala_dependencies 22 | .settings 23 | /lib/ 24 | R-unit-tests.log 25 | R/unit-tests.out 26 | R/cran-check.out 27 | R/pkg/vignettes/sparkr-vignettes.html 28 | R/pkg/tests/fulltests/Rplots.pdf 29 | build/*.jar 30 | build/apache-maven* 31 | build/scala* 32 | build/zinc* 33 | cache 34 | checkpoint 35 | conf/*.cmd 36 | conf/*.conf 37 | conf/*.properties 38 | conf/*.sh 39 | conf/*.xml 40 | conf/java-opts 41 | conf/slaves 42 | dependency-reduced-pom.xml 43 | derby.log 44 | dev/create-release/*final 45 | dev/create-release/*txt 46 | dev/pr-deps/ 47 | dist/ 48 | docs/_site 49 | docs/api 50 | sql/docs 51 | sql/site 52 | lib_managed/ 53 | lint-r-report.log 54 | log/ 55 | logs/ 56 | out/ 57 | project/boot/ 58 | project/build/target/ 59 | project/plugins/lib_managed/ 60 | project/plugins/project/build.properties 61 | project/plugins/src_managed/ 62 | project/plugins/target/ 63 | python/lib/pyspark.zip 64 | python/deps 65 | python/pyspark/python 66 | reports/ 67 | scalastyle-on-compile.generated.xml 68 | scalastyle-output.xml 69 | scalastyle.txt 70 | spark-*-bin-*.tgz 71 | spark-tests.log 72 | src_managed/ 73 | streaming-tests.log 74 | target/ 75 | unit-tests.log 76 | work/ 77 | 78 | # For Hive 79 | TempStatsStore/ 80 | metastore/ 81 | metastore_db/ 82 | sql/hive-thriftserver/test_warehouses 83 | warehouse/ 84 | spark-warehouse/ 85 | 86 | # For R session data 87 | .RData 88 | .RHistory 89 | .Rhistory 90 | *.Rproj 91 | *.Rproj.* 92 | 93 | .Rproj.user 94 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: scala 2 | scala: 3 | - 2.11.8 4 | 5 | cache: 6 | directories: 7 | - $HOME/.m2 8 | - ./build 9 | 10 | before_deploy: 11 | - mvn clean package -DskipTests 12 | 13 | deploy: 14 | - provider: pages 15 | skip_cleanup: true 16 | github_token: $GITHUB_TOKEN 17 | email: yaooqinn@hotmail.com 18 | name: Kent Yao 19 | on: 20 | branch: master 21 | - provider: releases 22 | api_key: $GITHUB_TOKEN 23 | file_glob: true 24 | file: target/spark-greenplum*.jar 25 | skip_cleanup: true 26 | on: 27 | tags: true 28 | 29 | jobs: 30 | include: 31 | - stage: spark2.4 32 | language: scala 33 | script: mvn clean install -Pspark-2.4 -Dmaven.javadoc.skip=true -B -V 34 | - stage: spark2.3 35 | language: scala 36 | script: mvn clean install -Dmaven.javadoc.skip=true -B -V 37 | 38 | after_success: 39 | - bash <(curl -s https://codecov.io/bash) 40 | -------------------------------------------------------------------------------- /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 | 203 | 204 | ======================================================================= 205 | Apache Spark Subcomponents: 206 | 207 | The Apache Spark project contains subcomponents with separate copyright 208 | notices and license terms. Your use of the source code for the these 209 | subcomponents is subject to the terms and conditions of the following 210 | licenses. 211 | 212 | 213 | ======================================================================== 214 | For heapq (pyspark/heapq3.py): 215 | ======================================================================== 216 | 217 | See license/LICENSE-heapq.txt 218 | 219 | ======================================================================== 220 | For SnapTree: 221 | ======================================================================== 222 | 223 | See license/LICENSE-SnapTree.txt 224 | 225 | ======================================================================== 226 | For jbcrypt: 227 | ======================================================================== 228 | 229 | See license/LICENSE-jbcrypt.txt 230 | 231 | ======================================================================== 232 | BSD-style licenses 233 | ======================================================================== 234 | 235 | The following components are provided under a BSD-style license. See project link for details. 236 | The text of each license is also included at licenses/LICENSE-[project].txt. 237 | 238 | (BSD 3 Clause) netlib core (com.github.fommil.netlib:core:1.1.2 - https://github.com/fommil/netlib-java/core) 239 | (BSD 3 Clause) JPMML-Model (org.jpmml:pmml-model:1.2.7 - https://github.com/jpmml/jpmml-model) 240 | (BSD License) AntLR Parser Generator (antlr:antlr:2.7.7 - http://www.antlr.org/) 241 | (BSD License) ANTLR 4.5.2-1 (org.antlr:antlr4:4.5.2-1 - http://wwww.antlr.org/) 242 | (BSD licence) ANTLR ST4 4.0.4 (org.antlr:ST4:4.0.4 - http://www.stringtemplate.org) 243 | (BSD licence) ANTLR StringTemplate (org.antlr:stringtemplate:3.2.1 - http://www.stringtemplate.org) 244 | (BSD License) Javolution (javolution:javolution:5.5.1 - http://javolution.org) 245 | (BSD) JLine (jline:jline:0.9.94 - http://jline.sourceforge.net) 246 | (BSD) ParaNamer Core (com.thoughtworks.paranamer:paranamer:2.3 - http://paranamer.codehaus.org/paranamer) 247 | (BSD) ParaNamer Core (com.thoughtworks.paranamer:paranamer:2.6 - http://paranamer.codehaus.org/paranamer) 248 | (BSD 3 Clause) Scala (http://www.scala-lang.org/download/#License) 249 | (Interpreter classes (all .scala files in repl/src/main/scala 250 | except for Main.Scala, SparkHelper.scala and ExecutorClassLoader.scala), 251 | and for SerializableMapWrapper in JavaUtils.scala) 252 | (BSD-like) Scala Actors library (org.scala-lang:scala-actors:2.11.7 - http://www.scala-lang.org/) 253 | (BSD-like) Scala Compiler (org.scala-lang:scala-compiler:2.11.7 - http://www.scala-lang.org/) 254 | (BSD-like) Scala Compiler (org.scala-lang:scala-reflect:2.11.7 - http://www.scala-lang.org/) 255 | (BSD-like) Scala Library (org.scala-lang:scala-library:2.11.7 - http://www.scala-lang.org/) 256 | (BSD-like) Scalap (org.scala-lang:scalap:2.11.7 - http://www.scala-lang.org/) 257 | (BSD-style) scalacheck (org.scalacheck:scalacheck_2.11:1.10.0 - http://www.scalacheck.org) 258 | (BSD-style) spire (org.spire-math:spire_2.11:0.7.1 - http://spire-math.org) 259 | (BSD-style) spire-macros (org.spire-math:spire-macros_2.11:0.7.1 - http://spire-math.org) 260 | (New BSD License) Kryo (com.esotericsoftware:kryo:3.0.3 - https://github.com/EsotericSoftware/kryo) 261 | (New BSD License) MinLog (com.esotericsoftware:minlog:1.3.0 - https://github.com/EsotericSoftware/minlog) 262 | (New BSD license) Protocol Buffer Java API (com.google.protobuf:protobuf-java:2.5.0 - http://code.google.com/p/protobuf) 263 | (New BSD license) Protocol Buffer Java API (org.spark-project.protobuf:protobuf-java:2.4.1-shaded - http://code.google.com/p/protobuf) 264 | (The BSD License) Fortran to Java ARPACK (net.sourceforge.f2j:arpack_combined_all:0.1 - http://f2j.sourceforge.net) 265 | (The BSD License) xmlenc Library (xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net) 266 | (The New BSD License) Py4J (net.sf.py4j:py4j:0.10.4 - http://py4j.sourceforge.net/) 267 | (Two-clause BSD-style license) JUnit-Interface (com.novocode:junit-interface:0.10 - http://github.com/szeiger/junit-interface/) 268 | (BSD licence) sbt and sbt-launch-lib.bash 269 | (BSD 3 Clause) d3.min.js (https://github.com/mbostock/d3/blob/master/LICENSE) 270 | (BSD 3 Clause) DPark (https://github.com/douban/dpark/blob/master/LICENSE) 271 | (BSD 3 Clause) CloudPickle (https://github.com/cloudpipe/cloudpickle/blob/master/LICENSE) 272 | 273 | ======================================================================== 274 | MIT licenses 275 | ======================================================================== 276 | 277 | The following components are provided under the MIT License. See project link for details. 278 | The text of each license is also included at licenses/LICENSE-[project].txt. 279 | 280 | (MIT License) JCL 1.1.1 implemented over SLF4J (org.slf4j:jcl-over-slf4j:1.7.5 - http://www.slf4j.org) 281 | (MIT License) JUL to SLF4J bridge (org.slf4j:jul-to-slf4j:1.7.5 - http://www.slf4j.org) 282 | (MIT License) SLF4J API Module (org.slf4j:slf4j-api:1.7.5 - http://www.slf4j.org) 283 | (MIT License) SLF4J LOG4J-12 Binding (org.slf4j:slf4j-log4j12:1.7.5 - http://www.slf4j.org) 284 | (MIT License) pyrolite (org.spark-project:pyrolite:2.0.1 - http://pythonhosted.org/Pyro4/) 285 | (MIT License) scopt (com.github.scopt:scopt_2.11:3.2.0 - https://github.com/scopt/scopt) 286 | (The MIT License) Mockito (org.mockito:mockito-core:1.9.5 - http://www.mockito.org) 287 | (MIT License) jquery (https://jquery.org/license/) 288 | (MIT License) AnchorJS (https://github.com/bryanbraun/anchorjs) 289 | (MIT License) graphlib-dot (https://github.com/cpettitt/graphlib-dot) 290 | (MIT License) dagre-d3 (https://github.com/cpettitt/dagre-d3) 291 | (MIT License) sorttable (https://github.com/stuartlangridge/sorttable) 292 | (MIT License) boto (https://github.com/boto/boto/blob/develop/LICENSE) 293 | (MIT License) datatables (http://datatables.net/license) 294 | (MIT License) mustache (https://github.com/mustache/mustache/blob/master/LICENSE) 295 | (MIT License) cookies (http://code.google.com/p/cookies/wiki/License) 296 | (MIT License) blockUI (http://jquery.malsup.com/block/) 297 | (MIT License) RowsGroup (http://datatables.net/license/mit) 298 | (MIT License) jsonFormatter (http://www.jqueryscript.net/other/jQuery-Plugin-For-Pretty-JSON-Formatting-jsonFormatter.html) 299 | (MIT License) modernizr (https://github.com/Modernizr/Modernizr/blob/master/LICENSE) 300 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # PostgreSQL & GreenPlum Data Source for Apache Spark [![License](https://img.shields.io/badge/license-Apache%202-4EB1BA.svg)](https://www.apache.org/licenses/LICENSE-2.0.html) [![](https://tokei.rs/b1/github/yaooqinn/spark-greenplum)](https://github.com/yaooqinn/spark-greenplum) [![GitHub release](https://img.shields.io/github/release/yaooqinn/spark-greenplum.svg)](https://github.com/yaooqinn/spark-greenplum/releases) [![codecov](https://codecov.io/gh/yaooqinn/spark-greenplum/branch/master/graph/badge.svg)](https://codecov.io/gh/yaooqinn/spark-greenplum) [![Build Status](https://travis-ci.com/yaooqinn/spark-greenplum.svg?branch=master)](https://travis-ci.com/yaooqinn/spark-greenplum)[![HitCount](http://hits.dwyl.io/yaooqinn/spark-greenplum.svg)](http://hits.dwyl.io/yaooqinn/spark-greenplum) 2 | 3 | A library for reading data from and transferring data to Greenplum databases with Apache Spark, for Spark SQL and DataFrames. 4 | 5 | This library is **100x faster** than Apache Spark's JDBC DataSource while transferring data from Spark to Greenpum databases. 6 | 7 | Also, this library is fully **transactional** . 8 | 9 | ## Try it now ! 10 | 11 | ### CTAS 12 | ```genericsql 13 | CREATE TABLE tbl 14 | USING greenplum 15 | options ( 16 | url "jdbc:postgresql://greenplum:5432/", 17 | delimiter "\t", 18 | dbschema "gptest", 19 | dbtable "store_sales", 20 | user 'gptest', 21 | password 'test') 22 | AS 23 | SELECT * FROM tpcds_100g.store_sales WHERE ss_sold_date_sk<=2451537 AND ss_sold_date_sk> 2451520; 24 | ``` 25 | 26 | ### View & Insert 27 | 28 | ```genericsql 29 | CREATE TEMPORARY TABLE tbl 30 | USING greenplum 31 | options ( 32 | url "jdbc:postgresql://greenplum:5432/", 33 | delimiter "\t", 34 | dbschema "gptest", 35 | dbtable "store_sales", 36 | user 'gptest', 37 | password 'test') 38 | 39 | INSERT INTO TABLE tbl SELECT * FROM tpcds_100g.store_sales WHERE ss_sold_date_sk<=2451537 AND ss_sold_date_sk> 2451520; 40 | 41 | ``` 42 | 43 | Please refer to [Spark SQL Guide - JDBC To Other Databases](http://spark.apache.org/docs/latest/sql-data-sources-jdbc.html) to learn more about the similar usage. 44 | -------------------------------------------------------------------------------- /_config.yml: -------------------------------------------------------------------------------- 1 | theme: jekyll-theme-leap-day 2 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | 7 | org.apache.spark 8 | spark-greenplum 9 | 1.0-SNAPSHOT 10 | jar 11 | 12 | 13 | 14 | yaooqinn 15 | Kent Yao 16 | NetEase Corp. 17 | yaooqinn@hotmail.com 18 | 19 | 20 | 21 | NetEase Corp. 22 | https://bigdata.163yun.com/mammut 23 | 24 | 25 | 26 | The Apache Software License, Version 2.0 27 | http://www.apache.org/licenses/LICENSE-2.0.txt 28 | manual 29 | 30 | 31 | 32 | 33 | 42.3.8 34 | 2.11 35 | 2.11.8 36 | 3.0.3 37 | 2.3.3 38 | 39 | 40 | 41 | 42 | org.scala-lang 43 | scala-library 44 | ${scala.ver} 45 | provided 46 | 47 | 48 | org.apache.spark 49 | spark-sql_${scala.binary.ver} 50 | ${spark.ver} 51 | provided 52 | 53 | 54 | org.postgresql 55 | postgresql 56 | ${postgresql.ver} 57 | compile 58 | 59 | 60 | 61 | 62 | org.ostermiller 63 | utils 64 | 1.07.00 65 | compile 66 | 67 | 68 | 69 | 70 | 71 | org.scalatest 72 | scalatest_${scala.binary.ver} 73 | ${scalatest.ver} 74 | test 75 | 76 | 77 | org.apache.spark 78 | spark-core_${scala.binary.ver} 79 | ${spark.ver} 80 | test 81 | test-jar 82 | 83 | 84 | 85 | io.airlift 86 | testing-postgresql-server 87 | 9.6.3-3 88 | test 89 | 90 | 91 | org.mockito 92 | mockito-core 93 | 1.10.19 94 | test 95 | 96 | 97 | 98 | target/scala-${scala.binary.ver}/classes 99 | target/scala-${scala.binary.ver}/test-classes 100 | 101 | 102 | ${project.basedir}/src/main/resources 103 | 104 | 105 | 106 | ${project.build.directory}/extra-resources 107 | true 108 | 109 | 110 | 111 | 112 | 113 | org.apache.maven.plugins 114 | maven-compiler-plugin 115 | 3.5.1 116 | 117 | ${java.version} 118 | ${java.version} 119 | UTF-8 120 | 1024m 121 | true 122 | 123 | -Xlint:all,-serial,-path 124 | 125 | 126 | 127 | 128 | 129 | net.alchim31.maven 130 | scala-maven-plugin 131 | 3.3.1 132 | 133 | 134 | eclipse-add-source 135 | 136 | add-source 137 | 138 | 139 | 140 | scala-compile-first 141 | 142 | compile 143 | 144 | 145 | 146 | scala-test-compile-first 147 | 148 | testCompile 149 | 150 | 151 | 152 | 153 | ${scala.ver} 154 | incremental 155 | true 156 | 157 | -unchecked 158 | -deprecation 159 | -feature 160 | -explaintypes 161 | -Yno-adapted-args 162 | 163 | 164 | -Xms1024m 165 | -Xmx1024m 166 | -XX:ReservedCodeCacheSize=512M 167 | 168 | 169 | -source 170 | ${java.version} 171 | -target 172 | ${java.version} 173 | -Xlint:all,-serial,-path,-try 174 | 175 | 176 | 177 | 178 | 179 | 180 | org.apache.maven.plugins 181 | maven-surefire-plugin 182 | 2.12.4 183 | 184 | true 185 | 186 | 187 | 188 | 189 | org.scalatest 190 | scalatest-maven-plugin 191 | 1.0 192 | 193 | ${project.build.directory}/surefire-reports 194 | . 195 | TestSuite.txt 196 | 197 | 198 | 199 | test 200 | 201 | test 202 | 203 | 204 | 205 | 206 | 207 | 208 | org.jacoco 209 | jacoco-maven-plugin 210 | 0.8.0 211 | 212 | 213 | pre-test 214 | 215 | prepare-agent 216 | 217 | 218 | 219 | report 220 | test 221 | 222 | report 223 | 224 | 225 | 226 | 227 | 228 | 229 | org.apache.maven.plugins 230 | maven-shade-plugin 231 | 232 | false 233 | 234 | 235 | org.ostermiller:* 236 | org.postgresql:* 237 | 238 | 239 | 240 | 241 | com.Ostermiller 242 | spark_greenplum_project.Ostermiller 243 | 244 | 245 | 246 | 247 | 248 | 249 | package 250 | 251 | shade 252 | 253 | 254 | 255 | 256 | 257 | 258 | 259 | 260 | 261 | spark-2.4 262 | 263 | 2.4.0 264 | 265 | 266 | 267 | -------------------------------------------------------------------------------- /scalastyle-config.xml: -------------------------------------------------------------------------------- 1 | 17 | 39 | 40 | 41 | Scalastyle standard configuration 42 | 43 | 44 | 45 | 46 | 47 | 48 | 49 | 50 | 51 | 67 | 68 | 69 | 70 | 71 | 72 | 73 | 74 | 75 | 76 | 77 | 78 | 79 | 80 | true 81 | 82 | 83 | 84 | 85 | 86 | 87 | 88 | 89 | 90 | 91 | 92 | 93 | 94 | 95 | 96 | 97 | 98 | 99 | 100 | 101 | 102 | 103 | 104 | 105 | 106 | 107 | 108 | 109 | 110 | 111 | 112 | 113 | 114 | 115 | 116 | 117 | 118 | 119 | 120 | 121 | 122 | 123 | 124 | 125 | ARROW, EQUALS, ELSE, TRY, CATCH, FINALLY, LARROW, RARROW 126 | 127 | 128 | 129 | 130 | 131 | ARROW, EQUALS, COMMA, COLON, IF, ELSE, DO, WHILE, FOR, MATCH, TRY, CATCH, FINALLY, LARROW, RARROW 132 | 133 | 134 | 135 | 136 | 137 | 138 | 139 | 140 | ^FunSuite[A-Za-z]*$ 141 | Tests must extend org.apache.spark.SparkFunSuite instead. 142 | 143 | 144 | 145 | 146 | ^println$ 147 | 151 | 152 | 153 | 154 | @VisibleForTesting 155 | 158 | 159 | 160 | 161 | Runtime\.getRuntime\.addShutdownHook 162 | 170 | 171 | 172 | 173 | mutable\.SynchronizedBuffer 174 | 182 | 183 | 184 | 185 | Class\.forName 186 | 193 | 194 | 195 | 196 | Await\.result 197 | 204 | 205 | 206 | 207 | Await\.ready 208 | 215 | 216 | 217 | 218 | 219 | JavaConversions 220 | Instead of importing implicits in scala.collection.JavaConversions._, import 221 | scala.collection.JavaConverters._ and use .asScala / .asJava methods 222 | 223 | 224 | 225 | org\.apache\.commons\.lang\. 226 | Use Commons Lang 3 classes (package org.apache.commons.lang3.*) instead 227 | of Commons Lang 2 (package org.apache.commons.lang.*) 228 | 229 | 230 | 231 | extractOpt 232 | Use Utils.jsonOption(x).map(.extract[T]) instead of .extractOpt[T], as the latter 233 | is slower. 234 | 235 | 236 | 237 | 238 | java,scala,3rdParty,spark 239 | javax?\..* 240 | scala\..* 241 | (?!org\.apache\.spark\.).* 242 | org\.apache\.spark\..* 243 | 244 | 245 | 246 | 247 | 248 | COMMA 249 | 250 | 251 | 252 | 253 | 254 | \)\{ 255 | 258 | 259 | 260 | 261 | (?m)^(\s*)/[*][*].*$(\r|)\n^\1 [*] 262 | Use Javadoc style indentation for multiline comments 263 | 264 | 265 | 266 | case[^\n>]*=>\s*\{ 267 | Omit braces in case clauses. 268 | 269 | 270 | 271 | 272 | 273 | 274 | 275 | 276 | 277 | 278 | 279 | 280 | 281 | 282 | 283 | 284 | 285 | 286 | 287 | 288 | 289 | 290 | 291 | 292 | 293 | 294 | 295 | 296 | 297 | 298 | 299 | 300 | 301 | 302 | 303 | 304 | 305 | 306 | 307 | 308 | 309 | 310 | 311 | 312 | 313 | 314 | 315 | 316 | 317 | 318 | 319 | 320 | 800> 321 | 322 | 323 | 324 | 325 | 30 326 | 327 | 328 | 329 | 330 | 10 331 | 332 | 333 | 334 | 335 | 50 336 | 337 | 338 | 339 | 340 | 341 | 342 | 343 | 344 | 345 | 346 | -1,0,1,2,3 347 | 348 | 349 | 350 | -------------------------------------------------------------------------------- /src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister: -------------------------------------------------------------------------------- 1 | org.apache.spark.sql.execution.datasources.greenplum.DefaultSource15 -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/execution/datasources/greenplum/DefaultSource.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.sql.execution.datasources.greenplum 19 | 20 | import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SQLContext} 21 | import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap 22 | import org.apache.spark.sql.execution.datasources.jdbc._ 23 | import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, DataSourceRegister, RelationProvider} 24 | import org.apache.spark.sql.types.StructType 25 | 26 | class DefaultSource 27 | extends RelationProvider with CreatableRelationProvider with DataSourceRegister { 28 | 29 | import GreenplumUtils._ 30 | 31 | override def shortName(): String = "greenplum" 32 | 33 | override def createRelation( 34 | sqlContext: SQLContext, 35 | parameters: Map[String, String]): BaseRelation = { 36 | import JDBCOptions._ 37 | 38 | val options = GreenplumOptions(CaseInsensitiveMap(parameters)) 39 | val partitionColumn = options.partitionColumn 40 | val lowerBound = options.lowerBound 41 | val upperBound = options.upperBound 42 | val numPartitions = options.numPartitions 43 | 44 | val partitionInfo = if (partitionColumn.isEmpty) { 45 | assert(lowerBound.isEmpty && upperBound.isEmpty, "When 'partitionColumn' is not specified, " + 46 | s"'$JDBC_LOWER_BOUND' and '$JDBC_UPPER_BOUND' are expected to be empty") 47 | null 48 | } else { 49 | assert(lowerBound.nonEmpty && upperBound.nonEmpty && numPartitions.nonEmpty, 50 | s"When 'partitionColumn' is specified, '$JDBC_LOWER_BOUND', '$JDBC_UPPER_BOUND', and " + 51 | s"'$JDBC_NUM_PARTITIONS' are also required") 52 | JDBCPartitioningInfo( 53 | partitionColumn.get, lowerBound.get, upperBound.get, numPartitions.get) 54 | } 55 | val parts = JDBCRelation.columnPartition(partitionInfo) 56 | GreenplumRelation(parts, options)(sqlContext.sparkSession) 57 | } 58 | 59 | override def createRelation( 60 | sqlContext: SQLContext, 61 | mode: SaveMode, 62 | parameters: Map[String, String], 63 | df: DataFrame): BaseRelation = { 64 | val options = GreenplumOptions(CaseInsensitiveMap(parameters)) 65 | val isCaseSensitive = sqlContext.conf.caseSensitiveAnalysis 66 | 67 | val m = options.maxConnections 68 | val conn = JdbcUtils.createConnectionFactory(options)() 69 | try { 70 | if (tableExists(conn, options.table)) { 71 | val tableSchema = JdbcUtils.getSchemaOption(conn, options) 72 | checkSchema(tableSchema, df.schema, isCaseSensitive) 73 | val orderedDf = reorderDataFrameColumns(df, tableSchema) 74 | // In fact, the mode here is Overwrite constantly, we add other modes just for compatible. 75 | mode match { 76 | case SaveMode.Overwrite 77 | if options.isTruncate && 78 | JdbcUtils.isCascadingTruncateTable(options.url).contains(false) => 79 | JdbcUtils.truncateTable(conn, options) 80 | nonTransactionalCopy( 81 | if (options.transactionOn) orderedDf.coalesce(1) else orderedDf.coalesce(m), 82 | orderedDf.schema, options) 83 | case SaveMode.Overwrite => 84 | transactionalCopy(orderedDf.coalesce(m), orderedDf.schema, options) 85 | case SaveMode.Append => 86 | nonTransactionalCopy( 87 | if (options.transactionOn) { 88 | orderedDf.coalesce(1) 89 | } else { 90 | orderedDf.coalesce(m) 91 | }, 92 | orderedDf.schema, options) 93 | case SaveMode.ErrorIfExists => 94 | throw new AnalysisException(s"Table or view '${options.table}' already exists. $mode") 95 | case SaveMode.Ignore => // do nothing 96 | } 97 | } else { 98 | transactionalCopy(df.coalesce(m), df.schema, options) 99 | } 100 | } finally { 101 | closeConnSilent(conn) 102 | } 103 | createRelation(sqlContext, parameters) 104 | } 105 | 106 | private def checkSchema( 107 | tableSchema: Option[StructType], 108 | dfSchema: StructType, 109 | isCaseSensitive: Boolean): Unit = { 110 | if (!tableSchema.isEmpty) { 111 | val columnNameEquality = if (isCaseSensitive) { 112 | org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution 113 | } else { 114 | org.apache.spark.sql.catalyst.analysis.caseInsensitiveResolution 115 | } 116 | val tableColumnNames = tableSchema.get.fieldNames 117 | dfSchema.fields.map { col => 118 | tableColumnNames.find(f => columnNameEquality(f, col.name)).getOrElse( 119 | throw new AnalysisException(s"Column ${col.name} not found int schema $tableSchema.") 120 | ) 121 | } 122 | } 123 | } 124 | } 125 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/execution/datasources/greenplum/DefaultSource15.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.sql.execution.datasources.greenplum 19 | 20 | import org.apache.spark.sql.sources.DataSourceRegister 21 | 22 | class DefaultSource15 extends DefaultSource with DataSourceRegister { 23 | 24 | override def shortName(): String = "greenplum" 25 | } 26 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/execution/datasources/greenplum/GreenplumOptions.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.sql.execution.datasources.greenplum 19 | 20 | import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap 21 | import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions 22 | import org.apache.spark.util.Utils 23 | 24 | /** 25 | * Options for the Greenplum data source. 26 | */ 27 | case class GreenplumOptions(@transient params: CaseInsensitiveMap[String]) 28 | extends JDBCOptions(params.updated("driver", "org.postgresql.Driver")) { 29 | 30 | val delimiter: String = params.getOrElse("delimiter", ",") 31 | assert(delimiter.length == 1, "The delimiter should be a single character.") 32 | 33 | /** 34 | * This option is only used for these cases: 35 | * 1. overwrite a gptable, which is a CascadingTruncateTable. 36 | * 2. append data to a gptable. 37 | */ 38 | val transactionOn: Boolean = params.getOrElse("transactionOn", "false").toBoolean 39 | 40 | /** Max number of times we are allowed to retry dropTempTable operation. */ 41 | val dropTempTableMaxRetries: Int = 3 42 | 43 | /** Timeout for copying a partition's data to greenplum. */ 44 | val copyTimeout = Utils.timeStringAsMs(params.getOrElse("copyTimeout", "1h")) 45 | assert(copyTimeout > 0, "The copy timeout should be positive, 10s, 10min, 1h etc.") 46 | 47 | /** Max task numbers write Greenplum concurrently */ 48 | val maxConnections = params.getOrElse("maxConnections", "100").toInt 49 | } 50 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/execution/datasources/greenplum/GreenplumRelation.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.sql.execution.datasources.greenplum 19 | 20 | import org.apache.spark.Partition 21 | import org.apache.spark.rdd.RDD 22 | import org.apache.spark.sql._ 23 | import org.apache.spark.sql.execution.datasources.jdbc.{JDBCRDD, JdbcUtils} 24 | import org.apache.spark.sql.jdbc.JdbcDialects 25 | import org.apache.spark.sql.sources.{BaseRelation, Filter, InsertableRelation, PrunedFilteredScan} 26 | import org.apache.spark.sql.types._ 27 | 28 | private[sql] case class GreenplumRelation( 29 | parts: Array[Partition], options: GreenplumOptions)(@transient val sparkSession: SparkSession) 30 | extends BaseRelation 31 | with PrunedFilteredScan 32 | with InsertableRelation { 33 | 34 | import GreenplumUtils._ 35 | 36 | override def sqlContext: SQLContext = sparkSession.sqlContext 37 | override val needConversion: Boolean = false 38 | override val schema: StructType = { 39 | val tableSchema = JDBCRDD.resolveTable(options) 40 | options.customSchema match { 41 | case Some(customSchema) => JdbcUtils.getCustomSchema( 42 | tableSchema, customSchema, sparkSession.sessionState.conf.resolver) 43 | case None => tableSchema 44 | } 45 | } 46 | 47 | // Check if JDBCRDD.compileFilter can accept input filters 48 | override def unhandledFilters(filters: Array[Filter]): Array[Filter] = { 49 | filters.filter(JDBCRDD.compileFilter(_, JdbcDialects.get(options.url)).isEmpty) 50 | } 51 | 52 | override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { 53 | // Rely on a type erasure hack to pass RDD[InternalRow] back as RDD[Row] 54 | JDBCRDD.scanTable( 55 | sparkSession.sparkContext, 56 | schema, 57 | requiredColumns, 58 | filters, 59 | parts, 60 | options).asInstanceOf[RDD[Row]] 61 | } 62 | 63 | override def insert(data: DataFrame, overwrite: Boolean): Unit = { 64 | val conn = JdbcUtils.createConnectionFactory(options)() 65 | val maxConns = options.maxConnections 66 | try { 67 | if (overwrite) { 68 | if (options.isTruncate && 69 | JdbcUtils.isCascadingTruncateTable(options.url).contains(false)) { 70 | JdbcUtils.truncateTable(conn, options) 71 | nonTransactionalCopy( 72 | if (options.transactionOn) data.coalesce(1) else data.coalesce(maxConns), 73 | schema, options) 74 | } else { 75 | transactionalCopy(data.coalesce(maxConns), schema, options) 76 | } 77 | } else { 78 | nonTransactionalCopy( 79 | if (options.transactionOn) { 80 | data.coalesce(1) 81 | } else { 82 | data.coalesce(maxConns) 83 | }, 84 | schema, options) 85 | } 86 | } finally { 87 | closeConnSilent(conn) 88 | } 89 | } 90 | 91 | override def toString: String = { 92 | val partitioningInfo = if (parts.nonEmpty) s" [numPartitions=${parts.length}]" else "" 93 | // credentials should not be included in the plan output, table information is sufficient. 94 | s"GreenplumRelation(${options.table})" + partitioningInfo 95 | } 96 | } 97 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/execution/datasources/greenplum/GreenplumUtils.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.sql.execution.datasources.greenplum 19 | 20 | import java.io._ 21 | import java.nio.charset.StandardCharsets 22 | import java.sql.{Connection, Date, Timestamp} 23 | import java.util.UUID 24 | import java.util.concurrent.{TimeoutException, TimeUnit} 25 | 26 | import scala.concurrent.Promise 27 | import scala.concurrent.duration.Duration 28 | import scala.util.Try 29 | 30 | import org.postgresql.copy.CopyManager 31 | import org.postgresql.core.BaseConnection 32 | 33 | import org.apache.spark.SparkEnv 34 | import org.apache.spark.internal.Logging 35 | import org.apache.spark.sql.{DataFrame, Row} 36 | import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils 37 | import org.apache.spark.sql.types._ 38 | import org.apache.spark.util.{LongAccumulator, ThreadUtils, Utils} 39 | 40 | object GreenplumUtils extends Logging { 41 | 42 | def makeConverter(dataType: DataType): (Row, Int) => String = dataType match { 43 | case StringType => (r: Row, i: Int) => r.getString(i) 44 | case BooleanType => (r: Row, i: Int) => r.getBoolean(i).toString 45 | case ByteType => (r: Row, i: Int) => r.getByte(i).toString 46 | case ShortType => (r: Row, i: Int) => r.getShort(i).toString 47 | case IntegerType => (r: Row, i: Int) => r.getInt(i).toString 48 | case LongType => (r: Row, i: Int) => r.getLong(i).toString 49 | case FloatType => (r: Row, i: Int) => r.getFloat(i).toString 50 | case DoubleType => (r: Row, i: Int) => r.getDouble(i).toString 51 | case DecimalType() => (r: Row, i: Int) => r.getDecimal(i).toString 52 | 53 | case DateType => 54 | (r: Row, i: Int) => r.getAs[Date](i).toString 55 | 56 | case TimestampType => (r: Row, i: Int) => r.getAs[Timestamp](i).toString 57 | 58 | case BinaryType => (r: Row, i: Int) => 59 | new String(r.getAs[Array[Byte]](i), StandardCharsets.UTF_8) 60 | 61 | case udt: UserDefinedType[_] => makeConverter(udt.sqlType) 62 | case _ => (row: Row, ordinal: Int) => row.get(ordinal).toString 63 | } 64 | 65 | def convertRow( 66 | row: Row, 67 | length: Int, 68 | delimiter: String, 69 | valueConverters: Array[(Row, Int) => String]): Array[Byte] = { 70 | var i = 0 71 | val values = new Array[String](length) 72 | while (i < length) { 73 | if (!row.isNullAt(i)) { 74 | values(i) = convertValue(valueConverters(i).apply(row, i), delimiter.charAt(0)) 75 | } else { 76 | values(i) = "NULL" 77 | } 78 | i += 1 79 | } 80 | (values.mkString(delimiter) + "\n").getBytes("UTF-8") 81 | } 82 | 83 | def convertValue(str: String, delimiter: Char): String = { 84 | str.flatMap { 85 | case '\\' => "\\\\" 86 | case '\n' => "\\n" 87 | case '\r' => "\\r" 88 | case `delimiter` => s"\\$delimiter" 89 | case c if c == 0 => "" // If this char is an empty character, drop it. 90 | case c => s"$c" 91 | } 92 | } 93 | 94 | /** 95 | * https://www.postgresql.org/docs/9.2/sql-copy.html 96 | * 97 | * Copy data to greenplum in a single transaction. 98 | * 99 | * @param df the [[DataFrame]] will be copy to the Greenplum 100 | * @param schema the table schema in Greemnplum 101 | * @param options Options for the Greenplum data source 102 | */ 103 | def transactionalCopy( 104 | df: DataFrame, 105 | schema: StructType, 106 | options: GreenplumOptions): Unit = { 107 | val randomString = UUID.randomUUID().toString.filterNot(_ == '-') 108 | val canonicalTblName = TableNameExtractor.extract(options.table) 109 | val schemaPrefix = canonicalTblName.schema.map(_ + ".").getOrElse("") 110 | val rawTblName = canonicalTblName.rawName 111 | val suffix = "sparkGpTmp" 112 | val quote = "\"" 113 | 114 | val tempTable = s"$schemaPrefix$quote${rawTblName}_${randomString}_$suffix$quote" 115 | val strSchema = JdbcUtils.schemaString(df, options.url, options.createTableColumnTypes) 116 | val createTempTbl = s"CREATE TABLE $tempTable ($strSchema) ${options.createTableOptions}" 117 | 118 | // Stage 1. create a _sparkGpTmp table as a shadow of the target Greenplum table. If this stage 119 | // fails, the whole process will abort. 120 | val conn = JdbcUtils.createConnectionFactory(options)() 121 | try { 122 | executeStatement(conn, createTempTbl) 123 | } finally { 124 | closeConnSilent(conn) 125 | } 126 | 127 | // Stage 2. Spark executors run copy task to Greenplum, and will increase the accumulator if 128 | // each task successfully copied. 129 | val accumulator = df.sparkSession.sparkContext.longAccumulator("copySuccess") 130 | df.foreachPartition { rows => 131 | copyPartition(rows, options, schema, tempTable, Some(accumulator)) 132 | } 133 | 134 | // Stage 3. if the accumulator value is not equal to the [[Dataframe]] instance's partition 135 | // number. The Spark job will fail with a [[PartitionCopyFailureException]]. 136 | // Otherwise, we will run a rename table ddl statement to rename the tmp table to the final 137 | // target table. 138 | val partNum = df.rdd.getNumPartitions 139 | val conn2 = JdbcUtils.createConnectionFactory(options)() 140 | try { 141 | if (accumulator.value == partNum) { 142 | if (tableExists(conn2, options.table)) { 143 | JdbcUtils.dropTable(conn2, options.table) 144 | } 145 | 146 | val newTableName = s"${options.table}".split("\\.").last 147 | val renameTempTbl = s"ALTER TABLE $tempTable RENAME TO $newTableName" 148 | executeStatement(conn2, renameTempTbl) 149 | } else { 150 | throw new PartitionCopyFailureException( 151 | s""" 152 | | Job aborted for that there are some partitions failed to copy data to greenPlum: 153 | | Total partitions is: $partNum and successful partitions is: ${accumulator.value}. 154 | | You can retry again. 155 | """.stripMargin) 156 | } 157 | } finally { 158 | if (tableExists(conn2, tempTable)) { 159 | retryingDropTableSilent(conn2, tempTable) 160 | } 161 | closeConnSilent(conn2) 162 | } 163 | } 164 | 165 | /** 166 | * Drop the table and retry automatically when exception occurred. 167 | */ 168 | def retryingDropTableSilent(conn: Connection, table: String): Unit = { 169 | val dropTmpTableMaxRetry = 3 170 | var dropTempTableRetryCount = 0 171 | var dropSuccess = false 172 | 173 | while (!dropSuccess && dropTempTableRetryCount < dropTmpTableMaxRetry) { 174 | try { 175 | JdbcUtils.dropTable(conn, table) 176 | dropSuccess = true 177 | } catch { 178 | case e: Exception => 179 | dropTempTableRetryCount += 1 180 | logWarning(s"Drop tempTable $table failed for $dropTempTableRetryCount" + 181 | s"/${dropTmpTableMaxRetry} times, and will retry.", e) 182 | } 183 | } 184 | if (!dropSuccess) { 185 | logError(s"Drop tempTable $table failed for $dropTmpTableMaxRetry times," + 186 | s" and will not retry.") 187 | } 188 | } 189 | 190 | /** 191 | * https://www.postgresql.org/docs/9.2/sql-copy.html 192 | * 193 | * Copy data to greenplum in these cases, which need update origin gptable. 194 | * 1. Overwrite an existed gptable, which is a CascadingTruncateTable. 195 | * 2. Append data to a gptable. 196 | * 197 | * When transcationOn option is true, we will coalesce the dataFrame to one partition, 198 | * and the copy operation for each partition is atomic. 199 | * 200 | * @param df the [[DataFrame]] will be copy to the Greenplum 201 | * @param schema the table schema in Greemnplum 202 | * @param options Options for the Greenplum data source 203 | */ 204 | def nonTransactionalCopy( 205 | df: DataFrame, 206 | schema: StructType, 207 | options: GreenplumOptions): Unit = { 208 | df.foreachPartition { rows => 209 | copyPartition(rows, options, schema, options.table) 210 | } 211 | } 212 | 213 | /** 214 | * Copy a partition's data to a gptable. 215 | * 216 | * @param rows rows of a partition will be copy to the Greenplum 217 | * @param options Options for the Greenplum data source 218 | * @param schema the table schema in Greemnplum 219 | * @param tableName the tableName, to which the data will be copy 220 | * @param accumulator account for recording the successful partition num 221 | */ 222 | def copyPartition( 223 | rows: Iterator[Row], 224 | options: GreenplumOptions, 225 | schema: StructType, 226 | tableName: String, 227 | accumulator: Option[LongAccumulator] = None): Unit = { 228 | val valueConverters: Array[(Row, Int) => String] = 229 | schema.map(s => makeConverter(s.dataType)).toArray 230 | val tmpDir = Utils.createTempDir(Utils.getLocalDir(SparkEnv.get.conf), "greenplum") 231 | val dataFile = new File(tmpDir, UUID.randomUUID().toString) 232 | logInfo(s"Start to write data to local tmp file: ${dataFile.getCanonicalPath}") 233 | val out = new BufferedOutputStream(new FileOutputStream(dataFile)) 234 | val startW = System.nanoTime() 235 | try { 236 | rows.foreach(r => out.write( 237 | convertRow(r, schema.length, options.delimiter, valueConverters))) 238 | } finally { 239 | out.close() 240 | } 241 | val endW = System.nanoTime() 242 | logInfo(s"Finished writing data to local tmp file: ${dataFile.getCanonicalPath}, " + 243 | s"time taken: ${(endW - startW) / math.pow(10, 9)}s") 244 | val in = new BufferedInputStream(new FileInputStream(dataFile)) 245 | val sql = s"COPY $tableName" + 246 | s" FROM STDIN WITH NULL AS 'NULL' DELIMITER AS E'${options.delimiter}'" 247 | 248 | val promisedCopyNums = Promise[Long] 249 | val conn = JdbcUtils.createConnectionFactory(options)() 250 | val copyManager = new CopyManager(conn.asInstanceOf[BaseConnection]) 251 | val copyThread = new Thread("copy-to-gp-thread") { 252 | override def run(): Unit = promisedCopyNums.complete(Try(copyManager.copyIn(sql, in))) 253 | } 254 | 255 | try { 256 | logInfo(s"Start copy steam to Greenplum with copy command $sql") 257 | val start = System.nanoTime() 258 | copyThread.start() 259 | try { 260 | val nums = ThreadUtils.awaitResult(promisedCopyNums.future, 261 | Duration(options.copyTimeout, TimeUnit.MILLISECONDS)) 262 | val end = System.nanoTime() 263 | logInfo(s"Copied $nums row(s) to Greenplum," + 264 | s" time taken: ${(end - start) / math.pow(10, 9)}s") 265 | } catch { 266 | case _: TimeoutException => 267 | throw new TimeoutException( 268 | s""" 269 | | The copy operation for copying this partition's data to greenplum has been running for 270 | | more than the timeout: ${TimeUnit.MILLISECONDS.toSeconds(options.copyTimeout)}s. 271 | | You can configure this timeout with option copyTimeout, such as "2h", "100min", 272 | | and default copyTimeout is "1h". 273 | """.stripMargin) 274 | } 275 | accumulator.foreach(_.add(1L)) 276 | } finally { 277 | copyThread.interrupt() 278 | copyThread.join() 279 | in.close() 280 | closeConnSilent(conn) 281 | } 282 | } 283 | 284 | def closeConnSilent(conn: Connection): Unit = { 285 | try { 286 | conn.close() 287 | } catch { 288 | case e: Exception => logWarning("Exception occured when closing connection.", e) 289 | } 290 | } 291 | 292 | def executeStatement(conn: Connection, sql: String): Unit = { 293 | val statement = conn.createStatement() 294 | try { 295 | statement.executeUpdate(sql) 296 | } finally { 297 | statement.close() 298 | } 299 | } 300 | 301 | def reorderDataFrameColumns(df: DataFrame, tableSchema: Option[StructType]): DataFrame = { 302 | tableSchema.map { schema => 303 | df.selectExpr(schema.map(filed => filed.name): _*) 304 | }.getOrElse(df) 305 | } 306 | 307 | /** 308 | * Returns true if the table already exists in the JDBC database. 309 | */ 310 | def tableExists(conn: Connection, table: String): Boolean = { 311 | val query = s"SELECT * FROM $table WHERE 1=0" 312 | Try { 313 | val statement = conn.prepareStatement(query) 314 | try { 315 | statement.executeQuery() 316 | } finally { 317 | statement.close() 318 | } 319 | }.isSuccess 320 | } 321 | } 322 | 323 | private[greenplum] case class CanonicalTblName(schema: Option[String], rawName: String) 324 | 325 | /** 326 | * Extract schema name and raw table name from a table name string. 327 | */ 328 | private[greenplum] object TableNameExtractor { 329 | private val nonSchemaTable = """[\"]*([0-9a-zA-Z_]+)[\"]*""".r 330 | private val schemaTable = """([\"]*[0-9a-zA-Z_]+[\"]*)\.[\"]*([0-9a-zA-Z_]+)[\"]*""".r 331 | 332 | def extract(tableName: String): CanonicalTblName = { 333 | tableName match { 334 | case nonSchemaTable(t) => CanonicalTblName(None, t) 335 | case schemaTable(schema, t) => CanonicalTblName(Some(schema), t) 336 | case _ => throw new IllegalArgumentException( 337 | s""" 338 | | The table name is illegal, you can set it with the dbtable option, such as 339 | | "schemaname"."tableName" or just "tableName" with a default schema "public". 340 | """.stripMargin 341 | ) 342 | } 343 | } 344 | } 345 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/execution/datasources/greenplum/PartitionCopyFailureException.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.sql.execution.datasources.greenplum 19 | 20 | /** 21 | * General exception caused by some partition failed for copying dataFrame to greenplum. 22 | */ 23 | class PartitionCopyFailureException( 24 | errorMsg: String, 25 | cause: Throwable = null) 26 | extends Exception(errorMsg, cause) 27 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/execution/datasources/greenplum/GreenplumOptionsSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.sql.execution.datasources.greenplum 19 | 20 | import java.util.{Date, TimeZone} 21 | 22 | import org.apache.commons.lang3.time.FastDateFormat 23 | import org.scalatest.Matchers 24 | 25 | import org.apache.spark.SparkFunSuite 26 | import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} 27 | 28 | class GreenplumOptionsSuite extends SparkFunSuite with Matchers { 29 | private val date = new Date(0) 30 | 31 | test("empty user specified options") { 32 | val e = intercept[IllegalArgumentException](GreenplumOptions(CaseInsensitiveMap(Map()))) 33 | e.getMessage should include("Option 'url' is required") 34 | } 35 | 36 | test("map with only url") { 37 | val e = intercept[IllegalArgumentException]( 38 | GreenplumOptions(CaseInsensitiveMap(Map("url" -> "")))) 39 | e.getMessage should include("Option 'dbtable' is required") 40 | } 41 | 42 | test("driver class should always using postgresql") { 43 | val options = GreenplumOptions(CaseInsensitiveMap(Map("url" -> "", "dbtable" -> "src"))) 44 | options.driverClass should be("org.postgresql.Driver") 45 | val options2 = GreenplumOptions(CaseInsensitiveMap( 46 | Map("url" -> "", 47 | "dbtable" -> "src", 48 | "driver" -> "org.mysql.Driver"))) 49 | options2.driverClass should be("org.postgresql.Driver") 50 | } 51 | 52 | test("as properties") { 53 | val options = GreenplumOptions(CaseInsensitiveMap(Map("url" -> "", "dbtable" -> "src"))) 54 | val properties = options.asProperties 55 | properties.getProperty("url") should be("") 56 | properties.get("dbtable") should be("src") 57 | properties.get("driver") should be("org.postgresql.Driver") 58 | } 59 | 60 | test("as connection properties") { 61 | val options = GreenplumOptions(CaseInsensitiveMap(Map("url" -> "", "dbtable" -> "src"))) 62 | val properties = options.asConnectionProperties 63 | properties.getProperty("url") should be(null) 64 | properties.get("dbtable") should be(null) 65 | properties.get("driver") should be(null) 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/sql/execution/datasources/greenplum/GreenplumUtilsSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.sql.execution.datasources.greenplum 19 | 20 | import java.io.File 21 | import java.sql.{Connection, Date, SQLException, Timestamp} 22 | import java.util.TimeZone 23 | 24 | import scala.concurrent.TimeoutException 25 | 26 | import io.airlift.testing.postgresql.TestingPostgreSqlServer 27 | import org.mockito.Matchers._ 28 | import org.mockito.Mockito._ 29 | import org.scalatest.mockito.MockitoSugar 30 | import org.apache.spark.SparkFunSuite 31 | 32 | import org.apache.spark.api.java.function.ForeachPartitionFunction 33 | import org.apache.spark.sql._ 34 | import org.apache.spark.sql.catalyst.expressions.GenericRow 35 | import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap 36 | import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils 37 | import org.apache.spark.sql.types._ 38 | import org.apache.spark.util.Utils 39 | 40 | class GreenplumUtilsSuite extends SparkFunSuite with MockitoSugar { 41 | var postgres: TestingPostgreSqlServer = _ 42 | var url: String = _ 43 | var sparkSession: SparkSession = _ 44 | var tempDir: File = _ 45 | 46 | override def beforeAll(): Unit = { 47 | tempDir = Utils.createTempDir() 48 | postgres = new TestingPostgreSqlServer("gptest", "gptest") 49 | url = postgres.getJdbcUrl 50 | sparkSession = SparkSession.builder() 51 | .config("spark.master", "local") 52 | .config("spark.app.name", "testGp") 53 | .config("spark.sql.warehouse.dir", s"${tempDir.getAbsolutePath}/warehouse") 54 | .config("spark.local.dir", s"${tempDir.getAbsolutePath}/local") 55 | .getOrCreate() 56 | } 57 | 58 | override def afterAll(): Unit = { 59 | try { 60 | if (postgres != null) { 61 | postgres.close() 62 | } 63 | if (sparkSession != null) { 64 | sparkSession.stop() 65 | } 66 | } finally { 67 | Utils.deleteRecursively(tempDir) 68 | } 69 | } 70 | 71 | test("make converter") { 72 | val options = GreenplumOptions(CaseInsensitiveMap(Map("url" -> "", "dbtable" -> "src"))) 73 | 74 | val now = System.currentTimeMillis() 75 | val row1 = Row(true, 1.toByte, 2.toShort, 3, 4.toLong, 76 | 5.toFloat, 6.toDouble, 7.toString, 8.toString.getBytes, 77 | new Date(now), 78 | new Timestamp(now), 79 | new java.math.BigDecimal(11), 80 | Array[String]("12", "12"), 81 | Map(13 -> 13, 130 -> 130), 82 | Row(14, "15")) 83 | 84 | val row2 = Row(null) 85 | 86 | val boolConverter = GreenplumUtils.makeConverter(BooleanType) 87 | assert(boolConverter.apply(row1, 0) === "true") 88 | intercept[NullPointerException](boolConverter.apply(row2, 0) === "") 89 | 90 | val byteConverter = GreenplumUtils.makeConverter(ByteType) 91 | assert(byteConverter(row1, 1) === "1") 92 | 93 | val shortConverter = GreenplumUtils.makeConverter(ShortType) 94 | assert(shortConverter(row1, 2) === "2") 95 | 96 | val intConverter = GreenplumUtils.makeConverter(IntegerType) 97 | assert(intConverter(row1, 3) === "3") 98 | 99 | val longConverter = GreenplumUtils.makeConverter(LongType) 100 | assert(longConverter(row1, 4) === "4") 101 | 102 | val floatConverter = GreenplumUtils.makeConverter(FloatType) 103 | assert(floatConverter(row1, 5) === "5.0") 104 | 105 | val doubleConverter = GreenplumUtils.makeConverter(DoubleType) 106 | assert(doubleConverter(row1, 6) === "6.0") 107 | 108 | val strConverter = GreenplumUtils.makeConverter(StringType) 109 | assert(strConverter(row1, 7) === "7") 110 | 111 | val binConverter = GreenplumUtils.makeConverter(BinaryType) 112 | assert(binConverter(row1, 8) === "8") 113 | 114 | val dateConverter = GreenplumUtils.makeConverter(DateType) 115 | assert(dateConverter(row1, 9) === new Date(now).toString) 116 | 117 | val tsConverter = GreenplumUtils.makeConverter(TimestampType) 118 | assert(tsConverter(row1, 10) === new Timestamp(now).toString) 119 | 120 | val decimalConverter = GreenplumUtils.makeConverter(DecimalType(2, 0)) 121 | assert(decimalConverter(row1, 11) === new java.math.BigDecimal(11).toString) 122 | 123 | // val arrConverter = GreenplumUtils.makeConverter(ArrayType(StringType), options) 124 | // assert(arrConverter(row1, 12) === Array[String]("12", "12").mkString("[", ",", "]")) 125 | // 126 | // val mapConverter = GreenplumUtils.makeConverter(MapType(IntegerType, IntegerType), options) 127 | // assert(mapConverter(row1, 13) === 128 | // Map(13 -> 13, 130 -> 130) 129 | // .map(e => e._1 + ":" + e._2).toSeq.sorted.mkString("{", ",", "}")) 130 | // 131 | // val structConverter = 132 | // GreenplumUtils.makeConverter( 133 | // StructType(Array(StructField("a", IntegerType), StructField("b", StringType))), options) 134 | // assert(structConverter(row1, 14) === "{\"a\":14,\"b\":15}") 135 | } 136 | 137 | test("test copy to greenplum") { 138 | withConnectionAndOptions { (conn, tblname, options) => 139 | // scalastyle:off 140 | val buffer = "测试".getBytes().toBuffer 141 | buffer += 0 142 | val strWithEmpty = new String(buffer.toArray) 143 | val kvs = Map[Int, String](0 -> " ", 1 -> "\t", 2 -> "\n", 3 -> "\r", 4 -> "\\t", 5 -> "\\n", 144 | 6 -> "\\", 7 -> ",", 8 -> "te\tst", 9 -> "1`'`", 10 -> "中文测试", 11 -> strWithEmpty) 145 | // scalastyle:on 146 | val rdd = sparkSession.sparkContext.parallelize(kvs.toSeq) 147 | val df = sparkSession.createDataFrame(rdd) 148 | val defaultSource = new DefaultSource 149 | 150 | defaultSource.createRelation(sparkSession.sqlContext, SaveMode.Overwrite, options.params, df) 151 | val stmt1 = conn.createStatement() 152 | stmt1.executeQuery(s"select * from $tblname") 153 | stmt1.setFetchSize(kvs.size + 1) 154 | var count = 0 155 | val result2 = stmt1.getResultSet 156 | while (result2.next()) { 157 | val k = result2.getInt(1) 158 | val v = result2.getString(2) 159 | count += 1 160 | assert(kvs(k).filterNot(_ == 0) === v) 161 | } 162 | assert(count === kvs.size) 163 | 164 | // Append the df's data to gptbl, so the size will double. 165 | defaultSource.createRelation(sparkSession.sqlContext, SaveMode.Append, options.params, df) 166 | val stmt2 = conn.createStatement() 167 | stmt2.executeQuery(s"select * from $tblname") 168 | stmt2.setFetchSize(kvs.size * 2 + 1) 169 | val result3 = stmt2.getResultSet 170 | count = 0 171 | while (result3.next()) { 172 | count += 1 173 | } 174 | assert(count === kvs.size * 2) 175 | 176 | // Overwrite gptbl with df's data. 177 | defaultSource.createRelation(sparkSession.sqlContext, SaveMode.Overwrite, options.params, df) 178 | val stat4 = conn.createStatement() 179 | stat4.executeQuery(s"select * from $tblname") 180 | stat4.setFetchSize(kvs.size + 1) 181 | val result4 = stat4.getResultSet 182 | count = 0 183 | while (result4.next()) { 184 | count += 1 185 | } 186 | assert(count === kvs.size) 187 | 188 | intercept[AnalysisException](defaultSource.createRelation(sparkSession.sqlContext, 189 | SaveMode.ErrorIfExists, options.params, df)) 190 | 191 | defaultSource.createRelation(sparkSession.sqlContext, SaveMode.Ignore, options.params, df) 192 | 193 | } 194 | } 195 | 196 | test("test covert value and row") { 197 | withConnectionAndOptions { (_, _, options) => 198 | val value = "test\t\rtest\n\\n\\," 199 | assert(GreenplumUtils.convertValue(value, '\t') === "test\\\t\\rtest\\n\\\\n\\\\,") 200 | 201 | val values = Array[Any]("\n", "\t", ",", "\r", "\\", "\\n") 202 | val schema = new StructType().add("c1", StringType).add("c2", StringType) 203 | .add("c3", StringType).add("c4", StringType).add("c5", StringType) 204 | .add("c6", StringType) 205 | val valueConverters: Array[(Row, Int) => String] = 206 | schema.map(s => GreenplumUtils.makeConverter(s.dataType)).toArray 207 | 208 | val row = new GenericRow(values) 209 | val str = GreenplumUtils.convertRow(row, schema.length, options.delimiter, valueConverters) 210 | assert(str === "\\n\t\\\t\t,\t\\r\t\\\\\t\\\\n\n".getBytes("utf-8")) 211 | } 212 | } 213 | 214 | test("test copy partition") { 215 | withConnectionAndOptions { (conn, tblname, options) => 216 | 217 | val values = Array[Any]("\n", "\t", ",", "\r", "\\", "\\n") 218 | val schema = new StructType().add("c1", StringType).add("c2", StringType) 219 | .add("c3", StringType).add("c4", StringType).add("c5", StringType) 220 | .add("c6", StringType) 221 | val rows = Array(new GenericRow(values)).toIterator 222 | 223 | val createTbl = s"CREATE TABLE $tblname(c1 text, c2 text, c3 text, c4 text, c5 text, c6 text)" 224 | GreenplumUtils.executeStatement(conn, createTbl) 225 | 226 | GreenplumUtils.copyPartition(rows, options, schema, tblname) 227 | val stat = conn.createStatement() 228 | val sql = s"SELECT * FROM $tblname" 229 | stat.executeQuery(sql) 230 | val result = stat.getResultSet 231 | result.next() 232 | for (i <- (0 until values.size)) { 233 | assert(result.getObject(i + 1) === values(i)) 234 | } 235 | assert(!result.next()) 236 | } 237 | } 238 | 239 | test("test transactions support") { 240 | withConnectionAndOptions { (conn, tblname, options) => 241 | // scalastyle:off 242 | val kvs = Map[Int, String](0 -> " ", 1 -> "\t", 2 -> "\n", 3 -> "\r", 4 -> "\\t", 243 | 5 -> "\\n", 6 -> "\\", 7 -> ",", 8 -> "te\tst", 9 -> "1`'`", 10 -> "中文测试") 244 | // scalastyle:on 245 | // This suffix should be consisted with the suffix in transactionalCopy 246 | val tempSuffix = "sparkGpTmp" 247 | val df = mock[DataFrame] 248 | val rdd = sparkSession.sparkContext.parallelize(kvs.toSeq) 249 | val realdf = sparkSession.createDataFrame(rdd) 250 | val schema = realdf.schema 251 | when(df.foreachPartition(any[ForeachPartitionFunction[Row]]())) 252 | .thenThrow(classOf[SQLException]) 253 | when(df.sparkSession).thenReturn(sparkSession) 254 | when(df.schema).thenReturn(schema) 255 | when(df.rdd).thenReturn(realdf.rdd) 256 | 257 | // This would touch an exception, gptable are not created and temp table would be removed 258 | intercept[PartitionCopyFailureException]( 259 | GreenplumUtils.transactionalCopy(df, schema, options)) 260 | 261 | val showTables = "SELECT table_name FROM information_schema.tables" 262 | val stat = conn.createStatement() 263 | val result = stat.executeQuery(showTables) 264 | while (result.next()) { 265 | val tbl = result.getString(1) 266 | assert(tbl != tblname && !tbl.endsWith(tempSuffix)) 267 | } 268 | } 269 | } 270 | 271 | test("test copyPartition with timeout exception") { 272 | val tblname = "tempTable" 273 | // Set the copyTimeout to 1ms, it must trigger a TimeoutException. 274 | val paras = CaseInsensitiveMap(Map("url" -> s"$url", "delimiter" -> "\t", 275 | "dbtable" -> "gptest", "copyTimeout" -> "1ms")) 276 | val options = GreenplumOptions(paras) 277 | val conn = JdbcUtils.createConnectionFactory(options)() 278 | 279 | try { 280 | val values = Array[Any]("\n", "\t", ",", "\r", "\\", "\\n") 281 | val schema = new StructType().add("c1", StringType).add("c2", StringType) 282 | .add("c3", StringType).add("c4", StringType).add("c5", StringType) 283 | .add("c6", StringType) 284 | val rows = Array(new GenericRow(values)).toIterator 285 | val createTbl = s"CREATE TABLE $tblname(c1 text, c2 text, c3 text, c4 text, c5 text, c6 text)" 286 | GreenplumUtils.executeStatement(conn, createTbl) 287 | intercept[TimeoutException](GreenplumUtils.copyPartition(rows, options, schema, tblname)) 288 | } finally { 289 | GreenplumUtils.closeConnSilent(conn) 290 | } 291 | } 292 | 293 | test("test reorder dataframe's columns when relative gp table is existed") { 294 | withConnectionAndOptions { (conn, tblname, options) => 295 | // scalastyle:off 296 | val kvs = Map[Int, String](0 -> " ", 1 -> "\t", 2 -> "\n", 3 -> "\r", 4 -> "\\t", 297 | 5 -> "\\n", 6 -> "\\", 7 -> ",", 8 -> "te\tst", 9 -> "1`'`", 10 -> "中文测试") 298 | // scalastyle:on 299 | val rdd = sparkSession.sparkContext.parallelize(kvs.toSeq) 300 | val df = sparkSession.createDataFrame(rdd) 301 | 302 | // create a gptable whose columns order is not equal with dataFrame 303 | val createTbl = s"CREATE TABLE $tblname (_2 text, _1 int)" 304 | GreenplumUtils.executeStatement(conn, createTbl) 305 | 306 | val defaultSource = new DefaultSource 307 | defaultSource.createRelation(sparkSession.sqlContext, SaveMode.Append, options.params, df) 308 | 309 | val stmt = conn.createStatement() 310 | stmt.executeQuery(s"select * from $tblname") 311 | stmt.setFetchSize(kvs.size + 1) 312 | val result4 = stmt.getResultSet 313 | var count = 0 314 | while (result4.next()) { 315 | count += 1 316 | } 317 | assert(count === kvs.size) 318 | } 319 | } 320 | 321 | test("test convert the table name to canonical table name") { 322 | val quote = "\"" 323 | val schema = "schema" 324 | val table = "table" 325 | 326 | val str1 = s"$table" 327 | val str2 = s"${quote}$table${quote}" 328 | val str3 = s"$schema.${quote}${quote}$str1${quote}" 329 | val str4 = s"${quote}test${quote}test" 330 | assert(TableNameExtractor.extract(str1) === CanonicalTblName(None, table)) 331 | assert(TableNameExtractor.extract(str2) === CanonicalTblName(None, table)) 332 | assert(TableNameExtractor.extract(str3) === CanonicalTblName(Some(schema), table)) 333 | intercept[IllegalArgumentException](TableNameExtractor.extract(str4)) 334 | } 335 | 336 | test("test schema and table names with double quotes") { 337 | val quote = "\"" 338 | val schemaTableNames = Map(s"${quote}163${quote}" -> s"${quote}tempTable${quote}", 339 | s"schemaName" -> s"${quote}tempTable${quote}", s"${quote}163${quote}" -> s"tempTable") 340 | 341 | schemaTableNames.foreach { schemaTableName => 342 | val schema = schemaTableName._1 343 | val tblname = schemaTableName._2 344 | val paras = CaseInsensitiveMap(Map("url" -> s"$url", "delimiter" -> "\t", 345 | "dbtable" -> s"$schema.$tblname")) 346 | val options = GreenplumOptions(paras) 347 | 348 | // scalastyle:off 349 | val kvs = Map[Int, String](0 -> " ", 1 -> "\t", 2 -> "\n", 3 -> "\r", 4 -> "\\t", 350 | 5 -> "\\n", 6 -> "\\", 7 -> ",", 8 -> "te\tst", 9 -> "1`'`", 10 -> "中文测试") 351 | // scalastyle:on 352 | val rdd = sparkSession.sparkContext.parallelize(kvs.toSeq) 353 | val df = sparkSession.createDataFrame(rdd) 354 | 355 | val conn = JdbcUtils.createConnectionFactory(options)() 356 | 357 | try { 358 | val createSchema = s"CREATE SCHEMA IF NOT EXISTS $schema " 359 | GreenplumUtils.executeStatement(conn, createSchema) 360 | val defaultSource = new DefaultSource 361 | defaultSource.createRelation(sparkSession.sqlContext, SaveMode.Append, options.params, df) 362 | 363 | assert(JdbcUtils.tableExists(conn, options)) 364 | 365 | val relation = defaultSource.createRelation(sparkSession.sqlContext, paras) 366 | relation.asInstanceOf[GreenplumRelation].insert(df, true) 367 | assert(JdbcUtils.tableExists(conn, options)) 368 | relation.asInstanceOf[GreenplumRelation].insert(df, false) 369 | assert(JdbcUtils.tableExists(conn, options)) 370 | val paras2 = paras ++: Map("transactionOn" -> "true", "truncate" -> "true") 371 | val relation2 = defaultSource.createRelation(sparkSession.sqlContext, paras2) 372 | relation2.asInstanceOf[GreenplumRelation].insert(df, true) 373 | relation2.asInstanceOf[GreenplumRelation].insert(df, false) 374 | defaultSource.createRelation(sparkSession.sqlContext, SaveMode.Append, paras2, df) 375 | defaultSource.createRelation(sparkSession.sqlContext, SaveMode.Overwrite, paras2, df) 376 | assert(JdbcUtils.tableExists(conn, options)) 377 | } finally { 378 | GreenplumUtils.closeConnSilent(conn) 379 | } 380 | } 381 | } 382 | 383 | def withConnectionAndOptions(f: (Connection, String, GreenplumOptions) => Unit): Unit = { 384 | val schema = "gptest" 385 | val paras = 386 | CaseInsensitiveMap(Map("url" -> s"$url", "delimiter" -> "\t", "dbtable" -> s"$schema.test", 387 | "transactionOn" -> "true")) 388 | val options = GreenplumOptions(paras) 389 | val conn = JdbcUtils.createConnectionFactory(options)() 390 | try { 391 | val createSchema = s"CREATE SCHEMA IF NOT EXISTS $schema" 392 | GreenplumUtils.executeStatement(conn, createSchema) 393 | f(conn, options.table, options) 394 | } finally { 395 | val dropTbl = s"DROP TABLE IF EXISTS ${options.table}" 396 | val dropSchema = s"DROP SCHEMA IF EXISTS $schema" 397 | GreenplumUtils.executeStatement(conn, dropTbl) 398 | GreenplumUtils.executeStatement(conn, dropSchema) 399 | GreenplumUtils.closeConnSilent(conn) 400 | } 401 | } 402 | } 403 | 404 | --------------------------------------------------------------------------------