From 80bc3da24b4dc466f503149b60c38071b9c2cf1c Mon Sep 17 00:00:00 2001 From: vikasrathee-cs Date: Wed, 20 Sep 2023 17:09:15 +0530 Subject: [PATCH] Redshift source and connector plugin added. Redshift source and connector plugin added. --- .../docs/Redshift-batchsource.md | 104 ++++++ .../docs/Redshift-connector.md | 27 ++ .../icons/Redshift-batchsource.png | Bin 0 -> 6145 bytes amazon-redshift-plugin/pom.xml | 134 +++++++ .../amazon/redshift/RedshiftConnector.java | 110 ++++++ .../redshift/RedshiftConnectorConfig.java | 87 +++++ .../amazon/redshift/RedshiftConstants.java | 27 ++ .../amazon/redshift/RedshiftDBRecord.java | 130 +++++++ .../amazon/redshift/RedshiftSchemaReader.java | 121 +++++++ .../amazon/redshift/RedshiftSource.java | 131 +++++++ .../redshift/RedshiftPluginTestBase.java | 218 ++++++++++++ .../redshift/RedshiftPluginTestSuite.java | 31 ++ .../redshift/RedshiftSourceTestRun.java | 332 ++++++++++++++++++ .../widgets/Redshift-batchsource.json | 240 +++++++++++++ .../widgets/Redshift-connector.json | 75 ++++ pom.xml | 1 + 16 files changed, 1768 insertions(+) create mode 100644 amazon-redshift-plugin/docs/Redshift-batchsource.md create mode 100644 amazon-redshift-plugin/docs/Redshift-connector.md create mode 100644 amazon-redshift-plugin/icons/Redshift-batchsource.png create mode 100644 amazon-redshift-plugin/pom.xml create mode 100644 amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftConnector.java create mode 100644 amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftConnectorConfig.java create mode 100644 amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftConstants.java create mode 100644 amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftDBRecord.java create mode 100644 amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftSchemaReader.java create mode 100644 amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftSource.java create mode 100644 amazon-redshift-plugin/src/test/java/io/cdap/plugin/amazon/redshift/RedshiftPluginTestBase.java create mode 100644 amazon-redshift-plugin/src/test/java/io/cdap/plugin/amazon/redshift/RedshiftPluginTestSuite.java create mode 100644 amazon-redshift-plugin/src/test/java/io/cdap/plugin/amazon/redshift/RedshiftSourceTestRun.java create mode 100644 amazon-redshift-plugin/widgets/Redshift-batchsource.json create mode 100644 amazon-redshift-plugin/widgets/Redshift-connector.json diff --git a/amazon-redshift-plugin/docs/Redshift-batchsource.md b/amazon-redshift-plugin/docs/Redshift-batchsource.md new file mode 100644 index 000000000..7bc81cfc0 --- /dev/null +++ b/amazon-redshift-plugin/docs/Redshift-batchsource.md @@ -0,0 +1,104 @@ +# Amazon RedShift Batch Source + + +Description +----------- +Reads from an Amazon RedShift database using a configurable SQL query. +Outputs one record for each row returned by the query. + + +Use Case +-------- +The source is used whenever you need to read from an Amazon RedShift database. For example, you may want +to create daily snapshots of a database table by using this source and writing to +a TimePartitionedFileSet. + + +Properties +---------- +**Reference Name:** Name used to uniquely identify this source for lineage, annotating metadata, etc. + +**JDBC Driver name:** Name of the JDBC driver to use. + +**Host:** Host URL of the current master instance of Redshift cluster. + +**Port:** Port that Redshift master instance is listening to. + +**Database:** Redshift database name. + +**Import Query:** The SELECT query to use to import data from the specified table. +You can specify an arbitrary number of columns to import, or import all columns using \*. The Query should +contain the '$CONDITIONS' string. For example, 'SELECT * FROM table WHERE $CONDITIONS'. +The '$CONDITIONS' string will be replaced by 'splitBy' field limits specified by the bounding query. +The '$CONDITIONS' string is not required if numSplits is set to one. + +**Bounding Query:** Bounding Query should return the min and max of the values of the 'splitBy' field. +For example, 'SELECT MIN(id),MAX(id) FROM table'. Not required if numSplits is set to one. + +**Split-By Field Name:** Field Name which will be used to generate splits. Not required if numSplits is set to one. + +**Number of Splits to Generate:** Number of splits to generate. + +**Username:** User identity for connecting to the specified database. + +**Password:** Password to use to connect to the specified database. + +**Connection Arguments:** A list of arbitrary string key/value pairs as connection arguments. These arguments +will be passed to the JDBC driver as connection arguments for JDBC drivers that may need additional configurations. + +**Schema:** The schema of records output by the source. This will be used in place of whatever schema comes +back from the query. However, it must match the schema that comes back from the query, +except it can mark fields as nullable and can contain a subset of the fields. + +**Fetch Size:** The number of rows to fetch at a time per split. Larger fetch size can result in faster import, +with the tradeoff of higher memory usage. + +Example +------ +Suppose you want to read data from an Amazon Redshift database named "prod" that is running on +"redshift.xyz.eu-central-1.redshift.amazonaws.com", port 5439, as "sa" user with "Test11" password. +Ensure that the driver for Redshift is installed (you can also provide driver name for some specific driver, +otherwise "redshift" will be used), then configure the plugin with:then configure plugin with: + + +``` +Reference Name: "src1" +Driver Name: "redshift" +Host: "redshift.xyz.eu-central-1.redshift.amazonaws.com" +Port: 5439 +Database: "prod" +Import Query: "select id, name, email, phone from users;" +Number of Splits to Generate: 1 +Username: "sa" +Password: "Test11" +``` + +Data Types Mapping +------------------ + +Mapping of Redshift types to CDAP schema: + +| Redshift Data Type | CDAP Schema Data Type | Comment | +|-----------------------------------------------------|-----------------------|----------------------------------------| +| bigint | long | | +| boolean | boolean | | +| character | string | | +| character varying | string | | +| double precision | double | | +| integer | int | | +| numeric(precision, scale)/decimal(precision, scale) | decimal | | +| numeric(with 0 precision) | string | | +| real | float | | +| smallint | int | | +| smallserial | int | | +| text | string | | +| date | date | | +| time [ (p) ] [ without time zone ] | time | | +| time [ (p) ] with time zone | string | | +| timestamp [ (p) ] [ without time zone ] | timestamp | | +| timestamp [ (p) ] with time zone | timestamp | stored in UTC format in database | +| xml | string | | +| json | string | | +| super | string | | +| geometry | bytes | | +| hllsketch | string | | diff --git a/amazon-redshift-plugin/docs/Redshift-connector.md b/amazon-redshift-plugin/docs/Redshift-connector.md new file mode 100644 index 000000000..bbc18e9a9 --- /dev/null +++ b/amazon-redshift-plugin/docs/Redshift-connector.md @@ -0,0 +1,27 @@ +# Amazon RedShift Connection + + +Description +----------- +Use this connection to access data in an Amazon RedShift database using JDBC. + +Properties +---------- +**Name:** Name of the connection. Connection names must be unique in a namespace. + +**Description:** Description of the connection. + +**JDBC Driver name:** Name of the JDBC driver to use. + +**Host:** Host of the current master instance of Redshift cluster. + +**Port:** Port that Redshift master instance is listening to. + +**Database:** Redshift database name. + +**Username:** User identity for connecting to the specified database. + +**Password:** Password to use to connect to the specified database. + +**Connection Arguments:** A list of arbitrary string key/value pairs as connection arguments. These arguments +will be passed to the JDBC driver as connection arguments for JDBC drivers that may need additional configurations. diff --git a/amazon-redshift-plugin/icons/Redshift-batchsource.png b/amazon-redshift-plugin/icons/Redshift-batchsource.png new file mode 100644 index 0000000000000000000000000000000000000000..11c334799daa890a320e5d4aa934d44a71bc988f GIT binary patch literal 6145 zcmZu#1yqzx+a^||Q;_adIwY4aC6*4Qdx2$PS&1bT0YOSB5m-P{K$=Aw=@M2-atS3C zqy(hl$M=1|@B8aHXP%k4=epy%=FEA{GcgAGT4cnG#5g!OWDm8~jc)poKaG&!=A9Vs z$#c`;`Wk7e;#7<2-`bB9h`*& zAl^4<92|uJxtk%x8E(%W0P*zll?woJ{)Ldc8UHbhaI*gefqMWsE%Xf7)u1qEb}3;= zVNp&+Vs>_R1(=hIoRPZbKkhd(Ag3D~?ky)G;_vS->@OhDAdiwoT# zgnR?N;PwGRUcOv^2l-bVb!T5k7}y&QhI+C8iEHlweF_J1a{fv5@8_R7A>RK^_$QiVSudEWxWws zk@zOlzcxydSU~s-5C?~L^r5W!WrW127Uq3WOr7C^6(52)IPjce(E0uiBta9&8DqCFWCwAOt zVy|_7-r0B-*v%1JTRD@l({?(tjYRg193K8W96r3aP&m^#90cvn@-(`yf90YLk9&&-Wm|%K8QXz1(z{; z>fEL99bj9DIec3ulx8C}OpWjHwr+s8)3J#3tZ5n7udB!qJ@wY|ZEGjDk_*vgq{KP& zS4WOq%C}#=*-Ob9vj=J>pnzCmVZw6)7ZV4mmT5~ z3Y|0z;lfl4UFs~uZ$%59szrESZZHl5rfCE?>Hzm?*D92sX+=6C`$LXVL*3GmmkL89 z;n1|zm$UwoDx=7Wl=raXq;FLpgA%-YjV{^WpQ&+XFlfEJHLY<(s>risp&+3!NS!h1 z6+i*{GC2O4znqEDEl%-g8K>Y;?d?@Z@xx>1% z$H1z?VjN4kp+h6NI+GSmZ=^^#%E~8_=JFM9d~li2vk>><5k+EE!4@{J*I(DJ5xMhx zhd;Y(o*xXLcr2bzl-G6OD#Ae-3~Cs@TMATiC$a@7L$h)%-~A@M(_|j$6MpXtUZ+g$ z>$h78Q)O3vOmJ@4R@Chwq>WVdTncqw76~wwN|&&H&6hVJ(@Oqv>zWqN@++JTS{9=R z`b3Ob>l`XXjPLmsGC50cRSEo%UTc`kmHu@|Jhw_Gm`qJv0UnTCe@@*N$Q@ksVEO*> z&Ub%Dp5F-^?`-c?MFx~8s&uK97nRFSJ1aJmA?|}j4wi$B*Yvk!(#{`s-rloSMuiO@ zT+E6AFC)22KU69A?DW8fVPE+hC6x!rd=BK|Q*4 zky(#(W#M3#nm4o7J|FTU8@eJ}6U?I|6%=|z0XH}K5L4-fmCAJT!KiE6bV7dpoWd;l zn$`n3I{Mzc=wa#4EIggz#E5z+uUylABTUkvq8~ zpwZ{Cp(;lKhIb2<)rm%k$gV&9cDKKJCxG|}6sAtOahZD60&}7=lhT#hWjW;fm41QJ z$FNr*gGN}bz;{!i5>SIl72-2Zw=u#E*ZyokTVUO*gTEdJ9d2g7pL?(rEx3QVm_b>l zgnS+7H&ynAs&ao)6~F(mXh)&oi)Z_@oaQFFCSwi0zL;Ml(=`}V9w_4_(Zz#O#AjtK zU}pT!q}Zy@)2PUz-dDXC$upv~`s zi76W(Uq!`&b8|G=V4_*Czj=O>anN-A?@|MHU$0feFOmL1-+IrHt%~{k+jO;Fj|Jbg zr`p`HJ?Uz{!rtd8+nZ+?sWb>H-MrlFB&5J=Q`$(SF8=lNV-PtsO?A#AJvN92``HKB z8jH;^Z4iU(&zV&lpmk=?{nH4oXut`A>OtQ zRs)gi&-Fa7jjzrspwt|n9CHr}+MQf6tuKs3w7Uoo>*dIK59OEw?HrF4>kVnn^kgo4 z!cTZ)I?YnHsi!~RCgxYLH@>%Gh1}Q=x zxgDFMcv+{9=f={SInK9n)=3POT`G#R;_oqEzc*GvHNNaC?bIIO%e9bEd^8lx_Q|k1 z7xl4zqC@8H3dkqUW0#HjYQV?qU>&41pyXLpD(7==s(SUxwH;sc3U3O9RQ^o5DS~=4 za2PNkIfEL7I_5J;zbTGLXEYil#}RHZU*LPC{+S_4=QYlJ2jQN!)#=HYh7WcnRc`>$ z8Y$^m+jE(^gk5_5n=XQK9fT*d>HIhO><7>VV*rL#9rbF>AnPPA@AWq0K4<6UChR>jFpIrzeC zndogOJnp9@@_rs$DtDrvG+zL%Q+XLxtFQ#1!I}Zh&a`(;CI9OF;vNyyC7s z2&ozVdZyS<%mmA*HxN|-GoPS<`)o<~S1`L#QW-v3fH9!Re2G`0xq(F_k8R@SAZpNr zXJVzenD*$7g_>@Y_LsVEl#FcJU$PCjIDJsg-+R5u($c1bUd&ZiSMJJWA^{W#(t6K- z)Ry*$7vWz{ouU}jnnpq7=-*7d?sy|z_C-jG8U!4oj&q|z(q+XG4ol{b-GNU>8R+y= zk9ujAemw7HCnA&B&>cg$#4EdMmyu=}m7}|u0;O1H3V2Ld=plmMIlyw458z?@J3E@G zUl3u24<&`DuSky>4*A&`!=n+a1oUI+!Wh4Dr7p1z74*J;7W!zyg=@c>MUkoW?} zhMI5p9l)!Vw4XrYw=Jy>ow@E*TKkqU(2)woE1kIn*sMJzx4s^?dfKwmbK>@%kXPQ2a!u zMT=N8Oa{^A{uA5!wO0C7*wRtc(|W21jOmdu4{OkwQMUR)LUhWv4T&6Kwxgc`m7Z5W zdyqSoIp~}y8ce3aKC$qXXJ795R$t&IcG?5LFnO+6U3_e_uHsUj*sszH*!dpV)b10t zgteySs%o*`rZ=U^bl4j8Cd`%2ZY-1O?#{yv^UN&e2cR>@64=yD@NZNMClY%bsXgfW zY9TC*{*XOas~$AIhdF%qhMNFb8cwiWhROSGj$(h@+t@9RF6F!itRWm0L(i44s5TLg!7JTIm?~iLJk|xHbofX@l zwJvWlYrPdXS9SHi`-Y3FGw?`ZA!nN&9(HM!401$AbZiQ}w&Ch8>Hi(0LV*|G;D?B3 z&qplwJsH0|jk~evnXt4VD+BZ>p1X7?UM0{Zy&V6@T8tuS*!&sTbS+q2++gIyopb(t zc1+z79YpG`qmH-#-RJ|~+J#nG)yuQn^*3gZqw1P4*=qq69R>&VlT!Y9^0sG<;~!eo zNB*dwPs>ucZ^OAhYC_u79;zC)**+U3Y8`&<#!mnQ0FUF<^Z7BC2*hG{Jl~Eoes!^b zZc7`*N89^&S4yxONs~P8Xhy_&p!w8|Fdqb`K6WW`~r7jQra#r2D4OPB~dqSZ`YOj~QPcGIHtu`o?edGQP zV|52XU5ULqi`f`Q)Rj$}EHA7Fk|I)T5EG=b zErUCoO}Gn>8PRchMH13XOY`ejaiMdpxAdd@&MmD-I0UiQJMp&p zraOna74A_wzGesGRd-LCDO6>=NZI?k2Yyr{FiPIfomyy-s_UHxRt32u#Cz)0i61Cm z4;JPZrJNn->S@0dBj(Z0<9U(BOlidzpR#2BfB{#Q&KfNi8E&SDPU6tGt^ZD!oV>bo z1#fwBuk_Y}Xv%S3aUGAm=ST>OV7plz#%`ZZ=WU*bL&m90d!n<>hl5qC>o1cM1~}=F zRM^X@CysTNOTIc_gUtnp*spzaY^n&$AdpE|d}KFS4GPn?h#vQ2NFr>lf+MxvDpQGT z+b709ELbS_0S@t^?~b9~zMx8?4M)Xf3>qf!T9>X8<4-I;cjP-9E_DK5g{^j82mbWKM6OK?h#4|H4C#JD2Vq?em3J)QM<`Az z?~7&1ivnMFbO@8(7S5BDHPv!V)U&ZxW#kv%H|7}zYBU{#}?_K zdjP6{z#*h-n>+0~1$%I?HMfLZ?V6iSjd+0fb1)A5ftqA^%cm-fd%{*uJ(2#R*e%`u zhEHi%r`QmAMc$Q3bMcIhU)90RX5*WWGbhT0-Qk^OPH95mMklt5z1KP??HwXMdAq4S`V9|KEQV7*YnyqMgq11WAZSv;KrnIEV3e!svBM{JhwqwuaT$n~SV_o1xDOO>OF zM|$N$qTxL`9L+)2dq092FOzE}%g@B^WY+!d1czz{NOT!r`Ud*(0ZdOLYJZtiymV-$ zS+7WscHpWw;HynxTkd3o36F_bC*S49UUtoMUB}y2HnTlA?H(_qbEy+~?B~-^FVhgQ zzjb|j_&M9vjnMkX%%gS!>4Q!E1Es6n^gh`o`Qb+7HF#&~RgU;?hen-K`5@{cyov!5 z%aTpXlbl{@^Y?3m*&B1-N)riZs~xz)UFwTGTBk|&CWPyDx10d*R5IKVzpMUGScx?D zfC%U-(pNirZ?xhov{5Z$*Dg2xwpozkv+K<|jl6dY&%$A8Grf|@t4|9$!wp$O$bxCk zXoH)$4L(wpJWNVrh(5W~toGx;l_AsNL47fO$AcYD?95Rx>NX+}&JgUi*K(c^1MLgK z%D1Ub;fzT}q-h*64kx*ORbt9dWxY&fR9&_uJG zXS@l?h$d;j8U1 zWh7h*i7a)DGv|9;jJeIv#}#3$PhOdL=RTpKP>@d;ytElu<&t^flOKc*w~AC%+TNGH z@MnUHw0sMYV_?_dAZx+(iO(SMW$WZle5|YF=W?w__?`4UJtt_cm0vo2_MwH~)rc?08laQF?douUlsQ((k z{k?KQ%X}{HbpV+1yzRLXvq>H3IGEg(TDfa$2v{&BW|qsOSvBHXFhN2GqQAmv6ia5U zx;qn(bMp*;=#^EMa%#U2OpW{mTbRW~g*p*xx|9LPTei40kVR}$2@ + + + + database-plugins-parent + io.cdap.plugin + 1.11.0-SNAPSHOT + + + Amazon Redshift plugin + amazon-redshift-plugin + 4.0.0 + + + 2.1.0.18 + + + + + redshift + http://redshift-maven-repository.s3-website-us-east-1.amazonaws.com/release + + + + + + io.cdap.cdap + cdap-etl-api + + + io.cdap.plugin + database-commons + ${project.version} + + + io.cdap.plugin + hydrator-common + + + com.google.guava + guava + + + + + com.amazon.redshift + redshift-jdbc42 + ${redshift-jdbc.version} + test + + + io.cdap.plugin + database-commons + ${project.version} + test-jar + test + + + io.cdap.cdap + hydrator-test + + + io.cdap.cdap + cdap-data-pipeline3_2.12 + + + junit + junit + + + io.cdap.cdap + cdap-api + provided + + + org.jetbrains + annotations + RELEASE + compile + + + + + + io.cdap + cdap-maven-plugin + + + org.apache.felix + maven-bundle-plugin + 5.1.2 + true + + + <_exportcontents> + io.cdap.plugin.amazon.redshift.*; + io.cdap.plugin.db.source.*; + org.apache.commons.lang; + org.apache.commons.logging.*; + org.codehaus.jackson.* + + *;inline=false;scope=compile + true + lib + + + + + package + + bundle + + + + + + + diff --git a/amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftConnector.java b/amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftConnector.java new file mode 100644 index 000000000..96ec94edf --- /dev/null +++ b/amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftConnector.java @@ -0,0 +1,110 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.plugin.amazon.redshift; + +import io.cdap.cdap.api.annotation.Category; +import io.cdap.cdap.api.annotation.Description; +import io.cdap.cdap.api.annotation.Name; +import io.cdap.cdap.api.annotation.Plugin; +import io.cdap.cdap.api.data.format.StructuredRecord; +import io.cdap.cdap.etl.api.batch.BatchSource; +import io.cdap.cdap.etl.api.connector.Connector; +import io.cdap.cdap.etl.api.connector.ConnectorSpec; +import io.cdap.cdap.etl.api.connector.ConnectorSpecRequest; +import io.cdap.cdap.etl.api.connector.PluginSpec; +import io.cdap.plugin.common.Constants; +import io.cdap.plugin.common.ReferenceNames; +import io.cdap.plugin.common.db.DBConnectorPath; +import io.cdap.plugin.common.db.DBPath; +import io.cdap.plugin.db.SchemaReader; +import io.cdap.plugin.db.connector.AbstractDBSpecificConnector; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.mapreduce.lib.db.DBWritable; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +/** + * Amazon Redshift Database Connector that connects to Amazon Redshift database via JDBC. + */ +@Plugin(type = Connector.PLUGIN_TYPE) +@Name(RedshiftConnector.NAME) +@Description("Connection to access data in Amazon Redshift using JDBC.") +@Category("Database") +public class RedshiftConnector extends AbstractDBSpecificConnector { + public static final String NAME = RedshiftConstants.PLUGIN_NAME; + private final RedshiftConnectorConfig config; + + public RedshiftConnector(RedshiftConnectorConfig config) { + super(config); + this.config = config; + } + + @Override + protected DBConnectorPath getDBConnectorPath(String path) throws IOException { + return new DBPath(path, true); + } + + @Override + public boolean supportSchema() { + return true; + } + + @Override + protected Class getDBRecordType() { + return RedshiftDBRecord.class; + } + + @Override + public StructuredRecord transform(LongWritable longWritable, RedshiftDBRecord redshiftDBRecord) { + return redshiftDBRecord.getRecord(); + } + + @Override + protected SchemaReader getSchemaReader(String sessionID) { + return new RedshiftSchemaReader(sessionID); + } + + @Override + protected String getTableName(String database, String schema, String table) { + return String.format("\"%s\".\"%s\"", schema, table); + } + + @Override + protected void setConnectorSpec(ConnectorSpecRequest request, DBConnectorPath path, + ConnectorSpec.Builder builder) { + Map sourceProperties = new HashMap<>(); + setConnectionProperties(sourceProperties, request); + builder + .addRelatedPlugin(new PluginSpec(RedshiftConstants.PLUGIN_NAME, + BatchSource.PLUGIN_TYPE, sourceProperties)); + + String schema = path.getSchema(); + sourceProperties.put(RedshiftSource.RedshiftSourceConfig.NUM_SPLITS, "1"); + sourceProperties.put(RedshiftSource.RedshiftSourceConfig.FETCH_SIZE, + RedshiftSource.RedshiftSourceConfig.DEFAULT_FETCH_SIZE); + String table = path.getTable(); + if (table == null) { + return; + } + sourceProperties.put(RedshiftSource.RedshiftSourceConfig.IMPORT_QUERY, + getTableQuery(path.getDatabase(), schema, table)); + sourceProperties.put(Constants.Reference.REFERENCE_NAME, ReferenceNames.cleanseReferenceName(table)); + } + +} diff --git a/amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftConnectorConfig.java b/amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftConnectorConfig.java new file mode 100644 index 000000000..f05f26d10 --- /dev/null +++ b/amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftConnectorConfig.java @@ -0,0 +1,87 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.plugin.amazon.redshift; + +import io.cdap.cdap.api.annotation.Description; +import io.cdap.cdap.api.annotation.Macro; +import io.cdap.cdap.api.annotation.Name; +import io.cdap.plugin.db.ConnectionConfig; +import io.cdap.plugin.db.connector.AbstractDBConnectorConfig; + +import javax.annotation.Nullable; + +/** + * Configuration for Redshift connector + */ +public class RedshiftConnectorConfig extends AbstractDBConnectorConfig { + + @Name(ConnectionConfig.HOST) + @Description( + "The endpoint of the Amazon Redshift cluster.") + @Macro + private String host; + + @Name(ConnectionConfig.PORT) + @Description("Database port number") + @Macro + @Nullable + private Integer port; + + @Name(ConnectionConfig.DATABASE) + @Description("Database name to connect to") + @Macro + private String database; + + public RedshiftConnectorConfig(String username, String password, String jdbcPluginName, + String connectionArguments, String host, + String database, @Nullable Integer port) { + this.user = username; + this.password = password; + this.jdbcPluginName = jdbcPluginName; + this.connectionArguments = connectionArguments; + this.host = host; + this.database = database; + this.port = port; + } + + public String getDatabase() { + return database; + } + + public String getHost() { + return host; + } + + public int getPort() { + return port == null ? 5439 : port; + } + + @Override + public String getConnectionString() { + return String.format( + RedshiftConstants.REDSHIFT_CONNECTION_STRING_FORMAT, + host, + getPort(), + database); + } + + @Override + public boolean canConnect() { + return super.canConnect() && !containsMacro(ConnectionConfig.HOST) && + !containsMacro(ConnectionConfig.PORT) && !containsMacro(ConnectionConfig.DATABASE); + } +} diff --git a/amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftConstants.java b/amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftConstants.java new file mode 100644 index 000000000..081052fb1 --- /dev/null +++ b/amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftConstants.java @@ -0,0 +1,27 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.plugin.amazon.redshift; + +/** Amazon Redshift constants. */ +public final class RedshiftConstants { + + private RedshiftConstants() { + } + + public static final String PLUGIN_NAME = "Redshift"; + public static final String REDSHIFT_CONNECTION_STRING_FORMAT = "jdbc:redshift://%s:%s/%s"; +} diff --git a/amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftDBRecord.java b/amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftDBRecord.java new file mode 100644 index 000000000..f6935bc29 --- /dev/null +++ b/amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftDBRecord.java @@ -0,0 +1,130 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.plugin.amazon.redshift; + +import io.cdap.cdap.api.data.format.StructuredRecord; +import io.cdap.cdap.api.data.schema.Schema; +import io.cdap.plugin.db.DBRecord; +import io.cdap.plugin.db.SchemaReader; +import io.cdap.plugin.util.DBUtils; + +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Timestamp; +import java.sql.Types; +import java.time.OffsetDateTime; +import java.time.ZoneId; +import java.time.ZonedDateTime; + +/** + * Writable class for Redshift Source + */ +public class RedshiftDBRecord extends DBRecord { + + /** + * Used in map-reduce. Do not remove. + */ + @SuppressWarnings("unused") + public RedshiftDBRecord() { + } + + @Override + protected void handleField(ResultSet resultSet, StructuredRecord.Builder recordBuilder, Schema.Field field, + int columnIndex, int sqlType, int sqlPrecision, int sqlScale) throws SQLException { + ResultSetMetaData metadata = resultSet.getMetaData(); + String columnTypeName = metadata.getColumnTypeName(columnIndex); + if (isUseSchema(metadata, columnIndex)) { + setFieldAccordingToSchema(resultSet, recordBuilder, field, columnIndex); + return; + } + + // HandleTimestamp + if (sqlType == Types.TIMESTAMP && columnTypeName.equalsIgnoreCase("timestamp")) { + Timestamp timestamp = resultSet.getTimestamp(columnIndex, DBUtils.PURE_GREGORIAN_CALENDAR); + if (timestamp != null) { + ZonedDateTime zonedDateTime = OffsetDateTime.of(timestamp.toLocalDateTime(), OffsetDateTime.now().getOffset()) + .atZoneSameInstant(ZoneId.of("UTC")); + Schema nonNullableSchema = field.getSchema().isNullable() ? + field.getSchema().getNonNullable() : field.getSchema(); + setZonedDateTimeBasedOnOutputSchema(recordBuilder, nonNullableSchema.getLogicalType(), + field.getName(), zonedDateTime); + } else { + recordBuilder.set(field.getName(), null); + } + return; + } + + // HandleTimestampTZ + if (sqlType == Types.TIMESTAMP && columnTypeName.equalsIgnoreCase("timestamptz")) { + OffsetDateTime timestamp = resultSet.getObject(columnIndex, OffsetDateTime.class); + if (timestamp != null) { + recordBuilder.setTimestamp(field.getName(), timestamp.atZoneSameInstant(ZoneId.of("UTC"))); + } else { + recordBuilder.set(field.getName(), null); + } + return; + } + + // HandleNumeric + int columnType = metadata.getColumnType(columnIndex); + if (columnType == Types.NUMERIC) { + Schema nonNullableSchema = field.getSchema().isNullable() ? + field.getSchema().getNonNullable() : field.getSchema(); + int precision = metadata.getPrecision(columnIndex); + if (precision == 0 && Schema.Type.STRING.equals(nonNullableSchema.getType())) { + // When output schema is set to String for precision less numbers + recordBuilder.set(field.getName(), resultSet.getString(columnIndex)); + } else if (Schema.LogicalType.DECIMAL.equals(nonNullableSchema.getLogicalType())) { + BigDecimal orgValue = resultSet.getBigDecimal(columnIndex); + if (orgValue != null) { + BigDecimal decimalValue = new BigDecimal(orgValue.toPlainString()) + .setScale(nonNullableSchema.getScale(), RoundingMode.HALF_EVEN); + recordBuilder.setDecimal(field.getName(), decimalValue); + } + } + return; + } + setField(resultSet, recordBuilder, field, columnIndex, sqlType, sqlPrecision, sqlScale); + } + + private void setZonedDateTimeBasedOnOutputSchema(StructuredRecord.Builder recordBuilder, + Schema.LogicalType logicalType, + String fieldName, + ZonedDateTime zonedDateTime) { + if (Schema.LogicalType.DATETIME.equals(logicalType)) { + recordBuilder.setDateTime(fieldName, zonedDateTime.toLocalDateTime()); + } else if (Schema.LogicalType.TIMESTAMP_MICROS.equals(logicalType)) { + recordBuilder.setTimestamp(fieldName, zonedDateTime); + } + } + + private static boolean isUseSchema(ResultSetMetaData metadata, int columnIndex) throws SQLException { + String columnTypeName = metadata.getColumnTypeName(columnIndex); + // If the column Type Name is present in the String mapped Redshift types then return true. + return RedshiftSchemaReader.STRING_MAPPED_REDSHIFT_TYPES_NAMES.contains(columnTypeName) + || RedshiftSchemaReader.STRING_MAPPED_REDSHIFT_TYPES.contains(metadata.getColumnType(columnIndex)); + } + + @Override + protected SchemaReader getSchemaReader() { + return new RedshiftSchemaReader(); + } + +} diff --git a/amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftSchemaReader.java b/amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftSchemaReader.java new file mode 100644 index 000000000..5df35940e --- /dev/null +++ b/amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftSchemaReader.java @@ -0,0 +1,121 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.plugin.amazon.redshift; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import io.cdap.cdap.api.data.schema.Schema; +import io.cdap.plugin.db.CommonSchemaReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Types; +import java.util.List; +import java.util.Set; + +/** + * Redshift Schema Reader class + */ +public class RedshiftSchemaReader extends CommonSchemaReader { + + private static final Logger LOG = LoggerFactory.getLogger(RedshiftSchemaReader.class); + + public static final Set STRING_MAPPED_REDSHIFT_TYPES = ImmutableSet.of( + Types.OTHER, Types.ARRAY, Types.SQLXML + ); + + public static final Set STRING_MAPPED_REDSHIFT_TYPES_NAMES = ImmutableSet.of( + "timetz", "money" + ); + + private final String sessionID; + + public RedshiftSchemaReader() { + this(null); + } + + public RedshiftSchemaReader(String sessionID) { + super(); + this.sessionID = sessionID; + } + + @Override + public Schema getSchema(ResultSetMetaData metadata, int index) throws SQLException { + String typeName = metadata.getColumnTypeName(index); + int columnType = metadata.getColumnType(index); + + if (STRING_MAPPED_REDSHIFT_TYPES.contains(columnType) || STRING_MAPPED_REDSHIFT_TYPES_NAMES.contains(typeName)) { + return Schema.of(Schema.Type.STRING); + } + if (typeName.equalsIgnoreCase("INT")) { + return Schema.of(Schema.Type.INT); + } + if (typeName.equalsIgnoreCase("BIGINT")) { + return Schema.of(Schema.Type.LONG); + } + + // If it is a numeric type without precision then use the Schema of String to avoid any precision loss + if (Types.NUMERIC == columnType) { + int precision = metadata.getPrecision(index); + if (precision == 0) { + LOG.warn(String.format("Field '%s' is a %s type without precision and scale, " + + "converting into STRING type to avoid any precision loss.", + metadata.getColumnName(index), + metadata.getColumnTypeName(index))); + return Schema.of(Schema.Type.STRING); + } + } + + if (typeName.equalsIgnoreCase("timestamp")) { + return Schema.of(Schema.LogicalType.DATETIME); + } + + return super.getSchema(metadata, index); + } + + @Override + public boolean shouldIgnoreColumn(ResultSetMetaData metadata, int index) throws SQLException { + if (sessionID == null) { + return false; + } + return metadata.getColumnName(index).equals("c_" + sessionID) || + metadata.getColumnName(index).equals("sqn_" + sessionID); + } + + @Override + public List getSchemaFields(ResultSet resultSet) throws SQLException { + List schemaFields = Lists.newArrayList(); + ResultSetMetaData metadata = resultSet.getMetaData(); + // ResultSetMetadata columns are numbered starting with 1 + for (int i = 1; i <= metadata.getColumnCount(); i++) { + if (shouldIgnoreColumn(metadata, i)) { + continue; + } + String columnName = metadata.getColumnName(i); + Schema columnSchema = getSchema(metadata, i); + // Setting up schema as nullable as cdata driver doesn't provide proper information about isNullable. + columnSchema = Schema.nullableOf(columnSchema); + Schema.Field field = Schema.Field.of(columnName, columnSchema); + schemaFields.add(field); + } + return schemaFields; + } + +} diff --git a/amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftSource.java b/amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftSource.java new file mode 100644 index 000000000..6924212eb --- /dev/null +++ b/amazon-redshift-plugin/src/main/java/io/cdap/plugin/amazon/redshift/RedshiftSource.java @@ -0,0 +1,131 @@ +/* + * Copyright © 2020 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.plugin.amazon.redshift; + +import io.cdap.cdap.api.annotation.Description; +import io.cdap.cdap.api.annotation.Macro; +import io.cdap.cdap.api.annotation.Metadata; +import io.cdap.cdap.api.annotation.MetadataProperty; +import io.cdap.cdap.api.annotation.Name; +import io.cdap.cdap.api.annotation.Plugin; +import io.cdap.cdap.etl.api.FailureCollector; +import io.cdap.cdap.etl.api.PipelineConfigurer; +import io.cdap.cdap.etl.api.batch.BatchSource; +import io.cdap.cdap.etl.api.batch.BatchSourceContext; +import io.cdap.cdap.etl.api.connector.Connector; +import io.cdap.plugin.common.Asset; +import io.cdap.plugin.common.ConfigUtil; +import io.cdap.plugin.common.LineageRecorder; +import io.cdap.plugin.db.SchemaReader; +import io.cdap.plugin.db.config.AbstractDBSpecificSourceConfig; +import io.cdap.plugin.db.source.AbstractDBSource; +import io.cdap.plugin.util.DBUtils; +import org.apache.hadoop.mapreduce.lib.db.DBWritable; + +import java.util.Collections; +import java.util.Map; +import javax.annotation.Nullable; + +/** Batch source to read from an Amazon Redshift database. */ +@Plugin(type = BatchSource.PLUGIN_TYPE) +@Name(RedshiftConstants.PLUGIN_NAME) +@Description( + "Reads from a Amazon Redshift database table(s) using a configurable SQL query." + + " Outputs one record for each row returned by the query.") +@Metadata(properties = {@MetadataProperty(key = Connector.PLUGIN_TYPE, value = RedshiftConnector.NAME)}) +public class RedshiftSource + extends AbstractDBSource { + + private final RedshiftSourceConfig redshiftSourceConfig; + + public RedshiftSource(RedshiftSourceConfig redshiftSourceConfig) { + super(redshiftSourceConfig); + this.redshiftSourceConfig = redshiftSourceConfig; + } + + @Override + public void configurePipeline(PipelineConfigurer pipelineConfigurer) { + super.configurePipeline(pipelineConfigurer); + } + + @Override + protected SchemaReader getSchemaReader() { + return new RedshiftSchemaReader(); + } + + @Override + protected Class getDBRecordType() { + return RedshiftDBRecord.class; + } + + @Override + protected String createConnectionString() { + return String.format( + RedshiftConstants.REDSHIFT_CONNECTION_STRING_FORMAT, + redshiftSourceConfig.connection.getHost(), + redshiftSourceConfig.connection.getPort(), + redshiftSourceConfig.connection.getDatabase()); + } + + @Override + protected LineageRecorder getLineageRecorder(BatchSourceContext context) { + String host = redshiftSourceConfig.getConnection().getHost(); + String fqn = DBUtils.constructFQN("redshift", host, + redshiftSourceConfig.getConnection().getPort(), + redshiftSourceConfig.getConnection().getDatabase(), + redshiftSourceConfig.getReferenceName()); + Asset.Builder assetBuilder = Asset.builder(redshiftSourceConfig.getReferenceName()).setFqn(fqn); + return new LineageRecorder(context, assetBuilder.build()); + } + + /** Redshift source config. */ + public static class RedshiftSourceConfig extends AbstractDBSpecificSourceConfig { + + @Name(ConfigUtil.NAME_USE_CONNECTION) + @Nullable + @Description("Whether to use an existing connection.") + private Boolean useConnection; + + @Name(ConfigUtil.NAME_CONNECTION) + @Macro + @Nullable + @Description("The existing connection to use.") + private RedshiftConnectorConfig connection; + + @Override + public Map getDBSpecificArguments() { + return Collections.emptyMap(); + } + + @Override + public Integer getFetchSize() { + Integer fetchSize = super.getFetchSize(); + return fetchSize == null ? Integer.parseInt(DEFAULT_FETCH_SIZE) : fetchSize; + } + + @Override + protected RedshiftConnectorConfig getConnection() { + return connection; + } + + @Override + public void validate(FailureCollector collector) { + ConfigUtil.validateConnection(this, useConnection, connection, collector); + super.validate(collector); + } + } +} diff --git a/amazon-redshift-plugin/src/test/java/io/cdap/plugin/amazon/redshift/RedshiftPluginTestBase.java b/amazon-redshift-plugin/src/test/java/io/cdap/plugin/amazon/redshift/RedshiftPluginTestBase.java new file mode 100644 index 000000000..5df4fb300 --- /dev/null +++ b/amazon-redshift-plugin/src/test/java/io/cdap/plugin/amazon/redshift/RedshiftPluginTestBase.java @@ -0,0 +1,218 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.plugin.amazon.redshift; + +import com.google.common.base.Charsets; +import com.google.common.base.Throwables; +import com.google.common.collect.Sets; +import io.cdap.cdap.api.artifact.ArtifactSummary; +import io.cdap.cdap.api.plugin.PluginClass; +import io.cdap.cdap.datapipeline.DataPipelineApp; +import io.cdap.cdap.proto.id.ArtifactId; +import io.cdap.cdap.proto.id.NamespaceId; +import io.cdap.plugin.db.ConnectionConfig; +import io.cdap.plugin.db.DBRecord; +import io.cdap.plugin.db.batch.DatabasePluginTestBase; +import io.cdap.plugin.db.sink.ETLDBOutputFormat; +import io.cdap.plugin.db.source.DataDrivenETLDBInputFormat; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.math.BigDecimal; +import java.sql.Connection; +import java.sql.Date; +import java.sql.Driver; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Time; +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.Calendar; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.TimeZone; + +/** + * Base test class for Redshift plugins. + */ +public abstract class RedshiftPluginTestBase extends DatabasePluginTestBase { + private static final Logger LOGGER = LoggerFactory.getLogger(RedshiftPluginTestBase.class); + protected static final ArtifactId DATAPIPELINE_ARTIFACT_ID = NamespaceId.DEFAULT.artifact("data-pipeline", "3.2.0"); + protected static final ArtifactSummary DATAPIPELINE_ARTIFACT = new ArtifactSummary("data-pipeline", "3.2.0"); + protected static final long CURRENT_TS = System.currentTimeMillis(); + + protected static final String JDBC_DRIVER_NAME = "redshift"; + protected static final Map BASE_PROPS = new HashMap<>(); + + protected static String connectionUrl; + protected static int year; + protected static final int PRECISION = 10; + protected static final int SCALE = 6; + private static int startCount; + + @BeforeClass + public static void setupTest() throws Exception { + if (startCount++ > 0) { + return; + } + + getProperties(); + + Calendar calendar = Calendar.getInstance(); + calendar.setTime(new Date(CURRENT_TS)); + year = calendar.get(Calendar.YEAR); + + setupBatchArtifacts(DATAPIPELINE_ARTIFACT_ID, DataPipelineApp.class); + + addPluginArtifact(NamespaceId.DEFAULT.artifact(JDBC_DRIVER_NAME, "1.0.0"), + DATAPIPELINE_ARTIFACT_ID, + RedshiftSource.class, DBRecord.class, + ETLDBOutputFormat.class, DataDrivenETLDBInputFormat.class, DBRecord.class); + + // add mysql 3rd party plugin + PluginClass mysqlDriver = new PluginClass(ConnectionConfig.JDBC_PLUGIN_TYPE, JDBC_DRIVER_NAME, + "redshift driver class", Driver.class.getName(), + null, Collections.emptyMap()); + addPluginArtifact(NamespaceId.DEFAULT.artifact("redshift-jdbc-connector", "1.0.0"), + DATAPIPELINE_ARTIFACT_ID, + Sets.newHashSet(mysqlDriver), Driver.class); + + TimeZone.setDefault(TimeZone.getTimeZone("UTC")); + + connectionUrl = "jdbc:redshift://" + BASE_PROPS.get(ConnectionConfig.HOST) + ":" + + BASE_PROPS.get(ConnectionConfig.PORT) + "/" + BASE_PROPS.get(ConnectionConfig.DATABASE); + Connection conn = createConnection(); + createTestTables(conn); + prepareTestData(conn); + } + + private static void getProperties() { + BASE_PROPS.put(ConnectionConfig.HOST, getPropertyOrSkip("redshift.clusterEndpoint")); + BASE_PROPS.put(ConnectionConfig.PORT, getPropertyOrSkip("redshift.port")); + BASE_PROPS.put(ConnectionConfig.DATABASE, getPropertyOrSkip("redshift.database")); + BASE_PROPS.put(ConnectionConfig.USER, getPropertyOrSkip("redshift.username")); + BASE_PROPS.put(ConnectionConfig.PASSWORD, getPropertyOrSkip("redshift.password")); + BASE_PROPS.put(ConnectionConfig.JDBC_PLUGIN_NAME, JDBC_DRIVER_NAME); + } + + protected static void createTestTables(Connection conn) throws SQLException { + try (Statement stmt = conn.createStatement()) { + // create a table that the action will truncate at the end of the run + stmt.execute("CREATE TABLE \"dbActionTest\" (x int, day varchar(10))"); + // create a table that the action will truncate at the end of the run + stmt.execute("CREATE TABLE \"postActionTest\" (x int, day varchar(10))"); + + stmt.execute("CREATE TABLE my_table" + + "(" + + "\"ID\" INT NOT NULL," + + "\"NAME\" VARCHAR(40) NOT NULL," + + "\"SCORE\" REAL," + + "\"GRADUATED\" BOOLEAN," + + "\"NOT_IMPORTED\" VARCHAR(30)," + + "\"SMALLINT_COL\" SMALLINT," + + "\"BIG\" BIGINT," + + "\"NUMERIC_COL\" NUMERIC(" + PRECISION + "," + SCALE + ")," + + "\"DECIMAL_COL\" DECIMAL(" + PRECISION + "," + SCALE + ")," + + "\"DOUBLE_PREC_COL\" DOUBLE PRECISION," + + "\"DATE_COL\" DATE," + + "\"TIME_COL\" TIME," + + "\"TIMESTAMP_COL\" TIMESTAMP(3)," + + "\"TEXT_COL\" TEXT," + + "\"CHAR_COL\" CHAR(100)," + + "\"BYTEA_COL\" BYTEA" + + ")"); + stmt.execute("CREATE TABLE \"MY_DEST_TABLE\" AS " + + "SELECT * FROM my_table"); + stmt.execute("CREATE TABLE your_table AS " + + "SELECT * FROM my_table"); + } + } + + protected static void prepareTestData(Connection conn) throws SQLException { + try ( + Statement stmt = conn.createStatement(); + PreparedStatement pStmt1 = + conn.prepareStatement("INSERT INTO my_table " + + "VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?," + + " ?, ?, ?, ?, ?, ?)"); + PreparedStatement pStmt2 = + conn.prepareStatement("INSERT INTO your_table " + + "VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?," + + " ?, ?, ?, ?, ?, ?)")) { + + stmt.execute("insert into \"dbActionTest\" values (1, '1970-01-01')"); + stmt.execute("insert into \"postActionTest\" values (1, '1970-01-01')"); + + populateData(pStmt1, pStmt2); + } + } + + private static void populateData(PreparedStatement... stmts) throws SQLException { + // insert the same data into both tables: my_table and your_table + for (PreparedStatement pStmt : stmts) { + for (int i = 1; i <= 5; i++) { + String name = "user" + i; + pStmt.setInt(1, i); + pStmt.setString(2, name); + pStmt.setDouble(3, 123.45 + i); + pStmt.setBoolean(4, (i % 2 == 0)); + pStmt.setString(5, "random" + i); + pStmt.setShort(6, (short) i); + pStmt.setLong(7, (long) i); + pStmt.setBigDecimal(8, new BigDecimal("123.45").add(new BigDecimal(i))); + pStmt.setBigDecimal(9, new BigDecimal("123.45").add(new BigDecimal(i))); + pStmt.setDouble(10, 123.45 + i); + pStmt.setDate(11, new Date(CURRENT_TS)); + pStmt.setTime(12, new Time(CURRENT_TS)); + pStmt.setTimestamp(13, new Timestamp(CURRENT_TS)); + pStmt.setString(14, name); + pStmt.setString(15, "char" + i); + pStmt.setBytes(16, name.getBytes(Charsets.UTF_8)); + pStmt.executeUpdate(); + } + } + } + + public static Connection createConnection() { + try { + Class.forName(Driver.class.getCanonicalName()); + return DriverManager.getConnection(connectionUrl, BASE_PROPS.get(ConnectionConfig.USER), + BASE_PROPS.get(ConnectionConfig.PASSWORD)); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + + @AfterClass + public static void tearDownDB() { + try (Connection conn = createConnection(); + Statement stmt = conn.createStatement()) { + executeCleanup(Arrays.asList(() -> stmt.execute("DROP TABLE my_table"), + () -> stmt.execute("DROP TABLE your_table"), + () -> stmt.execute("DROP TABLE postActionTest"), + () -> stmt.execute("DROP TABLE dbActionTest"), + () -> stmt.execute("DROP TABLE MY_DEST_TABLE")), LOGGER); + } catch (Exception e) { + LOGGER.warn("Fail to tear down.", e); + } + } +} diff --git a/amazon-redshift-plugin/src/test/java/io/cdap/plugin/amazon/redshift/RedshiftPluginTestSuite.java b/amazon-redshift-plugin/src/test/java/io/cdap/plugin/amazon/redshift/RedshiftPluginTestSuite.java new file mode 100644 index 000000000..95ad0938b --- /dev/null +++ b/amazon-redshift-plugin/src/test/java/io/cdap/plugin/amazon/redshift/RedshiftPluginTestSuite.java @@ -0,0 +1,31 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.plugin.amazon.redshift; + +import io.cdap.cdap.common.test.TestSuite; +import org.junit.runner.RunWith; +import org.junit.runners.Suite; + +/** + * This is a test suite that runs all the tests for Redshift plugins. + */ +@RunWith(TestSuite.class) +@Suite.SuiteClasses({ + RedshiftSourceTestRun.class, +}) +public class RedshiftPluginTestSuite extends RedshiftPluginTestBase { +} diff --git a/amazon-redshift-plugin/src/test/java/io/cdap/plugin/amazon/redshift/RedshiftSourceTestRun.java b/amazon-redshift-plugin/src/test/java/io/cdap/plugin/amazon/redshift/RedshiftSourceTestRun.java new file mode 100644 index 000000000..1ac41bcd0 --- /dev/null +++ b/amazon-redshift-plugin/src/test/java/io/cdap/plugin/amazon/redshift/RedshiftSourceTestRun.java @@ -0,0 +1,332 @@ +/* + * Copyright © 2023 Cask Data, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package io.cdap.plugin.amazon.redshift; + +import com.google.common.collect.ImmutableMap; +import io.cdap.cdap.api.common.Bytes; +import io.cdap.cdap.api.data.format.StructuredRecord; +import io.cdap.cdap.api.dataset.table.Table; +import io.cdap.cdap.etl.api.batch.BatchSource; +import io.cdap.cdap.etl.mock.batch.MockSink; +import io.cdap.cdap.etl.proto.v2.ETLBatchConfig; +import io.cdap.cdap.etl.proto.v2.ETLPlugin; +import io.cdap.cdap.etl.proto.v2.ETLStage; +import io.cdap.cdap.proto.artifact.AppRequest; +import io.cdap.cdap.proto.id.ApplicationId; +import io.cdap.cdap.proto.id.NamespaceId; +import io.cdap.cdap.test.ApplicationManager; +import io.cdap.cdap.test.DataSetManager; +import io.cdap.plugin.common.Constants; +import io.cdap.plugin.db.ConnectionConfig; +import io.cdap.plugin.db.DBConfig; +import io.cdap.plugin.db.source.AbstractDBSource; +import org.junit.Assert; +import org.junit.Test; + +import java.math.BigDecimal; +import java.math.MathContext; +import java.nio.ByteBuffer; +import java.sql.Date; +import java.sql.Time; +import java.text.SimpleDateFormat; +import java.time.LocalDate; +import java.time.LocalTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Test for Redshift source plugin. + */ +public class RedshiftSourceTestRun extends RedshiftPluginTestBase { + + @Test + @SuppressWarnings("ConstantConditions") + public void testDBMacroSupport() throws Exception { + String importQuery = "SELECT * FROM my_table WHERE \"DATE_COL\" <= '${logicalStartTime(yyyy-MM-dd,1d)}' " + + "AND $CONDITIONS"; + String boundingQuery = "SELECT MIN(ID),MAX(ID) from my_table"; + String splitBy = "ID"; + + ImmutableMap sourceProps = ImmutableMap.builder() + .putAll(BASE_PROPS) + .put(AbstractDBSource.DBSourceConfig.IMPORT_QUERY, importQuery) + .put(AbstractDBSource.DBSourceConfig.BOUNDING_QUERY, boundingQuery) + .put(AbstractDBSource.DBSourceConfig.SPLIT_BY, splitBy) + .put(Constants.Reference.REFERENCE_NAME, "DBTestSource").build(); + + ETLPlugin sourceConfig = new ETLPlugin( + RedshiftConstants.PLUGIN_NAME, + BatchSource.PLUGIN_TYPE, + sourceProps + ); + + ETLPlugin sinkConfig = MockSink.getPlugin("macroOutputTable"); + + ApplicationManager appManager = deployETL(sourceConfig, sinkConfig, + DATAPIPELINE_ARTIFACT, "testDBMacro"); + runETLOnce(appManager, ImmutableMap.of("logical.start.time", String.valueOf(CURRENT_TS))); + + DataSetManager outputManager = getDataset("macroOutputTable"); + Assert.assertTrue(MockSink.readOutput(outputManager).isEmpty()); + } + + @Test + @SuppressWarnings("ConstantConditions") + public void testDBSource() throws Exception { + String importQuery = "SELECT \"ID\", \"NAME\", \"SCORE\", \"GRADUATED\", \"SMALLINT_COL\", \"BIG\", " + + "\"NUMERIC_COL\", \"CHAR_COL\", \"DECIMAL_COL\", \"BYTEA_COL\", \"DATE_COL\", \"TIME_COL\", \"TIMESTAMP_COL\", " + + "\"TEXT_COL\", \"DOUBLE_PREC_COL\" FROM my_table " + + "WHERE \"ID\" < 3 AND $CONDITIONS"; + String boundingQuery = "SELECT MIN(\"ID\"),MAX(\"ID\") from my_table"; + String splitBy = "ID"; + ETLPlugin sourceConfig = new ETLPlugin( + RedshiftConstants.PLUGIN_NAME, + BatchSource.PLUGIN_TYPE, + ImmutableMap.builder() + .putAll(BASE_PROPS) + .put(AbstractDBSource.DBSourceConfig.IMPORT_QUERY, importQuery) + .put(AbstractDBSource.DBSourceConfig.BOUNDING_QUERY, boundingQuery) + .put(AbstractDBSource.DBSourceConfig.SPLIT_BY, splitBy) + .put(Constants.Reference.REFERENCE_NAME, "DBSourceTest") + .build(), + null + ); + + String outputDatasetName = "output-dbsourcetest"; + ETLPlugin sinkConfig = MockSink.getPlugin(outputDatasetName); + + ApplicationManager appManager = deployETL(sourceConfig, sinkConfig, + DATAPIPELINE_ARTIFACT, "testDBSource"); + runETLOnce(appManager); + + DataSetManager
outputManager = getDataset(outputDatasetName); + List outputRecords = MockSink.readOutput(outputManager); + + Assert.assertEquals(2, outputRecords.size()); + String userid = outputRecords.get(0).get("NAME"); + StructuredRecord row1 = "user1".equals(userid) ? outputRecords.get(0) : outputRecords.get(1); + StructuredRecord row2 = "user1".equals(userid) ? outputRecords.get(1) : outputRecords.get(0); + + // Verify data + Assert.assertEquals("user1", row1.get("NAME")); + Assert.assertEquals("user2", row2.get("NAME")); + Assert.assertEquals("user1", row1.get("TEXT_COL")); + Assert.assertEquals("user2", row2.get("TEXT_COL")); + Assert.assertEquals("char1", ((String) row1.get("CHAR_COL")).trim()); + Assert.assertEquals("char2", ((String) row2.get("CHAR_COL")).trim()); + Assert.assertEquals(124.45f, ((Float) row1.get("SCORE")).doubleValue(), 0.000001); + Assert.assertEquals(125.45f, ((Float) row2.get("SCORE")).doubleValue(), 0.000001); + Assert.assertEquals(false, row1.get("GRADUATED")); + Assert.assertEquals(true, row2.get("GRADUATED")); + Assert.assertNull(row1.get("NOT_IMPORTED")); + Assert.assertNull(row2.get("NOT_IMPORTED")); + + Assert.assertEquals(1, (int) row1.get("SMALLINT_COL")); + Assert.assertEquals(2, (int) row2.get("SMALLINT_COL")); + Assert.assertEquals(1, (long) row1.get("BIG")); + Assert.assertEquals(2, (long) row2.get("BIG")); + + Assert.assertEquals(new BigDecimal("124.45", new MathContext(PRECISION)).setScale(SCALE), + row1.getDecimal("NUMERIC_COL")); + Assert.assertEquals(new BigDecimal("125.45", new MathContext(PRECISION)).setScale(SCALE), + row2.getDecimal("NUMERIC_COL")); + Assert.assertEquals(new BigDecimal("124.45", new MathContext(PRECISION)).setScale(SCALE), + row1.getDecimal("DECIMAL_COL")); + + Assert.assertEquals(124.45, (double) row1.get("DOUBLE_PREC_COL"), 0.000001); + Assert.assertEquals(125.45, (double) row2.get("DOUBLE_PREC_COL"), 0.000001); + // Verify time columns + java.util.Date date = new java.util.Date(CURRENT_TS); + SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd"); + LocalDate expectedDate = Date.valueOf(sdf.format(date)).toLocalDate(); + sdf = new SimpleDateFormat("H:mm:ss"); + LocalTime expectedTime = Time.valueOf(sdf.format(date)).toLocalTime(); + ZonedDateTime expectedTs = date.toInstant().atZone(ZoneId.ofOffset("UTC", ZoneOffset.UTC)); + Assert.assertEquals(expectedDate, row1.getDate("DATE_COL")); + Assert.assertEquals(expectedTime, row1.getTime("TIME_COL")); + Assert.assertEquals(expectedTs, row1.getTimestamp("TIMESTAMP_COL", ZoneId.ofOffset("UTC", ZoneOffset.UTC))); + + // verify binary columns + Assert.assertEquals("user1", Bytes.toString(((ByteBuffer) row1.get("BYTEA_COL")).array(), 0, 5)); + Assert.assertEquals("user2", Bytes.toString(((ByteBuffer) row2.get("BYTEA_COL")).array(), 0, 5)); + } + + @Test + public void testDbSourceMultipleTables() throws Exception { + String importQuery = "SELECT \"my_table\".\"ID\", \"your_table\".\"NAME\" FROM \"my_table\", \"your_table\"" + + "WHERE \"my_table\".\"ID\" < 3 and \"my_table\".\"ID\" = \"your_table\".\"ID\" and $CONDITIONS"; + String boundingQuery = "SELECT MIN(MIN(\"my_table\".\"ID\"), MIN(\"your_table\".\"ID\")), " + + "MAX(MAX(\"my_table\".\"ID\"), MAX(\"your_table\".\"ID\"))"; + String splitBy = "\"my_table\".\"ID\""; + ETLPlugin sourceConfig = new ETLPlugin( + RedshiftConstants.PLUGIN_NAME, + BatchSource.PLUGIN_TYPE, + ImmutableMap.builder() + .putAll(BASE_PROPS) + .put(AbstractDBSource.DBSourceConfig.IMPORT_QUERY, importQuery) + .put(AbstractDBSource.DBSourceConfig.BOUNDING_QUERY, boundingQuery) + .put(AbstractDBSource.DBSourceConfig.SPLIT_BY, splitBy) + .put(Constants.Reference.REFERENCE_NAME, "DBMultipleTest") + .build(), + null + ); + + String outputDatasetName = "output-multitabletest"; + ETLPlugin sinkConfig = MockSink.getPlugin(outputDatasetName); + + ApplicationManager appManager = deployETL(sourceConfig, sinkConfig, + DATAPIPELINE_ARTIFACT, "testDBSourceWithMultipleTables"); + runETLOnce(appManager); + + // records should be written + DataSetManager
outputManager = getDataset(outputDatasetName); + List outputRecords = MockSink.readOutput(outputManager); + Assert.assertEquals(2, outputRecords.size()); + String userid = outputRecords.get(0).get("NAME"); + StructuredRecord row1 = "user1".equals(userid) ? outputRecords.get(0) : outputRecords.get(1); + StructuredRecord row2 = "user1".equals(userid) ? outputRecords.get(1) : outputRecords.get(0); + // Verify data + Assert.assertEquals("user1", row1.get("NAME")); + Assert.assertEquals("user2", row2.get("NAME")); + Assert.assertEquals(1, row1.get("ID").intValue()); + Assert.assertEquals(2, row2.get("ID").intValue()); + } + + @Test + public void testUserNamePasswordCombinations() throws Exception { + String importQuery = "SELECT * FROM \"my_table\" WHERE $CONDITIONS"; + String boundingQuery = "SELECT MIN(\"ID\"),MAX(\"ID\") from \"my_table\""; + String splitBy = "\"ID\""; + + ETLPlugin sinkConfig = MockSink.getPlugin("outputTable"); + + Map baseSourceProps = ImmutableMap.builder() + .put(ConnectionConfig.HOST, BASE_PROPS.get(ConnectionConfig.HOST)) + .put(ConnectionConfig.PORT, BASE_PROPS.get(ConnectionConfig.PORT)) + .put(ConnectionConfig.DATABASE, BASE_PROPS.get(ConnectionConfig.DATABASE)) + .put(ConnectionConfig.JDBC_PLUGIN_NAME, JDBC_DRIVER_NAME) + .put(AbstractDBSource.DBSourceConfig.IMPORT_QUERY, importQuery) + .put(AbstractDBSource.DBSourceConfig.BOUNDING_QUERY, boundingQuery) + .put(AbstractDBSource.DBSourceConfig.SPLIT_BY, splitBy) + .put(Constants.Reference.REFERENCE_NAME, "UserPassDBTest") + .build(); + + ApplicationId appId = NamespaceId.DEFAULT.app("dbTest"); + + // null user name, null password. Should succeed. + // as source + ETLPlugin dbConfig = new ETLPlugin(RedshiftConstants.PLUGIN_NAME, BatchSource.PLUGIN_TYPE, + baseSourceProps, null); + ETLStage table = new ETLStage("uniqueTableSink", sinkConfig); + ETLStage database = new ETLStage("databaseSource", dbConfig); + ETLBatchConfig etlConfig = ETLBatchConfig.builder() + .addStage(database) + .addStage(table) + .addConnection(database.getName(), table.getName()) + .build(); + AppRequest appRequest = new AppRequest<>(DATAPIPELINE_ARTIFACT, etlConfig); + deployApplication(appId, appRequest); + + // null user name, non-null password. Should fail. + // as source + Map noUser = new HashMap<>(baseSourceProps); + noUser.put(DBConfig.PASSWORD, "password"); + database = new ETLStage("databaseSource", new ETLPlugin(RedshiftConstants.PLUGIN_NAME, + BatchSource.PLUGIN_TYPE, noUser, null)); + etlConfig = ETLBatchConfig.builder() + .addStage(database) + .addStage(table) + .addConnection(database.getName(), table.getName()) + .build(); + assertDeploymentFailure(appId, etlConfig, DATAPIPELINE_ARTIFACT, + "Deploying DB Source with null username but non-null password should have failed."); + + // non-null username, non-null, but empty password. Should succeed. + // as source + Map emptyPassword = new HashMap<>(baseSourceProps); + emptyPassword.put(DBConfig.USER, "root"); + emptyPassword.put(DBConfig.PASSWORD, ""); + database = new ETLStage("databaseSource", new ETLPlugin(RedshiftConstants.PLUGIN_NAME, + BatchSource.PLUGIN_TYPE, emptyPassword, null)); + etlConfig = ETLBatchConfig.builder() + .addStage(database) + .addStage(table) + .addConnection(database.getName(), table.getName()) + .build(); + appRequest = new AppRequest<>(DATAPIPELINE_ARTIFACT, etlConfig); + deployApplication(appId, appRequest); + } + + @Test + public void testNonExistentDBTable() throws Exception { + // source + String importQuery = "SELECT \"ID\", \"NAME\" FROM \"dummy\" WHERE ID < 3 AND $CONDITIONS"; + String boundingQuery = "SELECT MIN(\"ID\"),MAX(\"ID\") FROM \"dummy\""; + String splitBy = "\"ID\""; + ETLPlugin sinkConfig = MockSink.getPlugin("table"); + ETLPlugin sourceBadNameConfig = new ETLPlugin( + RedshiftConstants.PLUGIN_NAME, + BatchSource.PLUGIN_TYPE, + ImmutableMap.builder() + .putAll(BASE_PROPS) + .put(AbstractDBSource.DBSourceConfig.IMPORT_QUERY, importQuery) + .put(AbstractDBSource.DBSourceConfig.BOUNDING_QUERY, boundingQuery) + .put(AbstractDBSource.DBSourceConfig.SPLIT_BY, splitBy) + .put(Constants.Reference.REFERENCE_NAME, "DBNonExistentTest") + .build(), + null); + ETLStage sink = new ETLStage("sink", sinkConfig); + ETLStage sourceBadName = new ETLStage("sourceBadName", sourceBadNameConfig); + + ETLBatchConfig etlConfig = ETLBatchConfig.builder() + .addStage(sourceBadName) + .addStage(sink) + .addConnection(sourceBadName.getName(), sink.getName()) + .build(); + ApplicationId appId = NamespaceId.DEFAULT.app("dbSourceNonExistingTest"); + assertDeployAppFailure(appId, etlConfig, DATAPIPELINE_ARTIFACT); + + // Bad connection + ETLPlugin sourceBadConnConfig = new ETLPlugin( + RedshiftConstants.PLUGIN_NAME, + BatchSource.PLUGIN_TYPE, + ImmutableMap.builder() + .put(ConnectionConfig.HOST, BASE_PROPS.get(ConnectionConfig.HOST)) + .put(ConnectionConfig.PORT, BASE_PROPS.get(ConnectionConfig.PORT)) + .put(ConnectionConfig.DATABASE, "dumDB") + .put(ConnectionConfig.USER, BASE_PROPS.get(ConnectionConfig.USER)) + .put(ConnectionConfig.PASSWORD, BASE_PROPS.get(ConnectionConfig.PASSWORD)) + .put(ConnectionConfig.JDBC_PLUGIN_NAME, JDBC_DRIVER_NAME) + .put(AbstractDBSource.DBSourceConfig.IMPORT_QUERY, importQuery) + .put(AbstractDBSource.DBSourceConfig.BOUNDING_QUERY, boundingQuery) + .put(AbstractDBSource.DBSourceConfig.SPLIT_BY, splitBy) + .put(Constants.Reference.REFERENCE_NAME, "RedshiftTest") + .build(), + null); + ETLStage sourceBadConn = new ETLStage("sourceBadConn", sourceBadConnConfig); + etlConfig = ETLBatchConfig.builder() + .addStage(sourceBadConn) + .addStage(sink) + .addConnection(sourceBadConn.getName(), sink.getName()) + .build(); + assertDeployAppFailure(appId, etlConfig, DATAPIPELINE_ARTIFACT); + } +} diff --git a/amazon-redshift-plugin/widgets/Redshift-batchsource.json b/amazon-redshift-plugin/widgets/Redshift-batchsource.json new file mode 100644 index 000000000..91e860ee9 --- /dev/null +++ b/amazon-redshift-plugin/widgets/Redshift-batchsource.json @@ -0,0 +1,240 @@ +{ + "metadata": { + "spec-version": "1.5" + }, + "display-name": "Redshift", + "configuration-groups": [ + { + "label": "Connection", + "properties": [ + { + "widget-type": "toggle", + "label": "Use connection", + "name": "useConnection", + "widget-attributes": { + "on": { + "value": "true", + "label": "YES" + }, + "off": { + "value": "false", + "label": "NO" + }, + "default": "false" + } + }, + { + "widget-type": "connection-select", + "label": "Connection", + "name": "connection", + "widget-attributes": { + "connectionType": "Redshift" + } + }, + { + "widget-type": "plugin-list", + "label": "JDBC Driver name", + "name": "jdbcPluginName", + "widget-attributes": { + "plugin-type": "jdbc" + } + }, + { + "widget-type": "textbox", + "label": "Host", + "name": "host", + "widget-attributes": { + "placeholder": "Redshift endpoint host name." + } + }, + { + "widget-type": "number", + "label": "Port", + "name": "port", + "widget-attributes": { + "default": "5439" + } + }, + { + "widget-type": "textbox", + "label": "Username", + "name": "user" + }, + { + "widget-type": "password", + "label": "Password", + "name": "password" + }, + { + "widget-type": "keyvalue", + "label": "Connection Arguments", + "name": "connectionArguments", + "widget-attributes": { + "showDelimiter": "false", + "key-placeholder": "Key", + "value-placeholder": "Value", + "kv-delimiter" : "=", + "delimiter" : ";" + } + } + ] + }, + { + "label": "Basic", + "properties": [ + { + "widget-type": "textbox", + "label": "Reference Name", + "name": "referenceName", + "widget-attributes": { + "placeholder": "Name used to identify this source for lineage. Typically, the name of the table/view." + } + }, + { + "widget-type": "textbox", + "label": "Database", + "name": "database" + }, + { + "widget-type": "connection-browser", + "widget-category": "plugin", + "widget-attributes": { + "connectionType": "Redshift", + "label": "Browse Database" + } + } + ] + }, + { + "label": "SQL Query", + "properties": [ + { + "widget-type": "textarea", + "label": "Import Query", + "name": "importQuery", + "widget-attributes": { + "rows": "4" + } + }, + { + "widget-type": "get-schema", + "widget-category": "plugin" + } + ] + }, + { + "label": "Advanced", + "properties": [ + { + "widget-type": "textarea", + "label": "Bounding Query", + "name": "boundingQuery", + "widget-attributes": { + "rows": "4" + } + }, + { + "widget-type": "textbox", + "label": "Split Column", + "name": "splitBy" + }, + { + "widget-type": "textbox", + "label": "Number of Splits", + "name": "numSplits", + "widget-attributes": { + "default": "1" + } + }, + { + "widget-type": "number", + "label": "Fetch Size", + "name": "fetchSize", + "widget-attributes": { + "default": "1000", + "minimum": "0" + } + } + ] + } + ], + "outputs": [ + { + "name": "schema", + "widget-type": "schema", + "widget-attributes": { + "schema-types": [ + "boolean", + "int", + "long", + "float", + "double", + "bytes", + "string" + ], + "schema-default-type": "string" + } + } + ], + "filters": [ + { + "name": "showConnectionProperties ", + "condition": { + "expression": "useConnection == false" + }, + "show": [ + { + "type": "property", + "name": "jdbcPluginName" + }, + { + "type": "property", + "name": "instanceType" + }, + { + "type": "property", + "name": "host" + }, + { + "type": "property", + "name": "port" + }, + { + "type": "property", + "name": "user" + }, + { + "type": "property", + "name": "password" + }, + { + "type": "property", + "name": "database" + }, + { + "type": "property", + "name": "connectionArguments" + } + ] + }, + { + "name": "showConnectionId", + "condition": { + "expression": "useConnection == true" + }, + "show": [ + { + "type": "property", + "name": "connection" + } + ] + }, + ], + "jump-config": { + "datasets": [ + { + "ref-property-name": "referenceName" + } + ] + } +} diff --git a/amazon-redshift-plugin/widgets/Redshift-connector.json b/amazon-redshift-plugin/widgets/Redshift-connector.json new file mode 100644 index 000000000..3a2af8e01 --- /dev/null +++ b/amazon-redshift-plugin/widgets/Redshift-connector.json @@ -0,0 +1,75 @@ +{ + "metadata": { + "spec-version": "1.0" + }, + "display-name": "Redshift", + "configuration-groups": [ + { + "label": "Basic", + "properties": [ + { + "widget-type": "plugin-list", + "label": "JDBC Driver name", + "name": "jdbcPluginName", + "widget-attributes": { + "plugin-type": "jdbc" + } + }, + { + "widget-type": "textbox", + "label": "Host", + "name": "host", + "widget-attributes": { + "default": "localhost" + } + }, + { + "widget-type": "number", + "label": "Port", + "name": "port", + "widget-attributes": { + "default": "5439" + } + }, + { + "widget-type": "textbox", + "label": "Database", + "name": "database" + } + ] + }, + { + "label": "Credentials", + "properties": [ + { + "widget-type": "textbox", + "label": "Username", + "name": "user" + }, + { + "widget-type": "password", + "label": "Password", + "name": "password" + } + ] + }, + { + "label": "Advanced", + "properties": [ + { + "widget-type": "keyvalue", + "label": "Connection Arguments", + "name": "connectionArguments", + "widget-attributes": { + "showDelimiter": "false", + "key-placeholder": "Key", + "value-placeholder": "Value", + "kv-delimiter": "=", + "delimiter": ";" + } + } + ] + } + ], + "outputs": [] +} diff --git a/pom.xml b/pom.xml index a6b40960c..35a2004c5 100644 --- a/pom.xml +++ b/pom.xml @@ -44,6 +44,7 @@ cloudsql-postgresql-plugin teradata-plugin generic-db-argument-setter + amazon-redshift-plugin