From cd5f6422d39d25495258e95f84d31e89f973e447 Mon Sep 17 00:00:00 2001 From: Chip Maurer Date: Fri, 5 Mar 2021 11:30:21 -0500 Subject: [PATCH 1/8] Initial drop of pravega connector Signed-off-by: Chip Maurer --- .gitignore | 28 + build.gradle | 112 +++ gradle/wrapper/gradle-wrapper.jar | Bin 0 -> 58910 bytes gradle/wrapper/gradle-wrapper.properties | 5 + gradlew | 185 +++++ gradlew.bat | 104 +++ settings.gradle | 5 + src/checkstyle/presto-checks.xml | 219 +++++ .../ClassLoaderSafeRecordSetProvider.java | 47 ++ .../presto/pravega/DelimRecordValue.java | 115 +++ .../presto/pravega/EventStreamIterator.java | 103 +++ .../facebook/presto/pravega/ObjectType.java | 20 + .../presto/pravega/PravegaColumnHandle.java | 237 ++++++ .../presto/pravega/PravegaConnector.java | 99 +++ .../pravega/PravegaConnectorConfig.java | 105 +++ .../pravega/PravegaConnectorFactory.java | 105 +++ .../presto/pravega/PravegaConnectorId.java | 53 ++ .../pravega/PravegaConnectorModule.java | 75 ++ .../presto/pravega/PravegaErrorCode.java | 44 + .../presto/pravega/PravegaHandleResolver.java | 89 ++ .../PravegaInternalFieldDescription.java | 121 +++ .../presto/pravega/PravegaKVRecordSet.java | 175 ++++ .../presto/pravega/PravegaKVTable.java | 59 ++ .../presto/pravega/PravegaMetadata.java | 255 ++++++ .../presto/pravega/PravegaObjectSchema.java | 58 ++ .../presto/pravega/PravegaPlugin.java | 53 ++ .../presto/pravega/PravegaProperties.java | 93 +++ .../presto/pravega/PravegaRecordCursor.java | 175 ++++ .../presto/pravega/PravegaRecordSet.java | 103 +++ .../pravega/PravegaRecordSetProvider.java | 233 ++++++ .../presto/pravega/PravegaRecordValue.java | 33 + .../presto/pravega/PravegaSegmentManager.java | 184 ++++ .../facebook/presto/pravega/PravegaSplit.java | 126 +++ .../presto/pravega/PravegaSplitManager.java | 289 +++++++ .../pravega/PravegaStreamDescription.java | 105 +++ .../PravegaStreamFieldDescription.java | 160 ++++ .../pravega/PravegaStreamFieldGroup.java | 82 ++ .../PravegaTableDescriptionSupplier.java | 784 ++++++++++++++++++ .../presto/pravega/PravegaTableHandle.java | 173 ++++ .../pravega/PravegaTableLayoutHandle.java | 48 ++ .../presto/pravega/PravegaTableName.java | 81 ++ .../pravega/PravegaTransactionHandle.java | 22 + .../presto/pravega/ProtobufCommon.java | 117 +++ .../facebook/presto/pravega/ReaderArgs.java | 76 ++ .../facebook/presto/pravega/ReaderType.java | 23 + .../presto/pravega/SegmentEventIterator.java | 50 ++ .../presto/pravega/SegmentRangeIterator.java | 102 +++ .../presto/pravega/StreamCutRange.java | 63 ++ .../presto/pravega/StreamCutSupplier.java | 102 +++ .../presto/pravega/TypedRecordValue.java | 104 +++ .../presto/pravega/decoder/AvroEvent.java | 38 + .../pravega/decoder/AvroRowDecoder.java | 52 ++ .../pravega/decoder/AvroSerializer.java | 103 +++ .../presto/pravega/decoder/BytesEvent.java | 38 + .../pravega/decoder/BytesEventDecoder.java | 37 + .../presto/pravega/decoder/CsvRowDecoder.java | 28 + .../presto/pravega/decoder/CsvSerializer.java | 44 + .../pravega/decoder/DecodableEvent.java | 62 ++ .../presto/pravega/decoder/EventDecoder.java | 21 + .../presto/pravega/decoder/JsonEvent.java | 38 + .../pravega/decoder/JsonRowDecoder.java | 35 + .../pravega/decoder/JsonSerializer.java | 88 ++ .../presto/pravega/decoder/KVSerializer.java | 24 + .../decoder/MultiSourceRowDecoder.java | 60 ++ .../decoder/ProtobufColumnDecoder.java | 172 ++++ .../presto/pravega/decoder/ProtobufEvent.java | 38 + .../pravega/decoder/ProtobufRowDecoder.java | 51 ++ .../pravega/decoder/ProtobufSerializer.java | 94 +++ .../pravega/util/ByteBufferInputStream.java | 46 + .../presto/pravega/util/PravegaNameUtils.java | 108 +++ .../pravega/util/PravegaSchemaUtils.java | 103 +++ .../util/PravegaSerializationUtils.java | 57 ++ .../services/com.facebook.presto.spi.Plugin | 1 + src/modernizer/violations.xml | 32 + .../presto/pravega/TestPravegaPlugin.java | 43 + 75 files changed, 7212 insertions(+) create mode 100644 .gitignore create mode 100644 build.gradle create mode 100644 gradle/wrapper/gradle-wrapper.jar create mode 100644 gradle/wrapper/gradle-wrapper.properties create mode 100755 gradlew create mode 100644 gradlew.bat create mode 100644 settings.gradle create mode 100644 src/checkstyle/presto-checks.xml create mode 100644 src/main/java/com/facebook/presto/pravega/ClassLoaderSafeRecordSetProvider.java create mode 100644 src/main/java/com/facebook/presto/pravega/DelimRecordValue.java create mode 100644 src/main/java/com/facebook/presto/pravega/EventStreamIterator.java create mode 100644 src/main/java/com/facebook/presto/pravega/ObjectType.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaColumnHandle.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaConnector.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaConnectorConfig.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaConnectorFactory.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaConnectorId.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaConnectorModule.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaErrorCode.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaHandleResolver.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaInternalFieldDescription.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaKVRecordSet.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaKVTable.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaMetadata.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaObjectSchema.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaPlugin.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaProperties.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaRecordCursor.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaRecordSet.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaRecordSetProvider.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaRecordValue.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaSegmentManager.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaSplit.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaSplitManager.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaStreamDescription.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaStreamFieldDescription.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaStreamFieldGroup.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaTableDescriptionSupplier.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaTableHandle.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaTableLayoutHandle.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaTableName.java create mode 100644 src/main/java/com/facebook/presto/pravega/PravegaTransactionHandle.java create mode 100644 src/main/java/com/facebook/presto/pravega/ProtobufCommon.java create mode 100644 src/main/java/com/facebook/presto/pravega/ReaderArgs.java create mode 100644 src/main/java/com/facebook/presto/pravega/ReaderType.java create mode 100644 src/main/java/com/facebook/presto/pravega/SegmentEventIterator.java create mode 100644 src/main/java/com/facebook/presto/pravega/SegmentRangeIterator.java create mode 100644 src/main/java/com/facebook/presto/pravega/StreamCutRange.java create mode 100644 src/main/java/com/facebook/presto/pravega/StreamCutSupplier.java create mode 100644 src/main/java/com/facebook/presto/pravega/TypedRecordValue.java create mode 100644 src/main/java/com/facebook/presto/pravega/decoder/AvroEvent.java create mode 100644 src/main/java/com/facebook/presto/pravega/decoder/AvroRowDecoder.java create mode 100644 src/main/java/com/facebook/presto/pravega/decoder/AvroSerializer.java create mode 100644 src/main/java/com/facebook/presto/pravega/decoder/BytesEvent.java create mode 100644 src/main/java/com/facebook/presto/pravega/decoder/BytesEventDecoder.java create mode 100644 src/main/java/com/facebook/presto/pravega/decoder/CsvRowDecoder.java create mode 100644 src/main/java/com/facebook/presto/pravega/decoder/CsvSerializer.java create mode 100644 src/main/java/com/facebook/presto/pravega/decoder/DecodableEvent.java create mode 100644 src/main/java/com/facebook/presto/pravega/decoder/EventDecoder.java create mode 100644 src/main/java/com/facebook/presto/pravega/decoder/JsonEvent.java create mode 100644 src/main/java/com/facebook/presto/pravega/decoder/JsonRowDecoder.java create mode 100644 src/main/java/com/facebook/presto/pravega/decoder/JsonSerializer.java create mode 100644 src/main/java/com/facebook/presto/pravega/decoder/KVSerializer.java create mode 100644 src/main/java/com/facebook/presto/pravega/decoder/MultiSourceRowDecoder.java create mode 100644 src/main/java/com/facebook/presto/pravega/decoder/ProtobufColumnDecoder.java create mode 100644 src/main/java/com/facebook/presto/pravega/decoder/ProtobufEvent.java create mode 100644 src/main/java/com/facebook/presto/pravega/decoder/ProtobufRowDecoder.java create mode 100644 src/main/java/com/facebook/presto/pravega/decoder/ProtobufSerializer.java create mode 100644 src/main/java/com/facebook/presto/pravega/util/ByteBufferInputStream.java create mode 100644 src/main/java/com/facebook/presto/pravega/util/PravegaNameUtils.java create mode 100644 src/main/java/com/facebook/presto/pravega/util/PravegaSchemaUtils.java create mode 100644 src/main/java/com/facebook/presto/pravega/util/PravegaSerializationUtils.java create mode 100644 src/main/resources/META-INF/services/com.facebook.presto.spi.Plugin create mode 100644 src/modernizer/violations.xml create mode 100644 src/test/java/com/facebook/presto/pravega/TestPravegaPlugin.java diff --git a/.gitignore b/.gitignore new file mode 100644 index 0000000..8d65bda --- /dev/null +++ b/.gitignore @@ -0,0 +1,28 @@ +*.iml +*.ipr +*.iws +.gradle +build/ +/var +/*/var/ +/presto-product-tests/**/var/ +test-output/ +test-reports/ +out/ +/atlassian-ide-plugin.xml +.idea +.DS_Store +.classpath +.settings +.project +temp-testng-customsuite.xml +test-output +.externalToolBuilders +*~ +benchmark_outputs +*.pyc +*.class +.checkstyle +.editorconfig +node_modules + diff --git a/build.gradle b/build.gradle new file mode 100644 index 0000000..5e1092d --- /dev/null +++ b/build.gradle @@ -0,0 +1,112 @@ +/* + * This file was generated by the Gradle 'init' task. + */ + +plugins { + id 'java' + id 'distribution' +} + +repositories { + mavenLocal() + maven { + url = uri('https://oss.jfrog.org/jfrog-dependencies') + } + + maven { + url = uri('https://jitpack.io') + } + + maven { + url = uri('https://repo.maven.apache.org/maven2') + } +} + +configurations { + prestoDB + prestoSQL +} + +dependencies { + compile 'com.facebook.airlift:bootstrap:0.191' + compile 'com.facebook.airlift:json:0.191' + compile 'com.facebook.airlift:log:0.191' + compile 'com.facebook.airlift:configuration:0.191' + compile 'com.google.guava:guava:26.0-jre' + compile 'com.google.inject:guice:4.2.0' + compile 'javax.validation:validation-api:1.1.0.Final' + compile 'javax.inject:javax.inject:1' + compile 'com.fasterxml.jackson.core:jackson-databind:2.10.0' + compile 'io.pravega:pravega-client:0.9.0-2705.09f82eb-SNAPSHOT' + compile 'io.pravega:pravega-common:0.9.0-2705.09f82eb-SNAPSHOT' + compile 'io.pravega:pravega-shared-protocol:0.9.0-2705.09f82eb-SNAPSHOT' + compile 'com.google.protobuf:protobuf-java:3.11.4' + compile 'com.github.everit-org.json-schema:org.everit.json.schema:1.12.1' + compile 'org.apache.avro:avro:1.8.1' + compile 'org.apache.commons:commons-lang3:3.7' + compile 'io.pravega:schemaregistry-contract:0.2.0-50.f1b6734-SNAPSHOT' + compile 'io.pravega:schemaregistry-common:0.2.0-50.f1b6734-SNAPSHOT' + compile 'io.pravega:schemaregistry-client:0.2.0-50.f1b6734-SNAPSHOT' + compile 'io.pravega:schemaregistry-serializers:0.2.0-50.f1b6734-SNAPSHOT' + compile 'io.pravega:schemaregistry-serializers-shared:0.2.0-50.f1b6734-SNAPSHOT' + compile 'io.pravega:schemaregistry-serializers-json:0.2.0-50.f1b6734-SNAPSHOT' + + compile 'com.facebook.presto:presto-main:0.247' + compile 'com.facebook.presto:presto-record-decoder:0.247' + compile 'com.facebook.presto:presto-spi:0.247' + compile 'com.facebook.presto:presto-common:0.247' + + runtimeOnly 'io.airlift:joda-to-java-time-bridge:3' + runtimeOnly 'com.facebook.airlift:log-manager:0.191' + runtimeOnly 'org.apache.zookeeper:zookeeper:3.5.7' + runtimeOnly 'com.101tec:zkclient:0.10' + + testImplementation 'com.facebook.presto:presto-tests:0.247' + testImplementation 'org.testng:testng:7.3.0' + testImplementation 'com.facebook.airlift:testing:0.191' + + compileOnly 'io.airlift:slice:0.38' + compileOnly 'io.airlift:units:1.3' + compileOnly 'com.fasterxml.jackson.core:jackson-annotations:2.10.0' + compileOnly 'org.openjdk.jol:jol-core:0.2' +} + +group = 'com.facebook.presto' +version = '0.1.0' +description = 'Pravega SQL :: Pravega PrestoDB Connector' +sourceCompatibility = '1.8' + +task getHomeDir { + doLast { + println gradle.gradleHomeDir + } +} + +jar { + from project.configurations.runtime, configurations.prestoDB +} + +task prestoSQLJar(type: Jar) { + from project.configurations.runtime, configurations.prestoSQL +} + +plugins.withType(DistributionPlugin) { + distTar { + compression = Compression.GZIP + extension = 'tar.gz' + } +} + +distributions { + main { + contents { + distributionBaseName = 'pravega' + from jar + from(project.configurations.runtime) + } + } +} + +test { + useTestNG() +} diff --git a/gradle/wrapper/gradle-wrapper.jar b/gradle/wrapper/gradle-wrapper.jar new file mode 100644 index 0000000000000000000000000000000000000000..62d4c053550b91381bbd28b1afc82d634bf73a8a GIT binary patch literal 58910 zcma&ObC74zk}X`WF59+k+qTVL*+!RbS9RI8Z5v&-ZFK4Nn|tqzcjwK__x+Iv5xL`> zj94dg?X`0sMHx^qXds{;KY)OMg#H>35XgTVfq6#vc9ww|9) z@UMfwUqk)B9p!}NrNqTlRO#i!ALOPcWo78-=iy}NsAr~T8T0X0%G{DhX~u-yEwc29WQ4D zuv2j{a&j?qB4wgCu`zOXj!~YpTNFg)TWoV>DhYlR^Gp^rkOEluvxkGLB?!{fD!T@( z%3cy>OkhbIKz*R%uoKqrg1%A?)uTZD&~ssOCUBlvZhx7XHQ4b7@`&sPdT475?*zWy z>xq*iK=5G&N6!HiZaD{NSNhWL;+>Quw_#ZqZbyglna!Fqn3N!$L`=;TFPrhodD-Q` z1l*=DP2gKJP@)cwI@-M}?M$$$%u~=vkeC%>cwR$~?y6cXx-M{=wdT4|3X(@)a|KkZ z`w$6CNS@5gWS7s7P86L<=vg$Mxv$?)vMj3`o*7W4U~*Nden}wz=y+QtuMmZ{(Ir1D zGp)ZsNiy{mS}Au5;(fYf93rs^xvi(H;|H8ECYdC`CiC&G`zw?@)#DjMc7j~daL_A$ z7e3nF2$TKlTi=mOftyFBt8*Xju-OY@2k@f3YBM)-v8+5_o}M?7pxlNn)C0Mcd@87?+AA4{Ti2ptnYYKGp`^FhcJLlT%RwP4k$ad!ho}-^vW;s{6hnjD0*c39k zrm@PkI8_p}mnT&5I@=O1^m?g}PN^8O8rB`;t`6H+?Su0IR?;8txBqwK1Au8O3BZAX zNdJB{bpQWR@J|e=Z>XSXV1DB{uhr3pGf_tb)(cAkp)fS7*Qv))&Vkbb+cvG!j}ukd zxt*C8&RN}5ck{jkw0=Q7ldUp0FQ&Pb_$M7a@^nf`8F%$ftu^jEz36d#^M8Ia{VaTy z5(h$I)*l3i!VpPMW+XGgzL~fcN?{~1QWu9!Gu0jOWWE zNW%&&by0DbXL&^)r-A*7R@;T$P}@3eOj#gqJ!uvTqBL5bupU91UK#d|IdxBUZAeh1 z>rAI#*Y4jv>uhOh7`S@mnsl0g@1C;k$Z%!d*n8#_$)l}-1&z2kr@M+xWoKR z!KySy-7h&Bf}02%JeXmQGjO3ntu={K$jy$rFwfSV8!zqAL_*&e2|CJ06`4&0+ceI026REfNT>JzAdwmIlKLEr2? zaZ#d*XFUN*gpzOxq)cysr&#6zNdDDPH% zd8_>3B}uA7;bP4fKVdd~Og@}dW#74ceETOE- zlZgQqQfEc?-5ly(Z5`L_CCM!&Uxk5#wgo=OLs-kFHFG*cTZ)$VE?c_gQUW&*!2@W2 z7Lq&_Kf88OCo?BHCtwe*&fu&8PQ(R5&lnYo8%+U73U)Ec2&|A)Y~m7(^bh299REPe zn#gyaJ4%o4>diN3z%P5&_aFUmlKytY$t21WGwx;3?UC}vlxi-vdEQgsKQ;=#sJ#ll zZeytjOad$kyON4XxC}frS|Ybh`Yq!<(IrlOXP3*q86ImyV*mJyBn$m~?#xp;EplcM z+6sez%+K}Xj3$YN6{}VL;BZ7Fi|iJj-ywlR+AP8lq~mnt5p_%VmN{Sq$L^z!otu_u znVCl@FgcVXo510e@5(wnko%Pv+^r^)GRh;>#Z(|#cLnu_Y$#_xG&nvuT+~gzJsoSi zBvX`|IS~xaold!`P!h(v|=>!5gk)Q+!0R1Ge7!WpRP{*Ajz$oGG$_?Ajvz6F0X?809o`L8prsJ*+LjlGfSziO;+ zv>fyRBVx#oC0jGK8$%$>Z;0+dfn8x;kHFQ?Rpi7(Rc{Uq{63Kgs{IwLV>pDK7yX-2 zls;?`h!I9YQVVbAj7Ok1%Y+F?CJa-Jl>1x#UVL(lpzBBH4(6v0^4 z3Tf`INjml5`F_kZc5M#^J|f%7Hgxg3#o}Zwx%4l9yYG!WaYUA>+dqpRE3nw#YXIX%= ziH3iYO~jr0nP5xp*VIa#-aa;H&%>{mfAPPlh5Fc!N7^{!z$;p-p38aW{gGx z)dFS62;V;%%fKp&i@+5x=Cn7Q>H`NofJGXmNeh{sOL+Nk>bQJJBw3K*H_$}%*xJM=Kh;s#$@RBR z|75|g85da@#qT=pD777m$wI!Q8SC4Yw3(PVU53bzzGq$IdGQoFb-c_(iA_~qD|eAy z@J+2!tc{|!8fF;%6rY9`Q!Kr>MFwEH%TY0y>Q(D}xGVJM{J{aGN0drG&|1xO!Ttdw z-1^gQ&y~KS5SeslMmoA$Wv$ly={f}f9<{Gm!8ycp*D9m*5Ef{ymIq!MU01*)#J1_! zM_i4{LYButqlQ>Q#o{~W!E_#(S=hR}kIrea_67Z5{W>8PD>g$f;dTvlD=X@T$8D0;BWkle@{VTd&D5^)U>(>g(jFt4lRV6A2(Te->ooI{nk-bZ(gwgh zaH4GT^wXPBq^Gcu%xW#S#p_&x)pNla5%S5;*OG_T^PhIIw1gXP&u5c;{^S(AC*+$> z)GuVq(FT@zq9;i{*9lEsNJZ)??BbSc5vF+Kdh-kL@`(`l5tB4P!9Okin2!-T?}(w% zEpbEU67|lU#@>DppToestmu8Ce=gz=e#V+o)v)#e=N`{$MI5P0O)_fHt1@aIC_QCv=FO`Qf=Ga%^_NhqGI)xtN*^1n{ z&vgl|TrKZ3Vam@wE0p{c3xCCAl+RqFEse@r*a<3}wmJl-hoJoN<|O2zcvMRl<#BtZ z#}-bPCv&OTw`GMp&n4tutf|er`@#d~7X+);##YFSJ)BitGALu}-N*DJdCzs(cQ?I- z6u(WAKH^NUCcOtpt5QTsQRJ$}jN28ZsYx+4CrJUQ%egH zo#tMoywhR*oeIkS%}%WUAIbM`D)R6Ya&@sZvvUEM7`fR0Ga03*=qaEGq4G7-+30Ck zRkje{6A{`ebq?2BTFFYnMM$xcQbz0nEGe!s%}O)m={`075R0N9KTZ>vbv2^eml>@}722%!r#6Wto}?vNst? zs`IasBtcROZG9+%rYaZe^=5y3chDzBf>;|5sP0!sP(t^= z^~go8msT@|rp8LJ8km?4l?Hb%o10h7(ixqV65~5Y>n_zG3AMqM3UxUNj6K-FUgMT7 z*Dy2Y8Ws+%`Z*~m9P zCWQ8L^kA2$rf-S@qHow$J86t)hoU#XZ2YK~9GXVR|*`f6`0&8j|ss_Ai-x=_;Df^*&=bW$1nc{Gplm zF}VF`w)`5A;W@KM`@<9Bw_7~?_@b{Z`n_A6c1AG#h#>Z$K>gX6reEZ*bZRjCup|0# zQ{XAb`n^}2cIwLTN%5Ix`PB*H^(|5S{j?BwItu+MS`1)VW=TnUtt6{3J!WR`4b`LW z?AD#ZmoyYpL=903q3LSM=&5eNP^dwTDRD~iP=}FXgZ@2WqfdyPYl$9do?wX{RU*$S zgQ{OqXK-Yuf4+}x6P#A*la&^G2c2TC;aNNZEYuB(f25|5eYi|rd$;i0qk7^3Ri8of ziP~PVT_|4$n!~F-B1_Et<0OJZ*e+MN;5FFH`iec(lHR+O%O%_RQhvbk-NBQ+$)w{D+dlA0jxI;z|P zEKW`!X)${xzi}Ww5G&@g0akBb_F`ziv$u^hs0W&FXuz=Ap>SUMw9=M?X$`lgPRq11 zqq+n44qL;pgGO+*DEc+Euv*j(#%;>p)yqdl`dT+Og zZH?FXXt`<0XL2@PWYp|7DWzFqxLK)yDXae&3P*#+f+E{I&h=$UPj;ey9b`H?qe*Oj zV|-qgI~v%&oh7rzICXfZmg$8$B|zkjliQ=e4jFgYCLR%yi!9gc7>N z&5G#KG&Hr+UEfB;M(M>$Eh}P$)<_IqC_WKOhO4(cY@Gn4XF(#aENkp&D{sMQgrhDT zXClOHrr9|POHqlmm+*L6CK=OENXbZ+kb}t>oRHE2xVW<;VKR@ykYq04LM9L-b;eo& zl!QQo!Sw{_$-qosixZJWhciN>Gbe8|vEVV2l)`#5vKyrXc6E`zmH(76nGRdL)pqLb@j<&&b!qJRLf>d`rdz}^ZSm7E;+XUJ ziy;xY&>LM?MA^v0Fu8{7hvh_ynOls6CI;kQkS2g^OZr70A}PU;i^~b_hUYN1*j-DD zn$lHQG9(lh&sDii)ip*{;Sb_-Anluh`=l~qhqbI+;=ZzpFrRp&T+UICO!OoqX@Xr_ z32iJ`xSpx=lDDB_IG}k+GTYG@K8{rhTS)aoN8D~Xfe?ul&;jv^E;w$nhu-ICs&Q)% zZ=~kPNZP0-A$pB8)!`TEqE`tY3Mx^`%O`?EDiWsZpoP`e-iQ#E>fIyUx8XN0L z@S-NQwc;0HjSZKWDL}Au_Zkbh!juuB&mGL0=nO5)tUd_4scpPy&O7SNS^aRxUy0^< zX}j*jPrLP4Pa0|PL+nrbd4G;YCxCK-=G7TG?dby~``AIHwxqFu^OJhyIUJkO0O<>_ zcpvg5Fk$Wpj}YE3;GxRK67P_Z@1V#+pu>pRj0!mFf(m_WR3w3*oQy$s39~U7Cb}p(N&8SEwt+)@%o-kW9Ck=^?tvC2$b9% ze9(Jn+H`;uAJE|;$Flha?!*lJ0@lKfZM>B|c)3lIAHb;5OEOT(2453m!LgH2AX=jK zQ93An1-#l@I@mwB#pLc;M7=u6V5IgLl>E%gvE|}Hvd4-bE1>gs(P^C}gTv*&t>W#+ zASLRX$y^DD3Jrht zwyt`yuA1j(TcP*0p*Xkv>gh+YTLrcN_HuaRMso~0AJg`^nL#52dGBzY+_7i)Ud#X) zVwg;6$WV20U2uyKt8<)jN#^1>PLg`I`@Mmut*Zy!c!zshSA!e^tWVoKJD%jN&ml#{ z@}B$j=U5J_#rc%T7(DGKF+WwIblEZ;Vq;CsG~OKxhWYGJx#g7fxb-_ya*D0=_Ys#f zhXktl=Vnw#Z_neW>Xe#EXT(4sT^3p6srKby4Ma5LLfh6XrHGFGgM;5Z}jv-T!f~=jT&n>Rk z4U0RT-#2fsYCQhwtW&wNp6T(im4dq>363H^ivz#>Sj;TEKY<)dOQU=g=XsLZhnR>e zd}@p1B;hMsL~QH2Wq>9Zb; zK`0`09fzuYg9MLJe~cdMS6oxoAD{kW3sFAqDxvFM#{GpP^NU@9$d5;w^WgLYknCTN z0)N425mjsJTI@#2kG-kB!({*+S(WZ-{SckG5^OiyP%(6DpRsx60$H8M$V65a_>oME z^T~>oG7r!ew>Y)&^MOBrgc-3PezgTZ2xIhXv%ExMFgSf5dQbD=Kj*!J4k^Xx!Z>AW ziZfvqJvtm|EXYsD%A|;>m1Md}j5f2>kt*gngL=enh<>#5iud0dS1P%u2o+>VQ{U%(nQ_WTySY(s#~~> zrTsvp{lTSup_7*Xq@qgjY@1#bisPCRMMHnOL48qi*jQ0xg~TSW%KMG9zN1(tjXix()2$N}}K$AJ@GUth+AyIhH6Aeh7qDgt#t*`iF5#A&g4+ zWr0$h9Zx6&Uo2!Ztcok($F>4NA<`dS&Js%L+67FT@WmI)z#fF~S75TUut%V($oUHw z$IJsL0X$KfGPZYjB9jaj-LaoDD$OMY4QxuQ&vOGo?-*9@O!Nj>QBSA6n$Lx|^ zky)4+sy{#6)FRqRt6nM9j2Lzba!U;aL%ZcG&ki1=3gFx6(&A3J-oo|S2_`*w9zT)W z4MBOVCp}?4nY)1))SOX#6Zu0fQQ7V{RJq{H)S#;sElY)S)lXTVyUXTepu4N)n85Xo zIpWPT&rgnw$D2Fsut#Xf-hO&6uA0n~a;a3!=_!Tq^TdGE&<*c?1b|PovU}3tfiIUu z){4W|@PY}zJOXkGviCw^x27%K_Fm9GuKVpd{P2>NJlnk^I|h2XW0IO~LTMj>2<;S* zZh2uRNSdJM$U$@=`zz}%;ucRx{aKVxxF7?0hdKh6&GxO6f`l2kFncS3xu0Ly{ew0& zeEP*#lk-8-B$LD(5yj>YFJ{yf5zb41PlW7S{D9zC4Aa4nVdkDNH{UsFJp)q-`9OYt zbOKkigbmm5hF?tttn;S4g^142AF^`kiLUC?e7=*JH%Qe>uW=dB24NQa`;lm5yL>Dyh@HbHy-f%6Vz^ zh&MgwYsh(z#_fhhqY$3*f>Ha}*^cU-r4uTHaT?)~LUj5``FcS46oyoI5F3ZRizVD% zPFY(_S&5GN8$Nl2=+YO6j4d|M6O7CmUyS&}m4LSn6}J`$M0ZzT&Ome)ZbJDFvM&}A zZdhDn(*viM-JHf84$!I(8eakl#zRjJH4qfw8=60 z11Ely^FyXjVvtv48-Fae7p=adlt9_F^j5#ZDf7)n!#j?{W?@j$Pi=k`>Ii>XxrJ?$ z^bhh|X6qC8d{NS4rX5P!%jXy=>(P+r9?W(2)|(=a^s^l~x*^$Enw$~u%WRuRHHFan{X|S;FD(Mr z@r@h^@Bs#C3G;~IJMrERd+D!o?HmFX&#i|~q(7QR3f8QDip?ms6|GV_$86aDb|5pc?_-jo6vmWqYi{P#?{m_AesA4xX zi&ki&lh0yvf*Yw~@jt|r-=zpj!bw<6zI3Aa^Wq{|*WEC}I=O!Re!l~&8|Vu<$yZ1p zs-SlwJD8K!$(WWyhZ+sOqa8cciwvyh%zd`r$u;;fsHn!hub0VU)bUv^QH?x30#;tH zTc_VbZj|prj7)d%ORU;Vs{#ERb>K8>GOLSImnF7JhR|g$7FQTU{(a7RHQ*ii-{U3X z^7+vM0R$8b3k1aSU&kxvVPfOz3~)0O2iTYinV9_5{pF18j4b{o`=@AZIOAwwedB2@ ztXI1F04mg{<>a-gdFoRjq$6#FaevDn$^06L)k%wYq03&ysdXE+LL1#w$rRS1Y;BoS zH1x}{ms>LHWmdtP(ydD!aRdAa(d@csEo z0EF9L>%tppp`CZ2)jVb8AuoYyu;d^wfje6^n6`A?6$&%$p>HcE_De-Zh)%3o5)LDa zskQ}%o7?bg$xUj|n8gN9YB)z!N&-K&!_hVQ?#SFj+MpQA4@4oq!UQ$Vm3B`W_Pq3J z=ngFP4h_y=`Iar<`EESF9){%YZVyJqLPGq07TP7&fSDmnYs2NZQKiR%>){imTBJth zPHr@p>8b+N@~%43rSeNuOz;rgEm?14hNtI|KC6Xz1d?|2J`QS#`OW7gTF_;TPPxu@ z)9J9>3Lx*bc>Ielg|F3cou$O0+<b34_*ZJhpS&$8DP>s%47a)4ZLw`|>s=P_J4u z?I_%AvR_z8of@UYWJV?~c4Yb|A!9n!LEUE6{sn@9+D=0w_-`szJ_T++x3MN$v-)0d zy`?1QG}C^KiNlnJBRZBLr4G~15V3$QqC%1G5b#CEB0VTr#z?Ug%Jyv@a`QqAYUV~^ zw)d|%0g&kl{j#FMdf$cn(~L@8s~6eQ)6{`ik(RI(o9s0g30Li{4YoxcVoYd+LpeLz zai?~r)UcbYr@lv*Z>E%BsvTNd`Sc?}*}>mzJ|cr0Y(6rA7H_6&t>F{{mJ^xovc2a@ zFGGDUcGgI-z6H#o@Gj29C=Uy{wv zQHY2`HZu8+sBQK*_~I-_>fOTKEAQ8_Q~YE$c?cSCxI;vs-JGO`RS464Ft06rpjn+a zqRS0Y3oN(9HCP@{J4mOWqIyD8PirA!pgU^Ne{LHBG;S*bZpx3|JyQDGO&(;Im8!ed zNdpE&?3U?E@O~>`@B;oY>#?gXEDl3pE@J30R1;?QNNxZ?YePc)3=NS>!STCrXu*lM z69WkLB_RBwb1^-zEm*tkcHz3H;?v z;q+x0Jg$|?5;e1-kbJnuT+^$bWnYc~1qnyVTKh*cvM+8yJT-HBs1X@cD;L$su65;i z2c1MxyL~NuZ9+)hF=^-#;dS#lFy^Idcb>AEDXu1!G4Kd8YPy~0lZz$2gbv?su}Zn} zGtIbeYz3X8OA9{sT(aleold_?UEV{hWRl(@)NH6GFH@$<8hUt=dNte%e#Jc>7u9xi zuqv!CRE@!fmZZ}3&@$D>p0z=*dfQ_=IE4bG0hLmT@OP>x$e`qaqf_=#baJ8XPtOpWi%$ep1Y)o2(sR=v)M zt(z*pGS$Z#j_xq_lnCr+x9fwiT?h{NEn#iK(o)G&Xw-#DK?=Ms6T;%&EE${Gq_%99 z6(;P~jPKq9llc+cmI(MKQ6*7PcL)BmoI}MYFO)b3-{j>9FhNdXLR<^mnMP`I7z0v` zj3wxcXAqi4Z0kpeSf>?V_+D}NULgU$DBvZ^=0G8Bypd7P2>;u`yW9`%4~&tzNJpgp zqB+iLIM~IkB;ts!)exn643mAJ8-WlgFE%Rpq!UMYtB?$5QAMm)%PT0$$2{>Yu7&U@ zh}gD^Qdgu){y3ANdB5{75P;lRxSJPSpQPMJOiwmpMdT|?=q;&$aTt|dl~kvS z+*i;6cEQJ1V`R4Fd>-Uzsc=DPQ7A7#VPCIf!R!KK%LM&G%MoZ0{-8&99H!|UW$Ejv zhDLX3ESS6CgWTm#1ZeS2HJb`=UM^gsQ84dQpX(ESWSkjn>O zVxg%`@mh(X9&&wN$lDIc*@>rf?C0AD_mge3f2KkT6kGySOhXqZjtA?5z`vKl_{(5g z&%Y~9p?_DL{+q@siT~*3Q*$nWXQfNN;%s_eHP_A;O`N`SaoB z6xYR;z_;HQ2xAa9xKgx~2f2xEKiEDpGPH1d@||v#f#_Ty6_gY>^oZ#xac?pc-F`@ z*}8sPV@xiz?efDMcmmezYVw~qw=vT;G1xh+xRVBkmN66!u(mRG3G6P#v|;w@anEh7 zCf94arw%YB*=&3=RTqX?z4mID$W*^+&d6qI*LA-yGme;F9+wTsNXNaX~zl2+qIK&D-aeN4lr0+yP;W>|Dh?ms_ogT{DT+ ztXFy*R7j4IX;w@@R9Oct5k2M%&j=c_rWvoul+` z<18FH5D@i$P38W9VU2(EnEvlJ(SHCqTNBa)brkIjGP|jCnK&Qi%97tikU}Y#3L?s! z2ujL%YiHO-#!|g5066V01hgT#>fzls7P>+%D~ogOT&!Whb4iF=CnCto82Yb#b`YoVsj zS2q^W0Rj!RrM@=_GuPQy5*_X@Zmu`TKSbqEOP@;Ga&Rrr>#H@L41@ZX)LAkbo{G8+ z;!5EH6vv-ip0`tLB)xUuOX(*YEDSWf?PIxXe`+_B8=KH#HFCfthu}QJylPMTNmoV; zC63g%?57(&osaH^sxCyI-+gwVB|Xs2TOf=mgUAq?V~N_5!4A=b{AXbDae+yABuuu3B_XSa4~c z1s-OW>!cIkjwJf4ZhvT|*IKaRTU)WAK=G|H#B5#NB9<{*kt?7`+G*-^<)7$Iup@Um z7u*ABkG3F*Foj)W9-I&@BrN8(#$7Hdi`BU#SR1Uz4rh&=Ey!b76Qo?RqBJ!U+rh(1 znw@xw5$)4D8OWtB_^pJO*d~2Mb-f~>I!U#*=Eh*xa6$LX?4Evp4%;ENQR!mF4`f7F zpG!NX=qnCwE8@NAbQV`*?!v0;NJ(| zBip8}VgFVsXFqslXUV>_Z>1gmD(7p#=WACXaB|Y`=Kxa=p@_ALsL&yAJ`*QW^`2@% zW7~Yp(Q@ihmkf{vMF?kqkY%SwG^t&CtfRWZ{syK@W$#DzegcQ1>~r7foTw3^V1)f2Tq_5f$igmfch;8 zT-<)?RKcCdQh6x^mMEOS;4IpQ@F2q-4IC4%*dU@jfHR4UdG>Usw4;7ESpORL|2^#jd+@zxz{(|RV*1WKrw-)ln*8LnxVkKDfGDHA%7`HaiuvhMu%*mY9*Ya{Ti#{DW?i0 zXXsp+Bb(_~wv(3t70QU3a$*<$1&zm1t++x#wDLCRI4K)kU?Vm9n2c0m@TyUV&&l9%}fulj!Z9)&@yIcQ3gX}l0b1LbIh4S z5C*IDrYxR%qm4LVzSk{0;*npO_SocYWbkAjA6(^IAwUnoAzw_Uo}xYFo?Y<-4Zqec z&k7HtVlFGyt_pA&kX%P8PaRD8y!Wsnv}NMLNLy-CHZf(ObmzV|t-iC#@Z9*d-zUsx zxcYWw{H)nYXVdnJu5o-U+fn~W z-$h1ax>h{NlWLA7;;6TcQHA>UJB$KNk74T1xNWh9)kwK~wX0m|Jo_Z;g;>^E4-k4R zRj#pQb-Hg&dAh}*=2;JY*aiNZzT=IU&v|lQY%Q|=^V5pvTR7^t9+@+ST&sr!J1Y9a z514dYZn5rg6@4Cy6P`-?!3Y& z?B*5zw!mTiD2)>f@3XYrW^9V-@%YFkE_;PCyCJ7*?_3cR%tHng9%ZpIU}LJM=a+0s z(SDDLvcVa~b9O!cVL8)Q{d^R^(bbG=Ia$)dVN_tGMee3PMssZ7Z;c^Vg_1CjZYTnq z)wnF8?=-MmqVOMX!iE?YDvHCN?%TQtKJMFHp$~kX4}jZ;EDqP$?jqJZjoa2PM@$uZ zF4}iab1b5ep)L;jdegC3{K4VnCH#OV;pRcSa(&Nm50ze-yZ8*cGv;@+N+A?ncc^2z9~|(xFhwOHmPW@ zR5&)E^YKQj@`g=;zJ_+CLamsPuvppUr$G1#9urUj+p-mPW_QSSHkPMS!52t>Hqy|g z_@Yu3z%|wE=uYq8G>4`Q!4zivS}+}{m5Zjr7kMRGn_p&hNf|pc&f9iQ`^%78rl#~8 z;os@rpMA{ZioY~(Rm!Wf#Wx##A0PthOI341QiJ=G*#}pDAkDm+{0kz&*NB?rC0-)glB{0_Tq*^o zVS1>3REsv*Qb;qg!G^9;VoK)P*?f<*H&4Su1=}bP^Y<2PwFpoqw#up4IgX3L z`w~8jsFCI3k~Y9g(Y9Km`y$0FS5vHb)kb)Jb6q-9MbO{Hbb zxg?IWQ1ZIGgE}wKm{axO6CCh~4DyoFU+i1xn#oyfe+<{>=^B5tm!!*1M?AW8c=6g+%2Ft97_Hq&ZmOGvqGQ!Bn<_Vw`0DRuDoB6q8ME<;oL4kocr8E$NGoLI zXWmI7Af-DR|KJw!vKp2SI4W*x%A%5BgDu%8%Iato+pWo5`vH@!XqC!yK}KLzvfS(q z{!y(S-PKbk!qHsgVyxKsQWk_8HUSSmslUA9nWOjkKn0%cwn%yxnkfxn?Y2rysXKS=t-TeI%DN$sQ{lcD!(s>(4y#CSxZ4R} zFDI^HPC_l?uh_)-^ppeYRkPTPu~V^0Mt}#jrTL1Q(M;qVt4zb(L|J~sxx7Lva9`mh zz!#A9tA*6?q)xThc7(gB2Ryam$YG4qlh00c}r&$y6u zIN#Qxn{7RKJ+_r|1G1KEv!&uKfXpOVZ8tK{M775ws%nDyoZ?bi3NufNbZs)zqXiqc zqOsK@^OnlFMAT&mO3`@3nZP$3lLF;ds|;Z{W(Q-STa2>;)tjhR17OD|G>Q#zJHb*> zMO<{WIgB%_4MG0SQi2;%f0J8l_FH)Lfaa>*GLobD#AeMttYh4Yfg22@q4|Itq};NB z8;o*+@APqy@fPgrc&PTbGEwdEK=(x5K!If@R$NiO^7{#j9{~w=RBG)ZkbOw@$7Nhl zyp{*&QoVBd5lo{iwl2gfyip@}IirZK;ia(&ozNl!-EEYc=QpYH_= zJkv7gA{!n4up6$CrzDJIBAdC7D5D<_VLH*;OYN>_Dx3AT`K4Wyx8Tm{I+xplKP6k7 z2sb!i7)~%R#J0$|hK?~=u~rnH7HCUpsQJujDDE*GD`qrWWog+C+E~GGy|Hp_t4--} zrxtrgnPh}r=9o}P6jpAQuDN}I*GI`8&%Lp-C0IOJt#op)}XSr!ova@w{jG2V=?GXl3zEJJFXg)U3N>BQP z*Lb@%Mx|Tu;|u>$-K(q^-HG!EQ3o93%w(A7@ngGU)HRWoO&&^}U$5x+T&#zri>6ct zXOB#EF-;z3j311K`jrYyv6pOPF=*`SOz!ack=DuEi({UnAkL5H)@R?YbRKAeP|06U z?-Ns0ZxD0h9D8)P66Sq$w-yF+1hEVTaul%&=kKDrQtF<$RnQPZ)ezm1`aHIjAY=!S z`%vboP`?7mItgEo4w50C*}Ycqp9_3ZEr^F1;cEhkb`BNhbc6PvnXu@wi=AoezF4~K zkxx%ps<8zb=wJ+9I8o#do)&{(=yAlNdduaDn!=xGSiuo~fLw~Edw$6;l-qaq#Z7?# zGrdU(Cf-V@$x>O%yRc6!C1Vf`b19ly;=mEu8u9|zitcG^O`lbNh}k=$%a)UHhDwTEKis2yc4rBGR>l*(B$AC7ung&ssaZGkY-h(fpwcPyJSx*9EIJMRKbMP9}$nVrh6$g-Q^5Cw)BeWqb-qi#37ZXKL!GR;ql)~ z@PP*-oP?T|ThqlGKR84zi^CN z4TZ1A)7vL>ivoL2EU_~xl-P{p+sE}9CRwGJDKy{>0KP+gj`H9C+4fUMPnIB1_D`A- z$1`G}g0lQmqMN{Y&8R*$xYUB*V}dQPxGVZQ+rH!DVohIoTbh%#z#Tru%Px@C<=|og zGDDwGq7yz`%^?r~6t&>x*^We^tZ4!E4dhwsht#Pb1kCY{q#Kv;z%Dp#Dq;$vH$-(9 z8S5tutZ}&JM2Iw&Y-7KY4h5BBvS=Ove0#+H2qPdR)WyI zYcj)vB=MA{7T|3Ij_PN@FM@w(C9ANBq&|NoW30ccr~i#)EcH)T^3St~rJ0HKKd4wr z@_+132;Bj+>UC@h)Ap*8B4r5A1lZ!Dh%H7&&hBnlFj@eayk=VD*i5AQc z$uN8YG#PL;cuQa)Hyt-}R?&NAE1QT>svJDKt*)AQOZAJ@ zyxJoBebiobHeFlcLwu_iI&NEZuipnOR;Tn;PbT1Mt-#5v5b*8ULo7m)L-eti=UcGf zRZXidmxeFgY!y80-*PH-*=(-W+fK%KyUKpg$X@tuv``tXj^*4qq@UkW$ZrAo%+hay zU@a?z&2_@y)o@D!_g>NVxFBO!EyB&6Z!nd4=KyDP^hl!*(k{dEF6@NkXztO7gIh zQ&PC+p-8WBv;N(rpfKdF^@Z~|E6pa)M1NBUrCZvLRW$%N%xIbv^uv?=C!=dDVq3%* zgvbEBnG*JB*@vXx8>)7XL*!{1Jh=#2UrByF7U?Rj_}VYw88BwqefT_cCTv8aTrRVjnn z1HNCF=44?*&gs2`vCGJVHX@kO z240eo#z+FhI0=yy6NHQwZs}a+J~4U-6X`@ zZ7j+tb##m`x%J66$a9qXDHG&^kp|GkFFMmjD(Y-k_ClY~N$H|n@NkSDz=gg?*2ga5 z)+f)MEY>2Lp15;~o`t`qj;S>BaE;%dv@Ux11yq}I(k|o&`5UZFUHn}1kE^gIK@qV& z!S2IhyU;->VfA4Qb}m7YnkIa9%z{l~iPWo2YPk-`hy2-Eg=6E$21plQA5W2qMZDFU z-a-@Dndf%#on6chT`dOKnU9}BJo|kJwgGC<^nfo34zOKH96LbWY7@Wc%EoFF=}`VU zksP@wd%@W;-p!e^&-)N7#oR331Q)@9cx=mOoU?_Kih2!Le*8fhsZ8Qvo6t2vt+UOZ zw|mCB*t2%z21YqL>whu!j?s~}-L`OS+jdg1(XnmYw$rg~r(?5Y+qTg`$F}q3J?GtL z@BN&8#`u2RqkdG4yGGTus@7U_%{6C{XAhFE!2SelH?KtMtX@B1GBhEIDL-Bj#~{4! zd}p7!#XE9Lt;sy@p5#Wj*jf8zGv6tTotCR2X$EVOOup;GnRPRVU5A6N@Lh8?eA7k? zn~hz&gY;B0ybSpF?qwQ|sv_yO=8}zeg2$0n3A8KpE@q26)?707pPw?H76lCpjp=5r z6jjp|auXJDnW}uLb6d7rsxekbET9(=zdTqC8(F5@NNqII2+~yB;X5iJNQSiv`#ozm zf&p!;>8xAlwoxUC3DQ#!31ylK%VrcwS<$WeCY4V63V!|221oj+5#r}fGFQ}|uwC0) zNl8(CF}PD`&Sj+p{d!B&&JtC+VuH z#>US`)YQrhb6lIAYb08H22y(?)&L8MIQsA{26X`R5Km{YU)s!x(&gIsjDvq63@X`{ z=7{SiH*_ZsPME#t2m|bS76Uz*z{cpp1m|s}HIX}Ntx#v7Eo!1%G9__4dGSGl`p+xi zZ!VK#Qe;Re=9bqXuW+0DSP{uZ5-QXrNn-7qW19K0qU}OhVru7}3vqsG?#D67 zb}crN;QwsH*vymw(maZr_o|w&@sQki(X+D)gc5Bt&@iXisFG;eH@5d43~Wxq|HO(@ zV-rip4n#PEkHCWCa5d?@cQp^B;I-PzOfag|t-cuvTapQ@MWLmh*41NH`<+A+JGyKX zyYL6Ba7qqa5j@3lOk~`OMO7f0!@FaOeZxkbG@vXP(t3#U*fq8=GAPqUAS>vW2uxMk{a(<0=IxB;# zMW;M+owrHaZBp`3{e@7gJCHP!I(EeyGFF;pdFPdeP+KphrulPSVidmg#!@W`GpD&d z9p6R`dpjaR2E1Eg)Ws{BVCBU9-aCgN57N~uLvQZH`@T+2eOBD%73rr&sV~m#2~IZx zY_8f8O;XLu2~E3JDXnGhFvsyb^>*!D>5EtlKPe%kOLv6*@=Jpci`8h0z?+fbBUg_7 zu6DjqO=$SjAv{|Om5)nz41ZkS4E_|fk%NDY509VV5yNeo%O|sb>7C#wj8mL9cEOFh z>nDz%?vb!h*!0dHdnxDA>97~EoT~!N40>+)G2CeYdOvJr5^VnkGz)et&T9hrD(VAgCAJjQ7V$O?csICB*HFd^k@$M5*v$PZJD-OVL?Ze(U=XGqZPVG8JQ z<~ukO%&%nNXYaaRibq#B1KfW4+XMliC*Tng2G(T1VvP;2K~;b$EAqthc${gjn_P!b zs62UT(->A>!ot}cJXMZHuy)^qfqW~xO-In2);e>Ta{LD6VG2u&UT&a@>r-;4<)cJ9 zjpQThb4^CY)Ev0KR7TBuT#-v}W?Xzj{c7$S5_zJA57Qf=$4^npEjl9clH0=jWO8sX z3Fuu0@S!WY>0XX7arjH`?)I<%2|8HfL!~#c+&!ZVmhbh`wbzy0Ux|Jpy9A{_7GGB0 zadZ48dW0oUwUAHl%|E-Q{gA{z6TXsvU#Hj09<7i)d}wa+Iya)S$CVwG{4LqtB>w%S zKZx(QbV7J9pYt`W4+0~f{hoo5ZG<0O&&5L57oF%hc0xGJ@Zrg_D&lNO=-I^0y#3mxCSZFxN2-tN_mU@7<@PnWG?L5OSqkm8TR!`| zRcTeWH~0z1JY^%!N<(TtxSP5^G9*Vw1wub`tC-F`=U)&sJVfvmh#Pi`*44kSdG};1 zJbHOmy4Ot|%_?@$N?RA9fF?|CywR8Sf(SCN_luM8>(u0NSEbKUy7C(Sk&OuWffj)f za`+mo+kM_8OLuCUiA*CNE|?jra$M=$F3t+h-)?pXz&r^F!ck;r##`)i)t?AWq-9A9 zSY{m~TC1w>HdEaiR*%j)L);H{IULw)uxDO>#+WcBUe^HU)~L|9#0D<*Ld459xTyew zbh5vCg$a>`RCVk)#~ByCv@Ce!nm<#EW|9j><#jQ8JfTmK#~jJ&o0Fs9jz0Ux{svdM4__<1 zrb>H(qBO;v(pXPf5_?XDq!*3KW^4>(XTo=6O2MJdM^N4IIcYn1sZZpnmMAEdt}4SU zPO54j2d|(xJtQ9EX-YrlXU1}6*h{zjn`in-N!Ls}IJsG@X&lfycsoCemt_Ym(PXhv zc*QTnkNIV=Ia%tg%pwJtT^+`v8ng>;2~ps~wdqZSNI7+}-3r+#r6p`8*G;~bVFzg= z!S3&y)#iNSUF6z;%o)%h!ORhE?CUs%g(k2a-d576uOP2@QwG-6LT*G!I$JQLpd`cz z-2=Brr_+z96a0*aIhY2%0(Sz=|D`_v_7h%Yqbw2)8@1DwH4s*A82krEk{ zoa`LbCdS)R?egRWNeHV8KJG0Ypy!#}kslun?67}^+J&02!D??lN~t@;h?GS8#WX`)6yC**~5YNhN_Hj}YG<%2ao^bpD8RpgV|V|GQwlL27B zEuah|)%m1s8C6>FLY0DFe9Ob66fo&b8%iUN=y_Qj;t3WGlNqP9^d#75ftCPA*R4E8 z)SWKBKkEzTr4JqRMEs`)0;x8C35yRAV++n(Cm5++?WB@ya=l8pFL`N0ag`lWhrYo3 zJJ$< zQ*_YAqIGR*;`VzAEx1Pd4b3_oWtdcs7LU2#1#Ls>Ynvd8k^M{Ef?8`RxA3!Th-?ui{_WJvhzY4FiPxA?E4+NFmaC-Uh*a zeLKkkECqy>Qx&1xxEhh8SzMML=8VP}?b*sgT9ypBLF)Zh#w&JzP>ymrM?nnvt!@$2 zh>N$Q>mbPAC2kNd&ab;FkBJ}39s*TYY0=@e?N7GX>wqaM>P=Y12lciUmve_jMF0lY zBfI3U2{33vWo(DiSOc}!5##TDr|dgX1Uojq9!vW3$m#zM_83EGsP6&O`@v-PDdO3P z>#!BEbqpOXd5s?QNnN!p+92SHy{sdpePXHL{d@c6UilT<#~I!tH$S(~o}c#(j<2%! zQvm}MvAj-95Ekx3D4+|e%!?lO(F+DFw9bxb-}rsWQl)b44###eUg4N?N-P(sFH2hF z`{zu?LmAxn2=2wCE8?;%ZDi#Y;Fzp+RnY8fWlzVz_*PDO6?Je&aEmuS>=uCXgdP6r zoc_JB^TA~rU5*geh{G*gl%_HnISMS~^@{@KVC;(aL^ZA-De+1zwUSXgT>OY)W?d6~ z72znET0m`53q%AVUcGraYxIcAB?OZA8AT!uK8jU+=t;WneL~|IeQ>$*dWa#x%rB(+ z5?xEkZ&b{HsZ4Ju9TQ|)c_SIp`7r2qMJgaglfSBHhl)QO1aNtkGr0LUn{@mvAt=}nd7#>7ru}&I)FNsa*x?Oe3-4G`HcaR zJ}c%iKlwh`x)yX1vBB;-Nr=7>$~(u=AuPX2#&Eh~IeFw%afU+U)td0KC!pHd zyn+X$L|(H3uNit-bpn7%G%{&LsAaEfEsD?yM<;U2}WtD4KuVKuX=ec9X zIe*ibp1?$gPL7<0uj*vmj2lWKe`U(f9E{KVbr&q*RsO;O>K{i-7W)8KG5~~uS++56 zm@XGrX@x+lGEjDQJp~XCkEyJG5Y57omJhGN{^2z5lj-()PVR&wWnDk2M?n_TYR(gM zw4kQ|+i}3z6YZq8gVUN}KiYre^sL{ynS}o{z$s&I z{(rWaLXxcQ=MB(Cz7W$??Tn*$1y(7XX)tv;I-{7F$fPB%6YC7>-Dk#=Y8o1=&|>t5 zV_VVts>Eb@)&4%m}!K*WfLoLl|3FW)V~E1Z!yu`Sn+bAP5sRDyu7NEbLt?khAyz-ZyL-}MYb&nQ zU16f@q7E1rh!)d%f^tTHE3cVoa%Xs%rKFc|temN1sa)aSlT*)*4k?Z>b3NP(IRXfq zlB^#G6BDA1%t9^Nw1BD>lBV(0XW5c?l%vyB3)q*;Z5V~SU;HkN;1kA3Nx!$!9wti= zB8>n`gt;VlBt%5xmDxjfl0>`K$fTU-C6_Z;!A_liu0@Os5reMLNk;jrlVF^FbLETI zW+Z_5m|ozNBn7AaQ<&7zk}(jmEdCsPgmo%^GXo>YYt82n&7I-uQ%A;k{nS~VYGDTn zlr3}HbWQG6xu8+bFu^9%%^PYCbkLf=*J|hr>Sw+#l(Y#ZGKDufa#f-f0k-{-XOb4i zwVG1Oa0L2+&(u$S7TvedS<1m45*>a~5tuOZ;3x%!f``{=2QQlJk|b4>NpD4&L+xI+ z+}S(m3}|8|Vv(KYAGyZK5x*sgwOOJklN0jsq|BomM>OuRDVFf_?cMq%B*iQ*&|vS9 zVH7Kh)SjrCBv+FYAE=$0V&NIW=xP>d-s7@wM*sdfjVx6-Y@=~>rz%2L*rKp|*WXIz z*vR^4tV&7MQpS9%{9b*>E9d_ls|toL7J|;srnW{l-}1gP_Qr-bBHt=}PL@WlE|&KH zCUmDLZb%J$ZzNii-5VeygOM?K8e$EcK=z-hIk63o4y63^_*RdaitO^THC{boKstphXZ2Z+&3ToeLQUG(0Frs?b zCxB+65h7R$+LsbmL51Kc)pz_`YpGEzFEclzb=?FJ=>rJwgcp0QH-UuKRS1*yCHsO) z-8t?Zw|6t($Eh&4K+u$I7HqVJBOOFCRcmMMH};RX_b?;rnk`rz@vxT_&|6V@q0~Uk z9ax|!pA@Lwn8h7syrEtDluZ6G!;@=GL> zse#PRQrdDs=qa_v@{Wv(3YjYD0|qocDC;-F~&{oaTP?@pi$n z1L6SlmFU2~%)M^$@C(^cD!y)-2SeHo3t?u3JiN7UBa7E2 z;<+_A$V084@>&u)*C<4h7jw9joHuSpVsy8GZVT;(>lZ(RAr!;)bwM~o__Gm~exd`K zKEgh2)w?ReH&syI`~;Uo4`x4$&X+dYKI{e`dS~bQuS|p zA`P_{QLV3r$*~lb=9vR^H0AxK9_+dmHX}Y} zIV*#65%jRWem5Z($ji{!6ug$En4O*=^CiG=K zp4S?+xE|6!cn$A%XutqNEgUqYY3fw&N(Z6=@W6*bxdp~i_yz5VcgSj=lf-6X1Nz75 z^DabwZ4*70$$8NsEy@U^W67tcy7^lNbu;|kOLcJ40A%J#pZe0d#n zC{)}+p+?8*ftUlxJE*!%$`h~|KZSaCb=jpK3byAcuHk7wk@?YxkT1!|r({P*KY^`u z!hw#`5$JJZGt@nkBK_nwWA31_Q9UGvv9r-{NU<&7HHMQsq=sn@O?e~fwl20tnSBG* zO%4?Ew6`aX=I5lqmy&OkmtU}bH-+zvJ_CFy z_nw#!8Rap5Wcex#5}Ldtqhr_Z$}@jPuYljTosS1+WG+TxZ>dGeT)?ZP3#3>sf#KOG z0)s%{cEHBkS)019}-1A2kd*it>y65-C zh7J9zogM74?PU)0c0YavY7g~%j%yiWEGDb+;Ew5g5Gq@MpVFFBNOpu0x)>Yn>G6uo zKE%z1EhkG_N5$a8f6SRm(25iH#FMeaJ1^TBcBy<04ID47(1(D)q}g=_6#^V@yI?Y&@HUf z`;ojGDdsvRCoTmasXndENqfWkOw=#cV-9*QClpI03)FWcx(m5(P1DW+2-{Hr-`5M{v##Zu-i-9Cvt;V|n)1pR^y ztp3IXzHjYWqabuPqnCY9^^;adc!a%Z35VN~TzwAxq{NU&Kp35m?fw_^D{wzB}4FVXX5Zk@#={6jRh%wx|!eu@Xp;%x+{2;}!&J4X*_SvtkqE#KDIPPn@ z5BE$3uRlb>N<2A$g_cuRQM1T#5ra9u2x9pQuqF1l2#N{Q!jVJ<>HlLeVW|fN|#vqSnRr<0 zTVs=)7d`=EsJXkZLJgv~9JB&ay16xDG6v(J2eZy;U%a@EbAB-=C?PpA9@}?_Yfb&) zBpsih5m1U9Px<+2$TBJ@7s9HW>W){i&XKLZ_{1Wzh-o!l5_S+f$j^RNYo85}uVhN# zq}_mN-d=n{>fZD2Lx$Twd2)}X2ceasu91}n&BS+4U9=Y{aZCgV5# z?z_Hq-knIbgIpnkGzJz-NW*=p?3l(}y3(aPCW=A({g9CpjJfYuZ%#Tz81Y)al?!S~ z9AS5#&nzm*NF?2tCR#|D-EjBWifFR=da6hW^PHTl&km-WI9*F4o>5J{LBSieVk`KO z2(^9R(zC$@g|i3}`mK-qFZ33PD34jd_qOAFj29687wCUy>;(Hwo%Me&c=~)V$ua)V zsaM(aThQ3{TiM~;gTckp)LFvN?%TlO-;$y+YX4i`SU0hbm<})t0zZ!t1=wY&j#N>q zONEHIB^RW6D5N*cq6^+?T}$3m|L{Fe+L!rxJ=KRjlJS~|z-&CC{#CU8`}2|lo~)<| zk?Wi1;Cr;`?02-C_3^gD{|Ryhw!8i?yx5i0v5?p)9wZxSkwn z3C;pz25KR&7{|rc4H)V~y8%+6lX&KN&=^$Wqu+}}n{Y~K4XpI-#O?L=(2qncYNePX zTsB6_3`7q&e0K67=Kg7G=j#?r!j0S^w7;0?CJbB3_C4_8X*Q%F1%cmB{g%XE&|IA7 z(#?AeG{l)s_orNJp!$Q~qGrj*YnuKlV`nVdg4vkTNS~w$4d^Oc3(dxi(W5jq0e>x} z(GN1?u2%Sy;GA|B%Sk)ukr#v*UJU%(BE9X54!&KL9A^&rR%v zIdYt0&D59ggM}CKWyxGS@ z>T#})2Bk8sZMGJYFJtc>D#k0+Rrrs)2DG;(u(DB_v-sVg=GFMlSCx<&RL;BH}d6AG3VqP!JpC0Gv6f8d|+7YRC@g|=N=C2 zo>^0CE0*RW?W))S(N)}NKA)aSwsR{1*rs$(cZIs?nF9)G*bSr%%SZo^YQ|TSz={jX z4Z+(~v_>RH0(|IZ-_D_h@~p_i%k^XEi+CJVC~B zsPir zA0Jm2yIdo4`&I`hd%$Bv=Rq#-#bh{Mxb_{PN%trcf(#J3S1UKDfC1QjH2E;>wUf5= ze8tY9QSYx0J;$JUR-0ar6fuiQTCQP#P|WEq;Ez|*@d?JHu-(?*tTpGHC+=Q%H>&I> z*jC7%nJIy+HeoURWN%3X47UUusY2h7nckRxh8-)J61Zvn@j-uPA@99|y48pO)0XcW zX^d&kW^p7xsvdX?2QZ8cEUbMZ7`&n{%Bo*xgFr4&fd#tHOEboQos~xm8q&W;fqrj} z%KYnnE%R`=`+?lu-O+J9r@+$%YnqYq!SVs>xp;%Q8p^$wA~oynhnvIFp^)Z2CvcyC zIN-_3EUHW}1^VQ0;Oj>q?mkPx$Wj-i7QoXgQ!HyRh6Gj8p~gH22k&nmEqUR^)9qni{%uNeV{&0-H60C zibHZtbV=8=aX!xFvkO}T@lJ_4&ki$d+0ns3FXb+iP-VAVN`B7f-hO)jyh#4#_$XG%Txk6M<+q6D~ zi*UcgRBOoP$7P6RmaPZ2%MG}CMfs=>*~(b97V4+2qdwvwA@>U3QQAA$hiN9zi%Mq{ z*#fH57zUmi)GEefh7@`Uy7?@@=BL7cXbd{O9)*lJh*v!@ z-6}p9u0AreiGauxn7JBEa-2w&d=!*TLJ49`U@D7%2ppIh)ynMaAE2Q4dl@47cNu{9 z&3vT#pG$#%hrXzXsj=&Ss*0;W`Jo^mcy4*L8b^sSi;H{*`zW9xX2HAtQ*sO|x$c6UbRA(7*9=;D~(%wfo(Z6#s$S zuFk`dr%DfVX5KC|Af8@AIr8@OAVj=6iX!~8D_P>p7>s!Hj+X0_t}Y*T4L5V->A@Zx zcm1wN;TNq=h`5W&>z5cNA99U1lY6+!!u$ib|41VMcJk8`+kP{PEOUvc@2@fW(bh5pp6>C3T55@XlpsAd#vn~__3H;Dz2w=t9v&{v*)1m4)vX;4 zX4YAjM66?Z7kD@XX{e`f1t_ZvYyi*puSNhVPq%jeyBteaOHo7vOr8!qqp7wV;)%jtD5>}-a?xavZ;i|2P3~7c)vP2O#Fb`Y&Kce zQNr7%fr4#S)OOV-1piOf7NgQvR{lcvZ*SNbLMq(olrdDC6su;ubp5un!&oT=jVTC3uTw7|r;@&y*s)a<{J zkzG(PApmMCpMmuh6GkM_`AsBE@t~)EDcq1AJ~N@7bqyW_i!mtHGnVgBA`Dxi^P93i z5R;}AQ60wy=Q2GUnSwz+W6C^}qn`S-lY7=J(3#BlOK%pCl=|RVWhC|IDj1E#+|M{TV0vE;vMZLy7KpD1$Yk zi0!9%qy8>CyrcRK`juQ)I};r)5|_<<9x)32b3DT1M`>v^ld!yabX6@ihf`3ZVTgME zfy(l-ocFuZ(L&OM4=1N#Mrrm_<>1DZpoWTO70U8+x4r3BpqH6z@(4~sqv!A9_L}@7 z7o~;|?~s-b?ud&Wx6==9{4uTcS|0-p@dKi0y#tPm2`A!^o3fZ8Uidxq|uz2vxf;wr zM^%#9)h^R&T;}cxVI(XX7kKPEVb);AQO?cFT-ub=%lZPwxefymBk+!H!W(o(>I{jW z$h;xuNUr#^0ivvSB-YEbUqe$GLSGrU$B3q28&oA55l)ChKOrwiTyI~e*uN;^V@g-Dm4d|MK!ol8hoaSB%iOQ#i_@`EYK_9ZEjFZ8Ho7P^er z^2U6ZNQ{*hcEm?R-lK)pD_r(e=Jfe?5VkJ$2~Oq^7YjE^5(6a6Il--j@6dBHx2Ulq z!%hz{d-S~i9Eo~WvQYDt7O7*G9CP#nrKE#DtIEbe_uxptcCSmYZMqT2F}7Kw0AWWC zPjwo0IYZ6klc(h9uL|NY$;{SGm4R8Bt^^q{e#foMxfCSY^-c&IVPl|A_ru!ebwR#7 z3<4+nZL(mEsU}O9e`^XB4^*m)73hd04HH%6ok^!;4|JAENnEr~%s6W~8KWD)3MD*+ zRc46yo<}8|!|yW-+KulE86aB_T4pDgL$XyiRW(OOcnP4|2;v!m2fB7Hw-IkY#wYfF zP4w;k-RInWr4fbz=X$J;z2E8pvAuy9kLJUSl8_USi;rW`kZGF?*Ur%%(t$^{Rg!=v zg;h3@!Q$eTa7S0#APEDHLvK%RCn^o0u!xC1Y0Jg!Baht*a4mmKHy~88md{YmN#x) zBOAp_i-z2h#V~*oO-9k(BizR^l#Vm%uSa^~3337d;f=AhVp?heJ)nlZGm`}D(U^2w z#vC}o1g1h?RAV^90N|Jd@M00PoNUPyA?@HeX0P7`TKSA=*4s@R;Ulo4Ih{W^CD{c8 ze(ipN{CAXP(KHJ7UvpOc@9SUAS^wKo3h-}BDZu}-qjdNlVtp^Z{|CxKOEo?tB}-4; zEXyDzGbXttJ3V$lLo-D?HYwZm7vvwdRo}P#KVF>F|M&eJ44n*ZO~0)#0e0Vy&j00I z{%IrnUvKp70P?>~J^$^0Wo%>le>re2ZSvRfes@dC-*e=DD1-j%<$^~4^4>Id5w^Fr z{RWL>EbUCcyC%1980kOYqZAcgdz5cS8c^7%vvrc@CSPIx;X=RuodO2dxk17|am?HJ@d~Mp_l8H?T;5l0&WGFoTKM{eP!L-a0O8?w zgBPhY78tqf^+xv4#OK2I#0L-cSbEUWH2z+sDur85*!hjEhFfD!i0Eyr-RRLFEm5(n z-RV6Zf_qMxN5S6#8fr9vDL01PxzHr7wgOn%0Htmvk9*gP^Um=n^+7GLs#GmU&a#U^4jr)BkIubQO7oUG!4CneO2Ixa`e~+Jp9m{l6apL8SOqA^ zvrfEUPwnHQ8;yBt!&(hAwASmL?Axitiqvx%KZRRP?tj2521wyxN3ZD9buj4e;2y6U zw=TKh$4%tt(eh|y#*{flUJ5t4VyP*@3af`hyY^YU3LCE3Z|22iRK7M7E;1SZVHbXF zKVw!L?2bS|kl7rN4(*4h2qxyLjWG0vR@`M~QFPsf^KParmCX;Gh4OX6Uy9#4e_%oK zv1DRnfvd$pu(kUoV(MmAc09ckDiuqS$a%!AQ1Z>@DM#}-yAP$l`oV`BDYpkqpk(I|+qk!yoo$TwWr6dRzLy(c zi+qbVlYGz0XUq@;Fm3r~_p%by)S&SVWS+wS0rC9bk^3K^_@6N5|2rtF)wI>WJ=;Fz zn8$h<|Dr%kN|nciMwJAv;_%3XG9sDnO@i&pKVNEfziH_gxKy{l zo`2m4rnUT(qenuq9B0<#Iy(RPxP8R)=5~9wBku=%&EBoZ82x1GlV<>R=hIqf0PK!V zw?{z9e^B`bGyg2nH!^x}06oE%J_JLk)^QyHLipoCs2MWIqc>vaxsJj(=gg1ZSa=u{ zt}od#V;e7sA4S(V9^<^TZ#InyVBFT(V#$fvI7Q+pgsr_2X`N~8)IOZtX}e(Bn(;eF zsNj#qOF_bHl$nw5!ULY{lNx@93Fj}%R@lewUuJ*X*1$K`DNAFpE z7_lPE+!}uZ6c?+6NY1!QREg#iFy=Z!OEW}CXBd~wW|r_9%zkUPR0A3m+@Nk%4p>)F zXVut7$aOZ6`w}%+WV$te6-IX7g2yms@aLygaTlIv3=Jl#Nr}nN zp|vH-3L03#%-1-!mY`1z?+K1E>8K09G~JcxfS)%DZbteGQnQhaCGE2Y<{ut#(k-DL zh&5PLpi9x3$HM82dS!M?(Z zEsqW?dx-K_GMQu5K54pYJD=5+Rn&@bGjB?3$xgYl-|`FElp}?zP&RAd<522c$Rv6} zcM%rYClU%JB#GuS>FNb{P2q*oHy}UcQ-pZ2UlT~zXt5*k-ZalE(`p7<`0n7i(r2k{ zb84&^LA7+aW1Gx5!wK!xTbw0slM?6-i32CaOcLC2B>ZRI16d{&-$QBEu1fKF0dVU>GTP05x2>Tmdy`75Qx! z^IG;HB9V1-D5&&)zjJ&~G}VU1-x7EUlT3QgNT<&eIDUPYey$M|RD6%mVkoDe|;2`8Z+_{0&scCq>Mh3hj|E*|W3;y@{$qhu77D)QJ` znD9C1AHCKSAHQqdWBiP`-cAjq7`V%~JFES1=i-s5h6xVT<50kiAH_dn0KQB4t*=ua zz}F@mcKjhB;^7ka@WbSJFZRPeYI&JFkpJ-!B z!ju#!6IzJ;D@$Qhvz9IGY5!%TD&(db3<*sCpZ?U#1^9RWQ zs*O-)j!E85SMKtoZzE^8{w%E0R0b2lwwSJ%@E}Lou)iLmPQyO=eirG8h#o&E4~eew z;h><=|4m0$`ANTOixHQOGpksXlF0yy17E&JksB4_(vKR5s$Ve+i;gco2}^RRJI+~R zWJ82WGigLIUwP!uSELh3AAs9HmY-kz=_EL-w|9}noKE#(a;QBpEx9 z4BT-zY=6dJT>72Hkz=9J1E=}*MC;zzzUWb@x(Ho8cU_aRZ?fxse5_Ru2YOvcr?kg&pt@v;{ai7G--k$LQtoYj+Wjk+nnZty;XzANsrhoH#7=xVqfPIW(p zX5{YF+5=k4_LBnhLUZxX*O?29olfPS?u*ybhM_y z*XHUqM6OLB#lyTB`v<BZ&YRs$N)S@5Kn_b3;gjz6>fh@^j%y2-ya({>Hd@kv{CZZ2e)tva7gxLLp z`HoGW);eRtov~Ro5tetU2y72~ zQh>D`@dt@s^csdfN-*U&o*)i3c4oBufCa0e|BwT2y%Y~=U7A^ny}tx zHwA>Wm|!SCko~UN?hporyQHRUWl3djIc722EKbTIXQ6>>iC!x+cq^sUxVSj~u)dsY zW8QgfZlE*2Os%=K;_vy3wx{0u!2%A)qEG-$R^`($%AOfnA^LpkB_}Dd7AymC)zSQr z>C&N8V57)aeX8ap!|7vWaK6=-3~ko9meugAlBKYGOjc#36+KJwQKRNa_`W@7;a>ot zdRiJkz?+QgC$b}-Owzuaw3zBVLEugOp6UeMHAKo2$m4w zpw?i%Lft^UtuLI}wd4(-9Z^*lVoa}11~+0|Hs6zAgJ01`dEA&^>Ai=mr0nC%eBd_B zzgv2G_~1c1wr*q@QqVW*Wi1zn=}KCtSwLjwT>ndXE_Xa22HHL_xCDhkM( zhbw+j4uZM|r&3h=Z#YrxGo}GX`)AZyv@7#7+nd-D?BZV>thtc|3jt30j$9{aIw9)v zDY)*fsSLPQTNa&>UL^RWH(vpNXT7HBv@9=*=(Q?3#H*crA2>KYx7Ab?-(HU~a275)MBp~`P)hhzSsbj|d`aBe(L*(;zif{iFJu**ZR zkL-tPyh!#*r-JVQJq>5b0?cCy!uSKef+R=$s3iA7*k*_l&*e!$F zYwGI;=S^0)b`mP8&Ry@{R(dPfykD&?H)na^ihVS7KXkxb36TbGm%X1!QSmbV9^#>A z-%X>wljnTMU0#d;tpw?O1W@{X-k*>aOImeG z#N^x?ehaaQd}ReQykp>i;92q@%$a!y1PNyPYDIvMm& zyYVwn;+0({W@3h(r&i#FuCDE)AC(y&Vu>4?1@j0|CWnhHUx4|zL7cdaA32RSk?wl% zMK^n42@i5AU>f70(huWfOwaucbaToxj%+)7hnG^CjH|O`A}+GHZyQ-X57(WuiyRXV zPf>0N3GJ<2Myg!sE4XJY?Z7@K3ZgHy8f7CS5ton0Eq)Cp`iLROAglnsiEXpnI+S8; zZn>g2VqLxi^p8#F#Laf3<00AcT}Qh&kQnd^28u!9l1m^`lfh9+5$VNv=?(~Gl2wAl zx(w$Z2!_oESg_3Kk0hUsBJ<;OTPyL(?z6xj6LG5|Ic4II*P+_=ac7KRJZ`(k2R$L# zv|oWM@116K7r3^EL*j2ktjEEOY9c!IhnyqD&oy7+645^+@z5Y|;0+dyR2X6^%7GD* zXrbPqTO}O={ z4cGaI#DdpP;5u?lcNb($V`l>H7k7otl_jQFu1hh>=(?CTPN#IPO%O_rlVX}_Nq;L< z@YNiY>-W~&E@=EC5%o_z<^3YEw)i_c|NXxHF{=7U7Ev&C`c^0Z4-LGKXu*Hkk&Av= zG&RAv{cR7o4${k~f{F~J48Ks&o(D@j-PQ2`LL@I~b=ifx3q!p6`d>~Y!<-^mMk3)e zhi1;(YLU5KH}zzZNhl^`0HT(r`5FfmDEzxa zk&J7WQ|!v~TyDWdXQ)!AN_Y%xM*!jv^`s)A`|F%;eGg27KYsrCE2H}7*r)zvum6B{ z$k5Har9pv!dcG%f|3hE(#hFH+12RZPycVi?2y`-9I7JHryMn3 z9Y8?==_(vOAJ7PnT<0&85`_jMD0#ipta~Q3M!q5H1D@Nj-YXI$W%OQplM(GWZ5Lpq z-He6ul|3<;ZQsqs!{Y7x`FV@pOQc4|N;)qgtRe(Uf?|YqZv^$k8On7DJ5>f2%M=TV zw~x}9o=mh$JVF{v4H5Su1pq66+mhTG6?F>Do}x{V(TgFwuLfvNP^ijkrp5#s4UT!~ zEU7pr8aA)2z1zb|X9IpmJykQcqI#(rS|A4&=TtWu@g^;JCN`2kL}%+K!KlgC z>P)v+uCeI{1KZpewf>C=?N7%1e10Y3pQCZST1GT5fVyB1`q)JqCLXM zSN0qlreH1=%Zg-5`(dlfSHI&2?^SQdbEE&W4#%Eve2-EnX>NfboD<2l((>>34lE%) zS6PWibEvuBG7)KQo_`?KHSPk+2P;`}#xEs}0!;yPaTrR#j(2H|#-CbVnTt_?9aG`o z(4IPU*n>`cw2V~HM#O`Z^bv|cK|K};buJ|#{reT8R)f+P2<3$0YGh!lqx3&a_wi2Q zN^U|U$w4NP!Z>5|O)>$GjS5wqL3T8jTn%Vfg3_KnyUM{M`?bm)9oqZP&1w1)o=@+(5eUF@=P~ zk2B5AKxQ96n-6lyjh&xD!gHCzD$}OOdKQQk7LXS-fk2uy#h{ktqDo{o&>O!6%B|)` zg?|JgcH{P*5SoE3(}QyGc=@hqlB5w;bnmF#pL4iH`TSuft$dE5j^qP2S)?)@pjRQZ zBfo6g>c!|bN-Y|(Wah2o61Vd|OtXS?1`Fu&mFZ^yzUd4lgu7V|MRdGj3e#V`=mnk- zZ@LHn?@dDi=I^}R?}mZwduik!hC%=Hcl56u{Wrk1|1SxlgnzG&e7Vzh*wNM(6Y!~m z`cm8Ygc1$@z9u9=m5vs1(XXvH;q16fxyX4&e5dP-{!Kd555FD6G^sOXHyaCLka|8j zKKW^E>}>URx736WWNf?U6Dbd37Va3wQkiE;5F!quSnVKnmaIRl)b5rM_ICu4txs+w zj}nsd0I_VG^<%DMR8Zf}vh}kk;heOQTbl ziEoE;9@FBIfR7OO9y4Pwyz02OeA$n)mESpj zdd=xPwA`nO06uGGsXr4n>Cjot7m^~2X~V4yH&- zv2llS{|und45}Pm1-_W@)a-`vFBpD~>eVP(-rVHIIA|HD@%7>k8JPI-O*<7X{L*Ik zh^K`aEN!BteiRaY82FVo6<^8_22=aDIa8P&2A3V<(BQ;;x8Zs-1WuLRWjQvKv1rd2 zt%+fZ!L|ISVKT?$3iCK#7whp|1ivz1rV*R>yc5dS3kIKy_0`)n*%bfNyw%e7Uo}Mnnf>QwDgeH$X5eg_)!pI4EJjh6?kkG2oc6Af0py z(txE}$ukD|Zn=c+R`Oq;m~CSY{ebu9?!is}01sOK_mB?{lSY33E=!KkKtMeI*FO2b z%95awv9;Z|UDp3xm+aP*5I!R-_M2;GxeCRx3ATS0iF<_Do2Mi)Hk2 zjBF35VB>(oamIYjunu?g0O-?LuOvtfs5F(iiIicbu$HMPPF%F>pE@hIRjzT)>aa=m zwe;H9&+2|S!m74!E3xfO{l3E_ab`Q^tZ4yH9=~o2DUEtEMDqG=&D*8!>?2uao%w`&)THr z^>=L3HJquY>6)>dW4pCWbzrIB+>rdr{s}}cL_?#!sOPztRwPm1B=!jP7lQG|Iy6rP zVqZDNA;xaUx&xUt?Ox|;`9?oz`C0#}mc<1Urs#vTW4wd{1_r`eX=BeSV z_9WV*9mz>PH6b^z{VYQJ1nSTSqOFHE9u>cY)m`Q>=w1NzUShxcHsAxasnF2BG;NQ; zqL1tjLjImz_`q=|bAOr_i5_NEijqYZ^;d5y3ZFj6kCYakJh**N_wbfH;ICXq?-p#r z{{ljNDPSytOaG#7=yPmA&5gyYI%^7pLnMOw-RK}#*dk=@usL;|4US?{@K%7esmc&n z5$D*+l&C9)Bo@$d;Nwipd!68&+NnOj^<~vRcKLX>e03E|;to;$ndgR;9~&S-ly5gf z{rzj+j-g$;O|u?;wwxrEpD=8iFzUHQfl{B>bLHqH(9P zI59SS2PEBE;{zJUlcmf(T4DrcO?XRWR}?fekN<($1&AJTRDyW+D*2(Gyi?Qx-i}gy z&BpIO!NeVdLReO!YgdUfnT}7?5Z#~t5rMWqG+$N2n%5o#Np6ccNly}#IZQsW4?|NV zR9hrcyP(l#A+U4XcQvT;4{#i)dU>HK>aS!k1<3s2LyAhm2(!Nu%vRC9T`_yn9D+r} z1i&U~IcQ?4xhZYyH6WL-f%}qIhZkc&}n2N0PM| z6|XA9d-y;!`D{p;xu*gv7a|zaZ*MiQ)}zPzW4GB0mr)}N-DmB&hl1&x`2@sxN572_ zS)RdJyR%<7kW0v3Q_|57JKy&9tUdbqz}|hwn84}U*0r^jt6Ssrp+#1y=JBcZ+F`f(N?O0XL1OFGN`1-r?S<#t4*C9|y~e)!UYZ zRQ3M8m%~M)VriIvn~XzoP;5qeu(ZI>Y#r zAd)J)G9)*BeE%gmm&M@Olg3DI_zokjh9NvdGbT z+u4(Y&uC6tBBefIg~e=J#8i1Zxr>RT)#rGaB2C71usdsT=}mm`<#WY^6V{L*J6v&l z1^Tkr6-+^PA)yC;s1O^3Q!)Reb=fxs)P~I*?i&j{Vbb(Juc?La;cA5(H7#FKIj0Or zgV0BO{DUs`I9HgQ{-!g@5P^Vr|C4}~w6b=#`Zx0XcVSd?(04HUHwK(gJNafgQNB9Z zCi3TgNXAeJ+x|X|b@27$RxuYYuNSUBqo#uyiH6H(b~K*#!@g__4i%HP5wb<+Q7GSb zTZjJw96htUaGZ89$K_iBo4xEOJ#DT#KRu9ozu!GH0cqR>hP$nk=KXM%Y!(%vWQ#}s zy=O#BZ>xjUejMH^F39Bf0}>D}yiAh^toa-ts#gt6Mk9h1D<9_mGMBhLT0Ce2O3d_U znaTkBaxd-8XgwSp5)x-pqX5=+{cSuk6kyl@k|5DQ!5zLUVV%1X9vjY0gerbuG6nwZu5KDMdq(&UMLZ zy?jW#F6joUtVyz`Y?-#Yc0=i*htOFwQ3`hk$8oq35D}0m$FAOp#UFTV3|U3F>@N?d zeXLZCZjRC($%?dz(41e~)CN10qjh^1CdAcY(<=GMGk@`b1ptA&L*{L@_M{%Vd5b*x#b1(qh=7((<_l%ZUaHtmgq} zjchBdiis{Afxf@3CjPR09E*2#X(`W#-n`~6PcbaL_(^3tfDLk?Nb6CkW9v!v#&pWJ3iV-9hz zngp#Q`w`r~2wt&cQ9#S7z0CA^>Mzm7fpt72g<0y-KT{G~l-@L#edmjZQ}7{*$mLgSdJfS$Ge{hrD=mr;GD)uYq8}xS zT>(w_;}894Kb}(P5~FOpFIEjadhmxD(PsZbKwa-qxVa7Oc7~ebPKMeN(pCRzq8s@l z`|l^*X1eK1+Spz--WkSW_nK`Cs@JmkY4+p=U91nJoy{tSH;TzuIyS)Q_(S@;Iakua zpuDo5W54Mo;jY@Ly1dY)j|+M%$FJ0`C=FW#%UvOd&?p}0QqL20Xt!#pr8ujy6CA-2 zFz6Ex5H1i)c9&HUNwG{8K%FRK7HL$RJwvGakleLLo}tsb>t_nBCIuABNo$G--_j!gV&t8L^4N6wC|aLC)l&w04CD6Vc#h^(YH@Zs4nwUGkhc_-yt{dK zMZ<%$swLmUl8`E~RLihGt@J5v;r;vT&*Q!Cx zZ55-zpb;W7_Q{tf$mQvF61(K>kwTq0x{#Din||)B{+6O#ArLi)kiHWVC4`fOT&B(h zw&YV`J1|^FLx~9Q%r-SFhYl4PywI7sF2Q$>4o50~dfp5nn}XHv-_DM?RGs#+4gM;% znU>k=81G~f6u%^Z{bcX&sUv*h|L+|mNq=W43y@{~C zpL-TW3hYPs0^*OqS#KQwA^CGG_A-6#`_{1LBCD&*3nY0UHWJj1D|VP%oQlFxLllaA zVI@2^)HZ%E*=RbQcFOKIP7?+|_xVK+2oG(t_EGl2y;Ovox zZb^qVpe!4^reKvpIBFzx;Ji=PmrV>uu-Hb>`s?k?YZQ?>av45>i(w0V!|n?AP|v5H zm`e&Tgli#lqGEt?=(?~fy<(%#nDU`O@}Vjib6^rfE2xn;qgU6{u36j_+Km%v*2RLnGpsvS+THbZ>p(B zgb{QvqE?~50pkLP^0(`~K& zjT=2Pt2nSnwmnDFi2>;*C|OM1dY|CAZ5R|%SAuU|5KkjRM!LW_)LC*A zf{f>XaD+;rl6Y>Umr>M8y>lF+=nSxZX_-Z7lkTXyuZ(O6?UHw^q; z&$Zsm4U~}KLWz8>_{p*WQ!OgxT1JC&B&>|+LE3Z2mFNTUho<0u?@r^d=2 z-av!n8r#5M|F%l;=D=S1mGLjgFsiYAOODAR}#e^a8 zfVt$k=_o}kt3PTz?EpLkt54dY}kyd$rU zVqc9SN>0c z753j-gdN~UiW*FUDMOpYEkVzP)}{Ds*3_)ZBi)4v26MQr140|QRqhFoP=a|;C{#KS zD^9b-9HM11W+cb1Y)HAuk<^GUUo(ut!5kILBzAe)Vaxwu4Up!7Ql*#DDu z>EB84&xSrh>0jT!*X81jJQq$CRHqNj29!V3FN9DCx)~bvZbLwSlo3l^zPb1sqBnp) zfZpo|amY^H*I==3#8D%x3>zh#_SBf?r2QrD(Y@El!wa;Ja6G9Y1947P*DC|{9~nO& z*vDnnU!8(cV%HevsraF%Y%2{Z>CL0?64eu9r^t#WjW4~3uw8d}WHzsV%oq-T)Y z0-c!FWX5j1{1##?{aTeCW2b$PEnwe;t`VPCm@sQ`+$$L2=3kBR%2XU1{_|__XJ$xt zibjY2QlDVs)RgHH*kl&+jn*JqquF)k_Ypibo00lcc<2RYqsi-G%}k0r(N97H7JEn7@E3ZTH0JK>d8)E~A-D z!B&z9zJw0Bi^fgQZI%LirYaBKnWBXgc`An*qvO^*$xymqKOp(+3}IsnVhu?YnN7qz zNJxDN-JWd7-vIiv2M9ih>x3gNVY%DzzY~dCnA}76IRl!`VM=6=TYQ=o&uuE8kHqZT zoUNod0v+s9D)7aLJ|hVqL0li1hg)%&MAciI(4YJ=%D4H$fGQ&Lu-?@>>@pEgC;ERrL= zI^cS&3q8fvEGTJZgZwL5j&jp%j9U^Of6pR{wA^u=tVt#yCQepXNIbynGnuWbsC_EE zRyMFq{5DK692-*kyGy~An>AdVR9u___fzmmJ4;^s0yAGgO^h{YFmqJ%ZJ_^0BgCET zE6(B*SzeZ4pAxear^B-YW<%BK->X&Cr`g9_;qH~pCle# zdY|UB5cS<}DFRMO;&czbmV(?vzikf)Ks`d$LL801@HTP5@r><}$xp}+Ip`u_AZ~!K zT}{+R9Wkj}DtC=4QIqJok5(~0Ll&_6PPVQ`hZ+2iX1H{YjI8axG_Bw#QJy`6T>1Nn z%u^l`>XJ{^vX`L0 z1%w-ie!dE|!SP<>#c%ma9)8K4gm=!inHn2U+GR+~ zqZVoa!#aS0SP(|**WfQSe?cA=1|Jwk`UDsny%_y{@AV??N>xWekf>_IZLUEK3{Ksi zWWW$if&Go~@Oz)`#=6t_bNtD$d9FMBN#&97+XKa+K2C@I9xWgTE{?Xnhc9_KKPcujj@NprM@e|KtV_SR+ zSpeJ!1FGJ=Te6={;;+;a46-*DW*FjTnBfeuzI_=I1yk8M(}IwEIGWV0Y~wia;}^dg z{BK#G7^J`SE10z4(_Me=kF&4ld*}wpNs91%2Ute>Om`byv9qgK4VfwPj$`axsiZ)wxS4k4KTLb-d~!7I@^Jq`>?TrixHk|9 zqCX7@sWcVfNP8N;(T>>PJgsklQ#GF>F;fz_Rogh3r!dy*0qMr#>hvSua;$d z3TCZ4tlkyWPTD<=5&*bUck~J;oaIzSQ0E03_2x{?weax^jL3o`ZP#uvK{Z5^%H4b6 z%Kbp6K?>{;8>BnQy64Jy$~DN?l(ufkcs6TpaO&i~dC>0fvi-I^7YT#h?m;TVG|nba%CKRG%}3P*wejg) zI(ow&(5X3HR_xk{jrnkA-hbwxEQh|$CET9Qv6UpM+-bY?E!XVorBvHoU59;q<9$hK z%w5K-SK zWT#1OX__$ceoq0cRt>9|)v}$7{PlfwN}%Wh3rwSl;%JD|k~@IBMd5}JD#TOvp=S57 zae=J#0%+oH`-Av}a(Jqhd4h5~eG5ASOD)DfuqujI6p!;xF_GFcc;hZ9k^a7c%%h(J zhY;n&SyJWxju<+r`;pmAAWJmHDs{)V-x7(0-;E?I9FWK@Z6G+?7Py8uLc2~Fh1^0K zzC*V#P88(6U$XBjLmnahi2C!a+|4a)5Ho5>owQw$jaBm<)H2fR=-B*AI8G@@P-8I8 zHios92Q6Nk-n0;;c|WV$Q);Hu4;+y%C@3alP`cJ2{z~*m-@de%OKVgiWp;4Q)qf9n zJ!vmx(C=_>{+??w{U^Bh|LFJ<6t}Er<-Tu{C{dv8eb(kVQ4!fOuopTo!^x1OrG}0D zR{A#SrmN`=7T29bzQ}bwX8OUufW9d9T4>WY2n15=k3_rfGOp6sK0oj7(0xGaEe+-C zVuWa;hS*MB{^$=0`bWF(h|{}?53{5Wf!1M%YxVw}io4u-G2AYN|FdmhI13HvnoK zNS2fStm=?8ZpKt}v1@Dmz0FD(9pu}N@aDG3BY8y`O*xFsSz9f+Y({hFx;P_h>ER_& z`~{z?_vCNS>agYZI?ry*V96_uh;|EFc0*-x*`$f4A$*==p`TUVG;YDO+I4{gJGrj^ zn?ud(B4BlQr;NN?vaz_7{&(D9mfd z8esj=a4tR-ybJjCMtqV8>zn`r{0g$hwoWRUI3}X5=dofN){;vNoftEwX>2t@nUJro z#%7rpie2eH1sRa9i6TbBA4hLE8SBK@blOs=ouBvk{zFCYn4xY;v3QSM%y6?_+FGDn z4A;m)W?JL!gw^*tRx$gqmBXk&VU=Nh$gYp+Swu!h!+e(26(6*3Q!(!MsrMiLri`S= zKItik^R9g!0q7y$lh+L4zBc-?Fsm8`CX1+f>4GK7^X2#*H|oK}reQnT{Mm|0ar<+S zRc_dM%M?a3bC2ILD`|;6vKA`a3*N~(cjw~Xy`zhuY2s{(7KLB{S>QtR3NBQ3>vd+= z#}Q)AJr7Y_-eV(sMN#x!uGX08oE*g=grB*|bBs}%^3!RVA4f%m3=1f0K=T^}iI&2K zuM2GG5_%+#v-&V>?x4W9wQ|jE2Q7Be8mOyJtZrqn#gXy-1fF1P$C8+We&B*-pi#q5 zETp%H6g+%#sH+L4=ww?-h;MRCd2J9zwQUe4gHAbCbH08gDJY;F6F)HtWCRW1fLR;)ysGZanlz*a+|V&@(ipWdB!tz=m_0 z6F}`d$r%33bw?G*azn*}Z;UMr{z4d9j~s`0*foZkUPwpJsGgoR0aF>&@DC;$A&(av z?b|oo;`_jd>_5nye`DVOcMLr-*Nw&nA z82E8Dw^$Lpso)gEMh?N|Uc^X*NIhg=U%enuzZOGi-xcZRUZmkmq~(cP{S|*+A6P;Q zprIkJkIl51@ng)8cR6QSXJtoa$AzT@*(zN3M+6`BTO~ZMo0`9$s;pg0HE3C;&;D@q zd^0zcpT+jC%&=cYJF+j&uzX87d(gP9&kB9|-zN=69ymQS9_K@h3ph&wD5_!4q@qI@ zBMbd`2JJ2%yNX?`3(u&+nUUJLZ=|{t7^Rpw#v-pqD2_3}UEz!QazhRty%|Q~WCo7$ z+sIugHA%Lmm{lBP#bnu_>G}Ja<*6YOvSC;89z67M%iG0dagOt1HDpDn$<&H0DWxMU zxOYaaks6%R@{`l~zlZ*~2}n53mn2|O&gE+j*^ypbrtBv{xd~G(NF?Z%F3>S6+qcry z?ZdF9R*a;3lqX_!rI(Cov8ER_mOqSn6g&ZU(I|DHo7Jj`GJ}mF;T(vax`2+B8)H_D zD0I;%I?*oGD616DsC#j0x*p+ZpBfd=9gR|TvB)832CRhsW_7g&WI@zp@r7dhg}{+4f=(cO2s+)jg0x(*6|^+6W_=YIfSH0lTcK* z%)LyaOL6em@*-_u)}Swe8rU)~#zT-vNiW(D*~?Zp3NWl1y#fo!3sK-5Ek6F$F5l3| zrFFD~WHz1}WHmzzZ!n&O8rTgfytJG*7iE~0`0;HGXgWTgx@2fD`oodipOM*MOWN-} zJY-^>VMEi8v23ZlOn0NXp{7!QV3F1FY_URZjRKMcY(2PV_ms}EIC^x z=EYB5UUQ{@R~$2Mwiw$_JAcF+szKB*n(`MYpDCl>~ss54uDQ%Xf-8|dgO zY)B_qju=IaShS|XsQo=nSYxV$_vQR@hd~;qW)TEfU|BA0&-JSwO}-a*T;^}l;MgLM zz}CjPlJX|W2vCzm3oHw3vqsRc3RY=2()}iw_k2#eKf&VEP7TQ;(DDzEAUgj!z_h2Br;Z3u=K~LqM6YOrlh)v9`!n|6M-s z?XvA~y<5?WJ{+yM~uPh7uVM&g-(;IC3>uA}ud?B3F zelSyc)Nx>(?F=H88O&_70%{ATsLVTAp88F-`+|egQ7C4rpIgOf;1tU1au+D3 zlz?k$jJtTOrl&B2%}D}8d=+$NINOZjY$lb{O<;oT<zXoAp01KYG$Y4*=)!&4g|FL(!54OhR-?)DXC&VS5E|1HGk8LY;)FRJqnz zb_rV2F7=BGwHgDK&4J3{%&IK~rQx<&Kea|qEre;%A~5YD6x`mo>mdR)l?Nd%T2(5U z_ciT02-zt_*C|vn?BYDuqSFrk3R(4B0M@CRFmG{5sovIq4%8AhjXA5UwRGo)MxZlI zI%vz`v8B+#ff*XtGnciczFG}l(I}{YuCco#2E6|+5WJ|>BSDfz0oT+F z%QI^ixD|^(AN`MS6J$ zXlKNTFhb>KDkJp*4*LaZ2WWA5YR~{`={F^hwXGG*rJYQA7kx|nwnC58!eogSIvy{F zm1C#9@$LhK^Tl>&iM0wsnbG7Y^MnQ=q))MgApj4)DQt!Q5S`h+5a%c7M!m%)?+h65 z0NHDiEM^`W+M4)=q^#sk(g!GTpB}edwIe>FJQ+jAbCo#b zXmtd3raGJNH8vnqMtjem<_)9`gU_-RF&ZK!aIenv7B2Y0rZhon=2yh&VsHzM|`y|0x$Zez$bUg5Nqj?@~^ zPN43MB}q0kF&^=#3C;2T*bDBTyO(+#nZnULkVy0JcGJ36or7yl1wt7HI_>V7>mdud zv2II9P61FyEXZuF$=69dn%Z6F;SOwyGL4D5mKfW)q4l$8yUhv7|>>h_-4T*_CwAyu7;DW}_H zo>N_7Gm6eed=UaiEp_7aZko@CC61@(E1be&5I9TUq%AOJW>s^9w%pR5g2{7HW9qyF zh+ZvX;5}PN0!B4q2FUy+C#w5J?0Tkd&S#~94(AP4%fRb^742pgH7Tb1))siXWXHUT z1Wn5CG&!mGtr#jq6(P#!ck@K+FNprcWP?^wA2>mHA03W?kj>5b|P0ErXS) zg2qDTjQ|grCgYhrH-RapWCvMq5vCaF?{R%*mu}1)UDll~6;}3Q*^QOfj!dlt02lSzK z?+P)02Rrq``NbU3j&s*;<%i4Y>y9NK&=&KsYwvEmf5jwTG6?+Pu1q9M8lLlx)uZZ7 zizhr~e0ktGs-=$li-2jz^_48-jk**y&5u0`B2gc#i$T1~t+AS*kEfR*b{^Ec>2-F~ zKYRl&uQ5yO@EtAZX8ZSqx;8+AKf+CqhlUSpp*VfyBMv+%wxN5GukZEi^_to%MFRc0 zdXqJ*jk?#uYT6EJe446@(f6G4vhnxQP|pGeJ?-#|Ksq?g*ky=}x+Qnx+!<>Y(XStN zQIND`{KU}&l)E*ntI^}kJ=ly8DML{!(58Xk4_bzIc@v~e;>wKl_`7G%pGz~4KH*CTp;_|52)d!+ximd$|8v@zzEq%j68QXkgf$7eM~xdM5q5i z{?qFx_W|eq@L03bWJfjy^z@()-iCjzjREuf zb_a(yTz)ZKWCF%Lp>^2-%Q?*t{06}x#DLN3cO=i>h6#-a`z;<5rBGGM6GA(WqvRcX%Pn?Uvs1#e|ePSNJEC%+X(YI$x)`s$%>O#%}D9dgqWfq4yfVz^%FglokdFR}uJQhx|}_w`9Ulx38Ha>ZslKs58c-@IFI&f;?xM zbK>rKNfPFsf>%+k6%(A6=7Aac^_qrOCNqb3ZVJ;8pt!?1DR*ynJb#@II9h?)xB)A~ zm9Kk)Hy}!Z+W}i6ZJDy+?yY_=#kWrzgV)2eZAx_E=}Nh7*#<&mQz`Umfe$+l^P(xd zN}PA2qII4}ddCU+PN+yxkH%y!Qe(;iH3W%bwM3NKbU_saBo<8x9fGNtTAc_SizU=o zC3n2;c%LoU^j90Sz>B_p--Fzqv7x7*?|~-x{haH8RP)p|^u$}S9pD-}5;88pu0J~9 zj}EC`Q^Fw}`^pvAs4qOIuxKvGN@DUdRQ8p-RXh=3S#<`3{+Qv6&nEm)uV|kRVnu6f zco{(rJaWw(T0PWim?kkj9pJ)ZsUk9)dSNLDHf`y&@wbd;_ita>6RXFJ+8XC*-wsiN z(HR|9IF283fn=DI#3Ze&#y3yS5;!yoIBAH(v}3p5_Zr+F99*%+)cp!Sy8e+lG?dOc zuEz<;3X9Z5kkpL_ZYQa`sioR_@_cG z8tT~GOSTWnO~#?$u)AcaBSaV7P~RT?Nn8(OSL1RmzPWRWQ$K2`6*)+&7^zZBeWzud z*xb3|Fc~|R9eH+lQ#4wF#c;)Gka6lL(63C;>(bZob!i8F-3EhYU3|6-JBC0*5`y0| zBs!Frs=s!Sy0qmQNgIH|F`6(SrD1js2prni_QbG9Sv@^Pu2szR9NZl8GU89gWWvVg z2^-b*t+F{Nt>v?js7hnlC`tRU(an0qQG7;h6T~ z-`vf#R-AE$pzk`M{gCaia}F`->O2)60AuGFAJg> z*O2IZqTx=AzDvC49?A92>bQLdb&32_4>0Bgp0ESXXnd4B)!$t$g{*FG%HYdt3b3a^J9#so%BJMyr2 z{y?rzW!>lr097b9(75#&4&@lkB1vT*w&0E>!dS+a|ZOu6t^zro2tiP)bhcNNxn zbJs3_Fz+?t;4bkd8GfDI7ccJ5zU`Bs~ zN~bci`c`a%DoCMel<-KUCBdZRmew`MbZEPYE|R#|*hhvhyhOL#9Yt7$g_)!X?fK^F z8UDz)(zpsvriJ5aro5>qy`Fnz%;IR$@Kg3Z3EE!fv9CAdrAym6QU82=_$_N5*({_1 z7!-=zy(R{xg9S519S6W{HpJZ8Is|kQ!0?`!vxDggmslD59)>iQ15f z7J8NqdR`9f8H|~iFGNsPV!N)(CC9JRmzL9S}7U-K@`X893f3f<8|8Ls!^eA^#(O6nA+ByFIXcz_WLbfeG|nHJ5_sJJ^gNJ%SI9#XEfNRbzV+!RkI zXS$MOVYb2!0vU}Gt7oUy*|WpF^*orBot~b2J@^be?Gq;U%#am8`PmH-UCFZ&uTJlnetYij0z{K1mmivk$bdPbLodu;-R@@#gAV!=d%(caz$E?r zURX0pqAn7UuF6dULnoF1dZ$WM)tHAM{eZK6DbU1J`V5Dw<;xk}Nl`h+nfMO_Rdv z3SyOMzAbYaD;mkxA7_I_DOs#Bk;e5D%gsS3q)hlmi1w{FsjKNJE22`AjmNiAPRnIc zcIkN25;rOn3FipAFd(PnlK9{03w6Q<(68#1Jw`{axEGQE{Ac>^U$h);h2ADICmaNxrfpb`Jdr*)Y1SicpYKCFv$3vf~;5aW>n^7QGa63MJ z;B1+Z>WQ615R2D8JmmT`T{QcgZ+Kz1hTu{9FOL}Q8+iFx-Vyi}ZVVcGjTe>QfA`7W zFoS__+;E_rQIQxd(Bq4$egKeKsk#-9=&A!)(|hBvydsr5ts0Zjp*%*C0lM2sIOx1s zg$xz?Fh?x!P^!vWa|}^+SY8oZHub7f;E!S&Q;F?dZmvBxuFEISC}$^B_x*N-xRRJh zn4W*ThEWaPD*$KBr8_?}XRhHY7h^U1aN6>m=n~?YJQd8+!Uyq_3^)~4>XjelM&!c9 zCo|0KsGq7!KsZ~9@%G?i>LaU7#uSTMpypocm*oqJHR|wOgVWc7_8PVuuw>x{kEG4T z$p^DV`}jUK39zqFc(d5;N+M!Zd3zhZN&?Ww(<@AV-&f!v$uV>%z+dg9((35o@4rqLvTC-se@hkn^6k7+xHiK-vTRvM8{bCejbU;1@U=*r}GTI?Oc$!b6NRcj83-zF; z=TB#ESDB`F`jf4)z=OS76Se}tQDDHh{VKJk#Ad6FDB_=afpK#pyRkGrk~OuzmQG)} z*$t!nZu$KN&B;|O-aD=H<|n6aGGJZ=K9QFLG0y=Jye_ElJFNZJT;fU8P8CZcLBERjioAOC0Vz_pIXIc};)8HjfPwNy zE!g|lkRv3qpmU?shz(BBt5%TbpJC3HzP9!t7k*Fh48!-HlJ4TTgdCr3rCU!iF}kgu z4Qs;K@XOY~4f~N}Jl8V_mGbwzvNLbl&0e9UG4W;kvjTK|5`-Ld+eQ6YRF`N0ct%u% z^3J_{7r#_W1zm|>IPN!yWCRrN)N!7v`~ptNkIXKipQ6ogFvcnI5ugxdoa{d;uD67g zgo^}QuZRkB540Vc!@c80(wFG=$ct}oHq(#W0+-XX(;Rrt`x=<45X}ficNtI2(&}=~ zb(!}tNz?s`wm{gK?2tdf+OEF;tzx<(3fMd7_tM@Ghs$Z(Os-H(kYq#qB|J-aC9Ku?fsWwJhB36c)A zu|a7ZF?V8X7l2g5~xqZf>2=6Dsi5lfo zKIRL&@MLJyaBE)V_9=pJYu%U2wxR*-(0MI5_|yqP`?h@cks(5LR@XUKLMI_xuVtiu zRvpDS8MyUMRFM6`P+Sjc!A_e^H38Qu7b{b7QZ>NHyA6k-YYygQuW&C_OGO(7V7?}r)zedSVpBI zuk29Z4GW3C0GpfozbZQya454sjt@ndQmsp=DA&@sWw&xmOlDk1JIcMNp~-ES$&A~k zG#W(6hBj?!Fu8Q4WYexoSBa8_5=v20xnx6H?e;$t)5|f&{7=vOye^&3_c-Ug?|a@e z=X`&qT_5B7N9vZoPBhXOTEDV;4&x2Je4}T(UB~O-$D#CjX77$R?RZ*`ed~$G;$4YS z4n*|Pop(!NN79Hk2}U#cfEEwdxM)xQm}$~rV03xc=#U@@Y*}qEmot5KvDb=8{!E-n zl4p?}&g2h^sUGyTcGh=0aQzQb*k;K;dvbeZUgmwEv>%#(EPtj=gHKdi|E8@w+|>KC zxEU>b>P+9Xf}pEyQK(}#QrBG4Jaf!iE!qpMbTu>gb!gtdq<`@xO+roQl+S_7)!G(% zdy)$iGmJ1cwP?F=IyyV1-$|kf|EKM3B@I&lZ%NI@VV;*mQdLWjc#t|Vbk_Q~>&O03 zIcSr$(qLAINj7a z;!||v&1D5SX#X@5jNd}jUsi-CH_Scjyht&}q2p*CJCC-`&NyXf)vD5{e!HO629D-O z%bZelTcq=DoRX>zeWCa^RmR3*{x9;3lZ75M#S)!W0bRIFH#P6b%{|HRSZ5!!I#s)W z_|XXZQ<0_`>b^^0Z>LU64Yg1w)8}#M^9se(OZ9~baZ7fsKFc;EtnB>kesci#>=icG zuHdjax2^=!_(9?0l7;G7^-}9>Y#M zm;9*GT~dBuYWdk49%mZM0=H#FY1)}7NE5DE_vsqrA0`?0R0q535qHjWXcl|gz9Fq$ zMKxgL;68l!gm3y0durIr3LHv~y*ABm` zYhQG0UW#hg@*A{&G!;$FS43}rIF$e6yRdGJWVR<}uuJ_5_8qa3xaHH^!VzUteVp;> z<0`M>3tnY$ZFb$(`0sg93TwGyP;`9UYUWxO&CvAnSzei&ap))NcW;R`tA=y^?mBmG+M*&bqW5kL$V(O;(p)aEk`^ci?2Jwxu>0sy>a7+Wa9t z5#I2o;+gr^9^&km^z7>xJWbN&Ft>Vna34E zI@BBzwX)R}K3SL?)enrDJ45QLt;-7CFJk{`cF3L4Z^CtG_r5)0)HV>BOYPIUh#D%| zYQAu31f{bm-D*`_k7DTTr?Nkw_gY%J1cb2&TdtibY?V=|SSIOlA;|5C!2@?YQ z-$?G0jj^mG|MP>DmbF7}T~C$H6=CpZ~hd zZ1C|xV@=h#^~`3LSCnmI(vZ|5r3>eq5*UB)dhdy``*gKY3Eg%jSK8I-`G+OWWlD)T zt$wSQ=||lSkiKy}YF-k}@W9EiS?)z`hK{R!dd-$BCJvBtAN-yXn3njU$MisEtp!?Q z%Vk-*(wy9dd15(-WFw_&^tT;;IpF?ox1`Qq3-0zVTk+$W_?q}GfAQlPcrB^?&tWSI z2BB!K=sH7FUYmXa_dcV^Z3>5z8}~W{S!$jVR_3hu_|wl2|gmRH8ftn^z@fW75*;-`;wU+fY+BR_yx6BZnE5_Hna({jrPiubRp$jZ=T=t$hx&NeCV1!vuCcl4PJ0p0Fjp>6K} zHkoD1gQk=P2hYcT%)cJ2Q5WuA|5_x+dX0%hnozfTF>$#Wz~X!MY>){H4#fB#7^ID* z1*o2Hzp}?WVs&gbS?Uq(CT0sP+F)u9{xfgg6o_{8J#m;|NeJqDHhb(Q8%z8aM_qeM zn83>d`uDd47WIuKp78JBYo2SYupGcNXIzeou^eMY`@%Bv8elZ>q~3uq#~IX)g%g;h zoUXymEd>|kVsMkyb&1l~lrE-`w(0PObapYa35DJ4Y03Jv_!DKp}0HTbOgZRM=;PSsuAJJJ1 zItc+tu9;ANG;qHaCI|T85!euhFK~VK^G2LZV1+cbzS?>ar@>emg;JTI5VAn1g5U~| zU=p&k0OlSzc$U=s#9_uL3&n|6A1X$XvrE9vFV@`A4G#!D1QcFCeE`F2N(deJx>)*A z$XIW0P~-NbAd=5i6`s<~(vAQX9t$dbVqc5|E|CHRtb$1(l&KSNh_t2#k_l95KnP86 z)ns_DGspv-M0z0#h2a+*oH|{5~j{ zXGD=}cLrBSESQ0u$XmQlFfWMCAWaS;wKK%#aSSYK=qljBiY(s zT$v;We24&$w=avIILsMt0%1fDyah|AlLNg#WL$Lu)tf}YfqO%+pH~QC*bZO4aM*i9 zrPFf|5!hv@XY8CzaFh*Dy9vH|2fKKr(@x}`L#9^*vOae|lk`adG#oZZAyk|TOV8`9L zc-sQu%y1MQes&J?)a1}Zc*>-P!6j-T#75V$lLC!TuMB(!G-+D2;XptUxymSPFI-K&0x}B1?h$ z3-9**-9!);fwyiWB5gS$i;P~c=^}5-6G@{4TWDBRDc6(M|%qa-mS`z`u9kWo{Xl_uc;hXOkRd literal 0 HcmV?d00001 diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties new file mode 100644 index 0000000..bb8b2fc --- /dev/null +++ b/gradle/wrapper/gradle-wrapper.properties @@ -0,0 +1,5 @@ +distributionBase=GRADLE_USER_HOME +distributionPath=wrapper/dists +distributionUrl=https\://services.gradle.org/distributions/gradle-6.5.1-bin.zip +zipStoreBase=GRADLE_USER_HOME +zipStorePath=wrapper/dists diff --git a/gradlew b/gradlew new file mode 100755 index 0000000..fbd7c51 --- /dev/null +++ b/gradlew @@ -0,0 +1,185 @@ +#!/usr/bin/env sh + +# +# Copyright 2015 the original author or authors. +# +# 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 +# +# https://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. +# + +############################################################################## +## +## Gradle start up script for UN*X +## +############################################################################## + +# Attempt to set APP_HOME +# Resolve links: $0 may be a link +PRG="$0" +# Need this for relative symlinks. +while [ -h "$PRG" ] ; do + ls=`ls -ld "$PRG"` + link=`expr "$ls" : '.*-> \(.*\)$'` + if expr "$link" : '/.*' > /dev/null; then + PRG="$link" + else + PRG=`dirname "$PRG"`"/$link" + fi +done +SAVED="`pwd`" +cd "`dirname \"$PRG\"`/" >/dev/null +APP_HOME="`pwd -P`" +cd "$SAVED" >/dev/null + +APP_NAME="Gradle" +APP_BASE_NAME=`basename "$0"` + +# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. +DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"' + +# Use the maximum available, or set MAX_FD != -1 to use that value. +MAX_FD="maximum" + +warn () { + echo "$*" +} + +die () { + echo + echo "$*" + echo + exit 1 +} + +# OS specific support (must be 'true' or 'false'). +cygwin=false +msys=false +darwin=false +nonstop=false +case "`uname`" in + CYGWIN* ) + cygwin=true + ;; + Darwin* ) + darwin=true + ;; + MINGW* ) + msys=true + ;; + NONSTOP* ) + nonstop=true + ;; +esac + +CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar + + +# Determine the Java command to use to start the JVM. +if [ -n "$JAVA_HOME" ] ; then + if [ -x "$JAVA_HOME/jre/sh/java" ] ; then + # IBM's JDK on AIX uses strange locations for the executables + JAVACMD="$JAVA_HOME/jre/sh/java" + else + JAVACMD="$JAVA_HOME/bin/java" + fi + if [ ! -x "$JAVACMD" ] ; then + die "ERROR: JAVA_HOME is set to an invalid directory: $JAVA_HOME + +Please set the JAVA_HOME variable in your environment to match the +location of your Java installation." + fi +else + JAVACMD="java" + which java >/dev/null 2>&1 || die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. + +Please set the JAVA_HOME variable in your environment to match the +location of your Java installation." +fi + +# Increase the maximum file descriptors if we can. +if [ "$cygwin" = "false" -a "$darwin" = "false" -a "$nonstop" = "false" ] ; then + MAX_FD_LIMIT=`ulimit -H -n` + if [ $? -eq 0 ] ; then + if [ "$MAX_FD" = "maximum" -o "$MAX_FD" = "max" ] ; then + MAX_FD="$MAX_FD_LIMIT" + fi + ulimit -n $MAX_FD + if [ $? -ne 0 ] ; then + warn "Could not set maximum file descriptor limit: $MAX_FD" + fi + else + warn "Could not query maximum file descriptor limit: $MAX_FD_LIMIT" + fi +fi + +# For Darwin, add options to specify how the application appears in the dock +if $darwin; then + GRADLE_OPTS="$GRADLE_OPTS \"-Xdock:name=$APP_NAME\" \"-Xdock:icon=$APP_HOME/media/gradle.icns\"" +fi + +# For Cygwin or MSYS, switch paths to Windows format before running java +if [ "$cygwin" = "true" -o "$msys" = "true" ] ; then + APP_HOME=`cygpath --path --mixed "$APP_HOME"` + CLASSPATH=`cygpath --path --mixed "$CLASSPATH"` + + JAVACMD=`cygpath --unix "$JAVACMD"` + + # We build the pattern for arguments to be converted via cygpath + ROOTDIRSRAW=`find -L / -maxdepth 1 -mindepth 1 -type d 2>/dev/null` + SEP="" + for dir in $ROOTDIRSRAW ; do + ROOTDIRS="$ROOTDIRS$SEP$dir" + SEP="|" + done + OURCYGPATTERN="(^($ROOTDIRS))" + # Add a user-defined pattern to the cygpath arguments + if [ "$GRADLE_CYGPATTERN" != "" ] ; then + OURCYGPATTERN="$OURCYGPATTERN|($GRADLE_CYGPATTERN)" + fi + # Now convert the arguments - kludge to limit ourselves to /bin/sh + i=0 + for arg in "$@" ; do + CHECK=`echo "$arg"|egrep -c "$OURCYGPATTERN" -` + CHECK2=`echo "$arg"|egrep -c "^-"` ### Determine if an option + + if [ $CHECK -ne 0 ] && [ $CHECK2 -eq 0 ] ; then ### Added a condition + eval `echo args$i`=`cygpath --path --ignore --mixed "$arg"` + else + eval `echo args$i`="\"$arg\"" + fi + i=`expr $i + 1` + done + case $i in + 0) set -- ;; + 1) set -- "$args0" ;; + 2) set -- "$args0" "$args1" ;; + 3) set -- "$args0" "$args1" "$args2" ;; + 4) set -- "$args0" "$args1" "$args2" "$args3" ;; + 5) set -- "$args0" "$args1" "$args2" "$args3" "$args4" ;; + 6) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" ;; + 7) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" ;; + 8) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" ;; + 9) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" "$args8" ;; + esac +fi + +# Escape application args +save () { + for i do printf %s\\n "$i" | sed "s/'/'\\\\''/g;1s/^/'/;\$s/\$/' \\\\/" ; done + echo " " +} +APP_ARGS=`save "$@"` + +# Collect all arguments for the java command, following the shell quoting and substitution rules +eval set -- $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS "\"-Dorg.gradle.appname=$APP_BASE_NAME\"" -classpath "\"$CLASSPATH\"" org.gradle.wrapper.GradleWrapperMain "$APP_ARGS" + +exec "$JAVACMD" "$@" diff --git a/gradlew.bat b/gradlew.bat new file mode 100644 index 0000000..a9f778a --- /dev/null +++ b/gradlew.bat @@ -0,0 +1,104 @@ +@rem +@rem Copyright 2015 the original author or authors. +@rem +@rem Licensed under the Apache License, Version 2.0 (the "License"); +@rem you may not use this file except in compliance with the License. +@rem You may obtain a copy of the License at +@rem +@rem https://www.apache.org/licenses/LICENSE-2.0 +@rem +@rem Unless required by applicable law or agreed to in writing, software +@rem distributed under the License is distributed on an "AS IS" BASIS, +@rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +@rem See the License for the specific language governing permissions and +@rem limitations under the License. +@rem + +@if "%DEBUG%" == "" @echo off +@rem ########################################################################## +@rem +@rem Gradle startup script for Windows +@rem +@rem ########################################################################## + +@rem Set local scope for the variables with windows NT shell +if "%OS%"=="Windows_NT" setlocal + +set DIRNAME=%~dp0 +if "%DIRNAME%" == "" set DIRNAME=. +set APP_BASE_NAME=%~n0 +set APP_HOME=%DIRNAME% + +@rem Resolve any "." and ".." in APP_HOME to make it shorter. +for %%i in ("%APP_HOME%") do set APP_HOME=%%~fi + +@rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. +set DEFAULT_JVM_OPTS="-Xmx64m" "-Xms64m" + +@rem Find java.exe +if defined JAVA_HOME goto findJavaFromJavaHome + +set JAVA_EXE=java.exe +%JAVA_EXE% -version >NUL 2>&1 +if "%ERRORLEVEL%" == "0" goto init + +echo. +echo ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. +echo. +echo Please set the JAVA_HOME variable in your environment to match the +echo location of your Java installation. + +goto fail + +:findJavaFromJavaHome +set JAVA_HOME=%JAVA_HOME:"=% +set JAVA_EXE=%JAVA_HOME%/bin/java.exe + +if exist "%JAVA_EXE%" goto init + +echo. +echo ERROR: JAVA_HOME is set to an invalid directory: %JAVA_HOME% +echo. +echo Please set the JAVA_HOME variable in your environment to match the +echo location of your Java installation. + +goto fail + +:init +@rem Get command-line arguments, handling Windows variants + +if not "%OS%" == "Windows_NT" goto win9xME_args + +:win9xME_args +@rem Slurp the command line arguments. +set CMD_LINE_ARGS= +set _SKIP=2 + +:win9xME_args_slurp +if "x%~1" == "x" goto execute + +set CMD_LINE_ARGS=%* + +:execute +@rem Setup the command line + +set CLASSPATH=%APP_HOME%\gradle\wrapper\gradle-wrapper.jar + + +@rem Execute Gradle +"%JAVA_EXE%" %DEFAULT_JVM_OPTS% %JAVA_OPTS% %GRADLE_OPTS% "-Dorg.gradle.appname=%APP_BASE_NAME%" -classpath "%CLASSPATH%" org.gradle.wrapper.GradleWrapperMain %CMD_LINE_ARGS% + +:end +@rem End local scope for the variables with windows NT shell +if "%ERRORLEVEL%"=="0" goto mainEnd + +:fail +rem Set variable GRADLE_EXIT_CONSOLE if you need the _script_ return code instead of +rem the _cmd.exe /c_ return code! +if not "" == "%GRADLE_EXIT_CONSOLE%" exit 1 +exit /b 1 + +:mainEnd +if "%OS%"=="Windows_NT" endlocal + +:omega diff --git a/settings.gradle b/settings.gradle new file mode 100644 index 0000000..7ef2f4b --- /dev/null +++ b/settings.gradle @@ -0,0 +1,5 @@ +/* + * This file was generated by the Gradle 'init' task. + */ + +rootProject.name = 'pravega' diff --git a/src/checkstyle/presto-checks.xml b/src/checkstyle/presto-checks.xml new file mode 100644 index 0000000..96b8e42 --- /dev/null +++ b/src/checkstyle/presto-checks.xml @@ -0,0 +1,219 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/src/main/java/com/facebook/presto/pravega/ClassLoaderSafeRecordSetProvider.java b/src/main/java/com/facebook/presto/pravega/ClassLoaderSafeRecordSetProvider.java new file mode 100644 index 0000000..e989994 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/ClassLoaderSafeRecordSetProvider.java @@ -0,0 +1,47 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.RecordSet; +import com.facebook.presto.spi.classloader.ThreadContextClassLoader; +import com.facebook.presto.spi.connector.ConnectorRecordSetProvider; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public class ClassLoaderSafeRecordSetProvider + implements ConnectorRecordSetProvider +{ + private final ConnectorRecordSetProvider delegate; + private final ClassLoader classLoader; + + public ClassLoaderSafeRecordSetProvider(ConnectorRecordSetProvider delegate, ClassLoader classLoader) + { + this.delegate = requireNonNull(delegate, "delegate is null"); + this.classLoader = requireNonNull(classLoader, "classLoader is null"); + } + + @Override + public RecordSet getRecordSet(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorSplit split, List columns) + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + return delegate.getRecordSet(transactionHandle, session, split, columns); + } + } +} diff --git a/src/main/java/com/facebook/presto/pravega/DelimRecordValue.java b/src/main/java/com/facebook/presto/pravega/DelimRecordValue.java new file mode 100644 index 0000000..91f666e --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/DelimRecordValue.java @@ -0,0 +1,115 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.google.common.base.Preconditions; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; + +import java.nio.ByteBuffer; + +// csv, tsv, etc +public class DelimRecordValue + implements PravegaRecordValue +{ + int positions; + int[] position; + + char fieldSep; + + boolean decoded; + + private ByteBuffer buf; + + private int len; + private int offset; + + public DelimRecordValue(char fieldSep) + { + this.fieldSep = fieldSep; + this.position = new int[1024]; + } + + public void setBuf(ByteBuffer buf) + { + this.buf = buf; + this.len = buf.limit(); + this.offset = buf.arrayOffset(); + this.decoded = false; + } + + public void decode() + { + position[0] = 0; + positions = 1; + + int pos = 0; + while (pos < len) { + if (buf.get(pos++) == fieldSep) { + position[positions++] = pos; + } + } + + decoded = true; + } + + public boolean decoded() + { + return decoded; + } + + private int fieldLen(int ordinalPosition) + { + return ordinalPosition + 1 == positions + ? len - position[ordinalPosition] + : position[ordinalPosition + 1] - position[ordinalPosition] - 1; + } + + private String toString(int ordinalPosition) + { + return new String(buf.array(), offset + position[ordinalPosition], fieldLen(ordinalPosition)); + } + + public boolean isNull(int unused, int ordinalPosition) + { + if (!decoded) { + decode(); + } + return ordinalPosition >= positions || fieldLen(ordinalPosition) == 0; + } + + public long getLong(int unused, int ordinalPosition) + { + Preconditions.checkState(decoded); + return Long.parseLong(toString(ordinalPosition)); + } + + public double getDouble(int unused, int ordinalPosition) + { + Preconditions.checkState(decoded); + return Double.parseDouble(toString(ordinalPosition)); + } + + public boolean getBoolean(int unused, int ordinalPosition) + { + Preconditions.checkState(decoded); + return Boolean.getBoolean(toString(ordinalPosition)); + } + + public Slice getSlice(int unused, int ordinalPosition) + { + Preconditions.checkState(decoded); + return Slices.wrappedBuffer(buf.array(), offset + position[ordinalPosition], fieldLen(ordinalPosition)); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/EventStreamIterator.java b/src/main/java/com/facebook/presto/pravega/EventStreamIterator.java new file mode 100644 index 0000000..7105c75 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/EventStreamIterator.java @@ -0,0 +1,103 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.airlift.log.Logger; +import com.facebook.presto.pravega.decoder.BytesEvent; +import com.facebook.presto.pravega.decoder.DecodableEvent; +import io.pravega.client.stream.EventRead; +import io.pravega.client.stream.EventStreamReader; +import io.pravega.client.stream.ReaderConfig; +import io.pravega.client.stream.ReaderGroupConfig; +import io.pravega.client.stream.impl.ByteBufferSerializer; + +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.UUID; + +import static com.facebook.presto.pravega.util.PravegaNameUtils.scopedName; + +public class EventStreamIterator + implements Iterator +{ + private static final Logger log = Logger.get(EventStreamIterator.class); + + private final PravegaSegmentManager segmentManager; + private final ReaderArgs readerArgs; + private EventStreamReader reader; + private final long readTimeoutMs; + + private ByteBuffer event; + + public EventStreamIterator(PravegaSegmentManager segmentManager, ReaderArgs readerArgs, PravegaProperties properties) + { + this.segmentManager = segmentManager; + this.readerArgs = readerArgs; + this.readTimeoutMs = properties.getEventReadTimeoutMs(); + } + + private void init() + { + log.info("open iterator for stream " + readerArgs); + String readerGroupName = readerArgs.getReaderGroup(); + if (readerArgs.getReaderGroup() == null) { + readerGroupName = "reader-group-" + UUID.randomUUID().toString(); + ReaderGroupConfig config = + ReaderGroupConfig.builder() + .stream(scopedName(readerArgs.getScope(), readerArgs.getStream()), + readerArgs.getStreamCutRange().getStart(), + readerArgs.getStreamCutRange().getEnd()) + .build(); + log.info("create reader group " + readerGroupName); + segmentManager.readerGroupManager( + readerArgs.getScope()).createReaderGroup(readerGroupName, config); + } + + String readerId = UUID.randomUUID().toString(); + log.info("create reader " + readerId); + reader = segmentManager.getEventStreamClientFactory(readerArgs.getScope()) + .createReader(readerId, + readerGroupName, + new ByteBufferSerializer(), + ReaderConfig.builder().build()); + } + + private boolean _next() + { + if (reader == null) { + init(); + } + + EventRead read; + do { + read = reader.readNextEvent(readTimeoutMs); + } while (read.isCheckpoint()); + event = read.getEvent(); + return event != null; + } + + @Override + public boolean hasNext() + { + return event != null || _next(); + } + + @Override + public DecodableEvent next() + { + BytesEvent bytesEvent = new BytesEvent(event); + event = null; + return bytesEvent; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/ObjectType.java b/src/main/java/com/facebook/presto/pravega/ObjectType.java new file mode 100644 index 0000000..f54c1de --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/ObjectType.java @@ -0,0 +1,20 @@ +/* + * 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 com.facebook.presto.pravega; + +public enum ObjectType +{ + STREAM, + KV_TABLE, +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaColumnHandle.java b/src/main/java/com/facebook/presto/pravega/PravegaColumnHandle.java new file mode 100644 index 0000000..0f4d136 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaColumnHandle.java @@ -0,0 +1,237 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.common.type.Type; +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.spi.ColumnMetadata; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +/** + * Pravega specific connector column handle. + */ +public final class PravegaColumnHandle + implements DecoderColumnHandle, Comparable +{ + private final String connectorId; + private final int ordinalPosition; + + /** + * Column Name + */ + private final String name; + + /** + * Column type + */ + private final Type type; + + /** + * Mapping hint for the decoder. Can be null. + */ + private final String mapping; + + /** + * Data format to use (selects the decoder). Can be null. + */ + private final String dataFormat; + + /** + * Additional format hint for the selected decoder. Selects a decoder subtype (e.g. which timestamp decoder). + */ + private final String formatHint; + + /** + * True if the key decoder should be used, false if the message decoder should be used. + */ + private final boolean keyDecoder; + + /** + * True if the column should be hidden. + */ + private final boolean hidden; + + /** + * True if the column is internal to the connector and not defined by a stream definition. + */ + private final boolean internal; + + /** + * table may have multiple schemas (kv table, 1 for key + 1 for value) + * to which does this column belong + */ + private final int schemaNum; + + @JsonCreator + public PravegaColumnHandle( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("ordinalPosition") int ordinalPosition, + @JsonProperty("name") String name, + @JsonProperty("type") Type type, + @JsonProperty("mapping") String mapping, + @JsonProperty("dataFormat") String dataFormat, + @JsonProperty("formatHint") String formatHint, + @JsonProperty("keyDecoder") boolean keyDecoder, + @JsonProperty("hidden") boolean hidden, + @JsonProperty("internal") boolean internal, + @JsonProperty("schemaNum") int schemaNum) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null"); + this.ordinalPosition = ordinalPosition; + this.name = requireNonNull(name, "name is null"); + this.type = requireNonNull(type, "type is null"); + this.mapping = mapping; + this.dataFormat = dataFormat; + this.formatHint = formatHint; + this.keyDecoder = keyDecoder; + this.hidden = hidden; + this.internal = internal; + this.schemaNum = schemaNum; + } + + @JsonProperty + public String getConnectorId() + { + return connectorId; + } + + @JsonProperty + public int getOrdinalPosition() + { + return ordinalPosition; + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @Override + @JsonProperty + public Type getType() + { + return type; + } + + @Override + @JsonProperty + public String getMapping() + { + return mapping; + } + + @Override + @JsonProperty + public String getDataFormat() + { + return dataFormat; + } + + @Override + @JsonProperty + public String getFormatHint() + { + return formatHint; + } + + @JsonProperty + public boolean isKeyDecoder() + { + return keyDecoder; + } + + @JsonProperty + public boolean isHidden() + { + return hidden; + } + + @Override + @JsonProperty + public boolean isInternal() + { + return internal; + } + + @JsonProperty + public int getSchemaNum() + { + return schemaNum; + } + + ColumnMetadata getColumnMetadata() + { + return new ColumnMetadata(name, type, null, hidden); + } + + @Override + public int hashCode() + { + return Objects.hash(connectorId, ordinalPosition, name, type, mapping, dataFormat, formatHint, keyDecoder, hidden, internal); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + PravegaColumnHandle other = (PravegaColumnHandle) obj; + return Objects.equals(this.connectorId, other.connectorId) && + Objects.equals(this.ordinalPosition, other.ordinalPosition) && + Objects.equals(this.name, other.name) && + Objects.equals(this.type, other.type) && + Objects.equals(this.mapping, other.mapping) && + Objects.equals(this.dataFormat, other.dataFormat) && + Objects.equals(this.formatHint, other.formatHint) && + Objects.equals(this.keyDecoder, other.keyDecoder) && + Objects.equals(this.hidden, other.hidden) && + Objects.equals(this.internal, other.internal); + } + + @Override + public int compareTo(PravegaColumnHandle otherHandle) + { + return Integer.compare(this.getOrdinalPosition(), otherHandle.getOrdinalPosition()); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("connectorId", connectorId) + .add("ordinalPosition", ordinalPosition) + .add("name", name) + .add("type", type) + .add("mapping", mapping) + .add("dataFormat", dataFormat) + .add("formatHint", formatHint) + .add("keyDecoder", keyDecoder) + .add("hidden", hidden) + .add("internal", internal) + .toString(); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaConnector.java b/src/main/java/com/facebook/presto/pravega/PravegaConnector.java new file mode 100644 index 0000000..5a2fd0f --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaConnector.java @@ -0,0 +1,99 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.airlift.bootstrap.LifeCycleManager; +import com.facebook.airlift.log.Logger; +import com.facebook.presto.spi.connector.Connector; +import com.facebook.presto.spi.connector.ConnectorMetadata; +import com.facebook.presto.spi.connector.ConnectorRecordSetProvider; +import com.facebook.presto.spi.connector.ConnectorSplitManager; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.facebook.presto.spi.session.PropertyMetadata; +import com.facebook.presto.spi.transaction.IsolationLevel; + +import java.util.List; + +import static com.facebook.presto.pravega.PravegaProperties.buildSessionProperties; +import static com.facebook.presto.spi.transaction.IsolationLevel.READ_COMMITTED; +import static com.facebook.presto.spi.transaction.IsolationLevel.checkConnectorSupports; +import static java.util.Objects.requireNonNull; + +/** + * Pravega specific implementation of the Presto Connector SPI. This is a read only connector. + */ +public class PravegaConnector + implements Connector +{ + private static final Logger log = Logger.get(PravegaConnector.class); + + private final LifeCycleManager lifeCycleManager; + private final ConnectorMetadata metadata; + private final ConnectorSplitManager splitManager; + private final ConnectorRecordSetProvider recordSetProvider; + + public PravegaConnector( + LifeCycleManager lifeCycleManager, + ConnectorMetadata metadata, + ConnectorSplitManager splitManager, + ConnectorRecordSetProvider recordSetProvider) + { + this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); + this.metadata = requireNonNull(metadata, "metadata is null"); + this.splitManager = requireNonNull(splitManager, "splitManager is null"); + this.recordSetProvider = requireNonNull(recordSetProvider, "recordSetProvider is null"); + } + + @Override + public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly) + { + checkConnectorSupports(READ_COMMITTED, isolationLevel); + return PravegaTransactionHandle.INSTANCE; + } + + @Override + public ConnectorMetadata getMetadata(ConnectorTransactionHandle transactionHandle) + { + return metadata; + } + + @Override + public ConnectorSplitManager getSplitManager() + { + return splitManager; + } + + @Override + public ConnectorRecordSetProvider getRecordSetProvider() + { + return recordSetProvider; + } + + @Override + public List> getSessionProperties() + { + return buildSessionProperties(); + } + + @Override + public final void shutdown() + { + try { + lifeCycleManager.stop(); + } + catch (Exception e) { + log.error(e, "Error shutting down connector"); + } + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaConnectorConfig.java b/src/main/java/com/facebook/presto/pravega/PravegaConnectorConfig.java new file mode 100644 index 0000000..45b4f26 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaConnectorConfig.java @@ -0,0 +1,105 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.airlift.configuration.Config; + +import javax.validation.constraints.NotNull; + +import java.io.File; +import java.net.URI; + +public class PravegaConnectorConfig +{ + /** + * Pravega Controller URI + */ + private URI controllerURI; + + /** + * Pravega Schema Registry URI + */ + private URI schemaRegistryURI; + + /** + * how long to cache schema+table objects for before retrieving for pravega + */ + private int tableCacheExpireSecs = Integer.MAX_VALUE; + + /** + * Whether internal columns are shown in table metadata or not. Default is no. + */ + private boolean hideInternalColumns = true; + + /** + * Folder holding the JSON description files for Pravega stream. + */ + private File tableDescriptionDir = new File("etc/pravega/"); + + @NotNull + public URI getControllerURI() + { + return this.controllerURI; + } + + @NotNull + public URI getSchemaRegistryURI() + { + return this.schemaRegistryURI; + } + + @NotNull + public File getTableDescriptionDir() + { + return tableDescriptionDir; + } + + @Config("pravega.table-description-dir") + public PravegaConnectorConfig setTableDescriptionDir(File tableDescriptionDir) + { + this.tableDescriptionDir = tableDescriptionDir; + return this; + } + + public int getTableCacheExpireSecs() + { + return this.tableCacheExpireSecs; + } + + @Config("pravega.controller") + public PravegaConnectorConfig setControllerURI(URI controllerURI) + { + this.controllerURI = controllerURI; + return this; + } + + @Config("pravega.schema-registry") + public PravegaConnectorConfig setSchemaRegistryURI(URI schemaRegistryURI) + { + this.schemaRegistryURI = schemaRegistryURI; + return this; + } + + public boolean isHideInternalColumns() + { + return hideInternalColumns; + } + + @Config("pravega.hide-internal-columns") + public PravegaConnectorConfig setHideInternalColumns(boolean hideInternalColumns) + { + this.hideInternalColumns = hideInternalColumns; + return this; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaConnectorFactory.java b/src/main/java/com/facebook/presto/pravega/PravegaConnectorFactory.java new file mode 100644 index 0000000..8153659 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaConnectorFactory.java @@ -0,0 +1,105 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.airlift.bootstrap.Bootstrap; +import com.facebook.airlift.bootstrap.LifeCycleManager; +import com.facebook.airlift.json.JsonModule; +import com.facebook.presto.common.type.TypeManager; +import com.facebook.presto.spi.ConnectorHandleResolver; +import com.facebook.presto.spi.NodeManager; +import com.facebook.presto.spi.connector.Connector; +import com.facebook.presto.spi.connector.ConnectorContext; +import com.facebook.presto.spi.connector.ConnectorFactory; +import com.facebook.presto.spi.connector.classloader.ClassLoaderSafeConnectorMetadata; +import com.facebook.presto.spi.connector.classloader.ClassLoaderSafeConnectorSplitManager; +import com.google.inject.Injector; +import com.google.inject.Scopes; + +import java.util.Map; +import java.util.Optional; + +import static com.google.common.base.Throwables.throwIfUnchecked; +import static java.util.Objects.requireNonNull; + +/** + * Creates Pravega Connectors based off connectorId and specific configuration. + */ +public class PravegaConnectorFactory + implements ConnectorFactory +{ + private final Optional tableDescriptionSupplier; + private final ClassLoader classLoader; + + PravegaConnectorFactory(Optional tableDescriptionSupplier, ClassLoader classLoader) + { + this.tableDescriptionSupplier = requireNonNull(tableDescriptionSupplier, "tableDescriptionSupplier is null"); + this.classLoader = requireNonNull(classLoader, "classLoader is null"); + } + + @Override + public String getName() + { + return "pravega"; + } + + @Override + public ConnectorHandleResolver getHandleResolver() + { + return new PravegaHandleResolver(); + } + + @Override + public Connector create(String catalogName, Map config, ConnectorContext context) + { + requireNonNull(catalogName, "catalogName is null"); + requireNonNull(config, "config is null"); + + try { + Bootstrap app = new Bootstrap( + new JsonModule(), + new PravegaConnectorModule(), + binder -> { + binder.bind(PravegaConnectorId.class).toInstance(new PravegaConnectorId(catalogName)); + binder.bind(TypeManager.class).toInstance(context.getTypeManager()); + binder.bind(NodeManager.class).toInstance(context.getNodeManager()); + + if (tableDescriptionSupplier.isPresent()) { + binder.bind(PravegaTableDescriptionSupplier.class).toInstance(tableDescriptionSupplier.get()); + } + else { + binder.bind(PravegaTableDescriptionSupplier.class).in(Scopes.SINGLETON); + } + }); + + Injector injector = app.doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); + + LifeCycleManager lifeCycleManager = injector.getInstance(LifeCycleManager.class); + PravegaMetadata pravegaMetadata = injector.getInstance(PravegaMetadata.class); + PravegaSplitManager pravegaSplitManager = injector.getInstance(PravegaSplitManager.class); + PravegaRecordSetProvider pravegaRecordSetProvider = injector.getInstance(PravegaRecordSetProvider.class); + + return new PravegaConnector(lifeCycleManager, + new ClassLoaderSafeConnectorMetadata(pravegaMetadata, classLoader), + new ClassLoaderSafeConnectorSplitManager(pravegaSplitManager, classLoader), + new ClassLoaderSafeRecordSetProvider(pravegaRecordSetProvider, classLoader)); + } + catch (Exception e) { + throwIfUnchecked(e); + throw new RuntimeException(e); + } + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaConnectorId.java b/src/main/java/com/facebook/presto/pravega/PravegaConnectorId.java new file mode 100644 index 0000000..8646321 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaConnectorId.java @@ -0,0 +1,53 @@ +/* + * 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 com.facebook.presto.pravega; + +import java.util.Objects; + +import static java.util.Objects.requireNonNull; + +public class PravegaConnectorId +{ + private final String connectorId; + + public PravegaConnectorId(String connectorId) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null"); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + PravegaConnectorId other = (PravegaConnectorId) obj; + return Objects.equals(this.connectorId, other.connectorId); + } + + @Override + public int hashCode() + { + return Objects.hash(connectorId); + } + + @Override + public String toString() + { + return connectorId; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaConnectorModule.java b/src/main/java/com/facebook/presto/pravega/PravegaConnectorModule.java new file mode 100644 index 0000000..78c2323 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaConnectorModule.java @@ -0,0 +1,75 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.airlift.configuration.AbstractConfigurationAwareModule; +import com.facebook.presto.common.type.Type; +import com.facebook.presto.common.type.TypeManager; +import com.facebook.presto.decoder.DecoderModule; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer; +import com.google.inject.Binder; +import com.google.inject.Scopes; + +import javax.inject.Inject; + +import static com.facebook.airlift.configuration.ConfigBinder.configBinder; +import static com.facebook.airlift.json.JsonBinder.jsonBinder; +import static com.facebook.airlift.json.JsonCodecBinder.jsonCodecBinder; +import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature; +import static java.util.Objects.requireNonNull; + +/** + * Guice module for the Pravega connector. + */ +public class PravegaConnectorModule + extends AbstractConfigurationAwareModule +{ + @Override + public void setup(Binder binder) + { + binder.bind(PravegaMetadata.class).in(Scopes.SINGLETON); + binder.bind(PravegaSplitManager.class).in(Scopes.SINGLETON); + binder.bind(PravegaRecordSetProvider.class).in(Scopes.SINGLETON); + + binder.bind(PravegaSegmentManager.class).in(Scopes.SINGLETON); + configBinder(binder).bindConfig(PravegaConnectorConfig.class); + + jsonBinder(binder).addDeserializerBinding(Type.class).to(TypeDeserializer.class); + jsonCodecBinder(binder).bindJsonCodec(PravegaStreamDescription.class); + + binder.install(new DecoderModule()); + } + + public static final class TypeDeserializer + extends FromStringDeserializer + { + private static final long serialVersionUID = 1L; + + private final TypeManager typeManager; + + @Inject + public TypeDeserializer(TypeManager typeManager) + { + super(Type.class); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + } + + @Override + protected Type _deserialize(String value, DeserializationContext context) + { + return typeManager.getType(parseTypeSignature(value)); + } + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaErrorCode.java b/src/main/java/com/facebook/presto/pravega/PravegaErrorCode.java new file mode 100644 index 0000000..10137f7 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaErrorCode.java @@ -0,0 +1,44 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.spi.ErrorCode; +import com.facebook.presto.spi.ErrorCodeSupplier; +import com.facebook.presto.spi.ErrorType; + +import static com.facebook.presto.spi.ErrorType.EXTERNAL; + +/** + * Pravega connector specific error codes. + */ +public enum PravegaErrorCode + implements ErrorCodeSupplier +{ + PRAVEGA_SPLIT_ERROR(0, EXTERNAL), + + PRAVEGA_READER_ERROR(1, EXTERNAL); + + private final ErrorCode errorCode; + + PravegaErrorCode(int code, ErrorType type) + { + errorCode = new ErrorCode(code + 0x0102_0000, name(), type); + } + + @Override + public ErrorCode toErrorCode() + { + return errorCode; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaHandleResolver.java b/src/main/java/com/facebook/presto/pravega/PravegaHandleResolver.java new file mode 100644 index 0000000..7f2fa6b --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaHandleResolver.java @@ -0,0 +1,89 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ConnectorHandleResolver; +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.ConnectorTableHandle; +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +/** + * Pravega specific {@link com.facebook.presto.spi.ConnectorHandleResolver} implementation. + */ +public class PravegaHandleResolver + implements ConnectorHandleResolver +{ + @Override + public Class getTableHandleClass() + { + return PravegaTableHandle.class; + } + + @Override + public Class getColumnHandleClass() + { + return PravegaColumnHandle.class; + } + + @Override + public Class getSplitClass() + { + return PravegaSplit.class; + } + + @Override + public Class getTableLayoutHandleClass() + { + return PravegaTableLayoutHandle.class; + } + + @Override + public Class getTransactionHandleClass() + { + return PravegaTransactionHandle.class; + } + + static PravegaTableHandle convertTableHandle(ConnectorTableHandle tableHandle) + { + requireNonNull(tableHandle, "tableHandle is null"); + checkArgument(tableHandle instanceof PravegaTableHandle, "tableHandle is not an instance of PravegaTableHandle"); + return (PravegaTableHandle) tableHandle; + } + + static PravegaColumnHandle convertColumnHandle(ColumnHandle columnHandle) + { + requireNonNull(columnHandle, "columnHandle is null"); + checkArgument(columnHandle instanceof PravegaColumnHandle, "columnHandle is not an instance of PravegaColumnHandle"); + return (PravegaColumnHandle) columnHandle; + } + + static PravegaSplit convertSplit(ConnectorSplit split) + { + requireNonNull(split, "split is null"); + checkArgument(split instanceof PravegaSplit, "split is not an instance of PravegaSplit"); + return (PravegaSplit) split; + } + + static PravegaTableLayoutHandle convertLayout(ConnectorTableLayoutHandle layout) + { + requireNonNull(layout, "layout is null"); + checkArgument(layout instanceof PravegaTableLayoutHandle, "layout is not an instance of PravegaTableLayoutHandle"); + return (PravegaTableLayoutHandle) layout; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaInternalFieldDescription.java b/src/main/java/com/facebook/presto/pravega/PravegaInternalFieldDescription.java new file mode 100644 index 0000000..ce154ce --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaInternalFieldDescription.java @@ -0,0 +1,121 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.common.type.BigintType; +import com.facebook.presto.common.type.BooleanType; +import com.facebook.presto.common.type.Type; +import com.facebook.presto.spi.ColumnMetadata; + +import java.util.Map; + +import static com.facebook.presto.common.type.VarcharType.createUnboundedVarcharType; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static java.util.Arrays.stream; +import static java.util.Objects.requireNonNull; +import static java.util.function.Function.identity; + +/** + * Describes an internal (managed by the connector) field which is added to each table row. The definition itself makes the row + * show up in the tables (the columns are hidden by default, so they must be explicitly selected) but unless the field is hooked in using the + * forBooleanValue/forLongValue/forBytesValue methods and the resulting FieldValueProvider is then passed into the appropriate row decoder, the fields + * will be null. Most values are assigned in the {@link com.facebook.presto.pravega.PravegaRecordSet}. + */ +public enum PravegaInternalFieldDescription +{ + /** + * _segment_id - Pravega segment id. + */ + SEGMENT_ID_FIELD("_segment_id", BigintType.BIGINT, "Segment Id"), + + /** + * _segment_offset - The current offset of the event in the segment. + */ + SEGMENT_OFFSET_FIELD("_segment_offset", BigintType.BIGINT, "Offset for the event within the segment"), + + /** + * _event_corrupt - True if the row converter could not read the event. May be null if the row + * converter does not set a value (e.g. the dummy row converter does not). + */ + EVENT_CORRUPT_FIELD("_event_corrupt", BooleanType.BOOLEAN, "Event data is corrupt"), + + /** + * _event - Represents the full stream as a text column. Format is UTF-8 which may be wrong + * for some stream. TODO: make charset configurable. + */ + EVENT_FIELD("_message", createUnboundedVarcharType(), "Event text"), + + /** + * _event_length - length in bytes of the mevent. + */ + EVENT_LENGTH_FIELD("_event_length", BigintType.BIGINT, "Total number of event bytes"); + + private static final Map BY_COLUMN_NAME = + stream(PravegaInternalFieldDescription.values()) + .collect(toImmutableMap(PravegaInternalFieldDescription::getColumnName, identity())); + + public static PravegaInternalFieldDescription forColumnName(String columnName) + { + PravegaInternalFieldDescription description = BY_COLUMN_NAME.get(columnName); + checkArgument(description != null, "Unknown internal column name %s", columnName); + return description; + } + + private final String columnName; + private final Type type; + private final String comment; + + PravegaInternalFieldDescription( + String columnName, + Type type, + String comment) + { + checkArgument(!isNullOrEmpty(columnName), "name is null or is empty"); + this.columnName = columnName; + this.type = requireNonNull(type, "type is null"); + this.comment = requireNonNull(comment, "comment is null"); + } + + public String getColumnName() + { + return columnName; + } + + public Type getType() + { + return type; + } + + PravegaColumnHandle getColumnHandle(String connectorId, int index, boolean hidden) + { + return new PravegaColumnHandle(connectorId, + index, + getColumnName(), + getType(), + null, + null, + null, + false, + hidden, + true, + 1); + } + + ColumnMetadata getColumnMetadata(boolean hidden) + { + return new ColumnMetadata(columnName, type, comment, hidden); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaKVRecordSet.java b/src/main/java/com/facebook/presto/pravega/PravegaKVRecordSet.java new file mode 100644 index 0000000..e5aae8d --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaKVRecordSet.java @@ -0,0 +1,175 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.airlift.log.Logger; +import com.facebook.presto.common.type.Type; +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.pravega.decoder.DecodableEvent; +import com.facebook.presto.pravega.decoder.KVSerializer; +import com.facebook.presto.pravega.decoder.MultiSourceRowDecoder; +import com.facebook.presto.spi.RecordCursor; +import com.facebook.presto.spi.RecordSet; +import com.google.common.collect.ImmutableList; +import io.pravega.client.tables.IteratorItem; +import io.pravega.client.tables.KeyValueTable; +import io.pravega.client.tables.KeyValueTableClientConfiguration; +import io.pravega.client.tables.TableEntry; +import io.pravega.common.util.AsyncIterator; + +import java.util.Iterator; +import java.util.List; + +import static com.facebook.presto.pravega.util.PravegaSerializationUtils.deserialize; +import static java.util.Objects.requireNonNull; + +public class PravegaKVRecordSet + implements RecordSet +{ + private static final Logger log = Logger.get(PravegaKVRecordSet.class); + + private final PravegaProperties properties; + + private final PravegaSplit split; + private final PravegaSegmentManager segmentManager; + + private final MultiSourceRowDecoder eventDecoder; + + private final List columnHandles; + private final List columnTypes; + + private final List> serializers; + + PravegaKVRecordSet(PravegaProperties properties, + PravegaSplit split, + PravegaSegmentManager segmentManager, + List columnHandles, + MultiSourceRowDecoder eventDecoder, + List> serializers) + { + this.properties = requireNonNull(properties, "properties is null"); + + this.split = requireNonNull(split, "split is null"); + + this.segmentManager = requireNonNull(segmentManager, "segmentManager is null"); + + this.eventDecoder = requireNonNull(eventDecoder, "rowDecoder is null"); + + this.columnHandles = requireNonNull(columnHandles, "columnHandles is null"); + + this.serializers = requireNonNull(serializers, "serializers is null"); + + ImmutableList.Builder typeBuilder = ImmutableList.builder(); + + for (DecoderColumnHandle handle : columnHandles) { + typeBuilder.add(handle.getType()); + } + + this.columnTypes = typeBuilder.build(); + } + + @Override + public List getColumnTypes() + { + return columnTypes; + } + + @Override + public RecordCursor cursor() + { + // listing from a kv table.keyfamily + // + // iterator we get will fetch N items at a time - this is batchIter + // from there we keep a local iterator to iterate over those N items - this is itemIter + // when itemIter is done, fetch more from batchIter + // batchIter returns null on EOF + + PravegaKVTable kvTable = deserialize(split.getReaderArgs(), PravegaKVTable.class); + AsyncIterator>> batchIter = constructIterator(kvTable); + + final Iterator eventIterator = new Iterator() + { + boolean eof; + DecodableEvent nextEvent; + Iterator> itemIter; + + @Override + public boolean hasNext() + { + return nextEvent != null || _next() != null; + } + + @Override + public DecodableEvent next() + { + try { + if (nextEvent != null) { + return nextEvent; + } + return _next(); + } + finally { + nextEvent = null; + } + } + + Iterator> iter() + { + // return next batch of key/values + IteratorItem> item = batchIter.getNext().join(); + if (item == null) { + return null; + } + + return item.getItems().iterator(); + } + + private DecodableEvent _next() + { + if (eof) { + return null; + } + + if (itemIter == null || !itemIter.hasNext()) { + // no more items, get next batch + itemIter = iter(); + if (itemIter == null) { + eof = true; + return null; + } + } + + TableEntry tableEntry = itemIter.next(); + // 1 serializer for key, 1 for value + DecodableEvent value = serializers.get(1).toEvent(tableEntry.getValue()); + nextEvent = serializers.get(0).toEvent(tableEntry.getKey().getKey()); + nextEvent.setNext(value); // key, followed by value + return nextEvent; + } + }; + + return new PravegaRecordCursor(eventIterator, columnHandles, eventDecoder, properties, "kv"); + } + + AsyncIterator>> constructIterator(final PravegaKVTable kvTable) + { + KeyValueTable table = segmentManager.getKeyValueTableFactory(kvTable.getScope()) + .forKeyValueTable(kvTable.getTable(), + serializers.get(0), + serializers.get(1), + KeyValueTableClientConfiguration.builder().build()); + + return table.entryIterator(kvTable.getKeyFamily(), 1024, null); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaKVTable.java b/src/main/java/com/facebook/presto/pravega/PravegaKVTable.java new file mode 100644 index 0000000..5b68ac6 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaKVTable.java @@ -0,0 +1,59 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.Serializable; + +import static java.util.Objects.requireNonNull; + +public class PravegaKVTable + implements Serializable +{ + private final String scope; + + private final String table; + + private final String keyFamily; + + @JsonCreator + public PravegaKVTable(@JsonProperty("scope") String scope, + @JsonProperty("table") String table, + @JsonProperty("keyFamily") String keyFamily) + { + this.scope = requireNonNull(scope, "scope is null"); + this.table = requireNonNull(table, "table is null"); + this.keyFamily = requireNonNull(keyFamily, "keyFamily is null"); + } + + @JsonProperty + public String getScope() + { + return scope; + } + + @JsonProperty + public String getTable() + { + return table; + } + + @JsonProperty + public String getKeyFamily() + { + return keyFamily; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaMetadata.java b/src/main/java/com/facebook/presto/pravega/PravegaMetadata.java new file mode 100644 index 0000000..012fd77 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaMetadata.java @@ -0,0 +1,255 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.airlift.log.Logger; +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ColumnMetadata; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorTableHandle; +import com.facebook.presto.spi.ConnectorTableLayout; +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.facebook.presto.spi.ConnectorTableLayoutResult; +import com.facebook.presto.spi.ConnectorTableMetadata; +import com.facebook.presto.spi.Constraint; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.SchemaTablePrefix; +import com.facebook.presto.spi.TableNotFoundException; +import com.facebook.presto.spi.connector.ConnectorMetadata; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +import javax.inject.Inject; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +import static com.facebook.presto.pravega.PravegaHandleResolver.convertColumnHandle; +import static com.facebook.presto.pravega.PravegaHandleResolver.convertTableHandle; +import static com.facebook.presto.pravega.util.PravegaNameUtils.groupId; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +/** + * Manages the Pravega connector specific metadata information. The Connector provides an additional set of columns + * for each table that are created as hidden columns. See {@link PravegaInternalFieldDescription} for a list + * of per-stream additional columns. + */ +public class PravegaMetadata + implements ConnectorMetadata +{ + private static final Logger log = Logger.get(PravegaMetadata.class); + private final String connectorId; + private final boolean hideInternalColumns; + private final PravegaTableDescriptionSupplier tableDescSupplier; + + @Inject + public PravegaMetadata( + PravegaConnectorId connectorId, + PravegaConnectorConfig pravegaConnectorConfig, + PravegaTableDescriptionSupplier tableDescSupplier) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); + + requireNonNull(pravegaConnectorConfig, "pravegaConnectorConfig is null"); + this.hideInternalColumns = pravegaConnectorConfig.isHideInternalColumns(); + + requireNonNull(tableDescSupplier, "pravegaTableDescriptionSupplier is null"); + this.tableDescSupplier = tableDescSupplier; + } + + @Override + public List listSchemaNames(ConnectorSession session) + { + log.debug("Listing schema names"); + return tableDescSupplier.listSchemas(); + } + + List extractSchema(PravegaStreamDescription table) + { + table.getEvent().orElseThrow(() -> + new IllegalArgumentException("unable to extract schema from " + table)); + List list = new ArrayList<>(table.getEvent().get().size()); + table.getEvent().get().forEach(event -> { + list.add(new PravegaObjectSchema(event.getDataFormat(), event.getDataSchema())); + }); + return list; + } + + @Override + public PravegaTableHandle getTableHandle(ConnectorSession session, SchemaTableName schemaTableName) + { + log.debug("getTableHandle for %s", schemaTableName); + PravegaStreamDescription table = tableDescSupplier.getTable(schemaTableName); + if (table == null) { + return null; + } + + return new PravegaTableHandle(connectorId, + schemaTableName.getSchemaName(), + schemaTableName.getTableName(), + table.getObjectName(), + table.getObjectType(), + table.getObjectArgs(), + extractSchema(table), + groupId(schemaTableName.getSchemaName(), table.getObjectName())); + } + + @Override + public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle tableHandle) + { + return getTableMetadata(convertTableHandle(tableHandle).toSchemaTableName()); + } + + @Override + public List listTables(ConnectorSession session, Optional schemaNameOrNull) + { + return tableDescSupplier.listTables(schemaNameOrNull).stream() + .filter(p -> !p.getHidden()) + .map(PravegaTableName::getSchemaTableName) + .collect(Collectors.toList()); + } + + @SuppressWarnings("ValueOfIncrementOrDecrementUsed") + @Override + public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) + { + log.debug("Getting column handles"); + PravegaTableHandle pravegaTableHandle = convertTableHandle(tableHandle); + + PravegaStreamDescription pravegaStreamDescription = + tableDescSupplier.getTable(pravegaTableHandle.toSchemaTableName()); + if (pravegaStreamDescription == null) { + throw new TableNotFoundException(pravegaTableHandle.toSchemaTableName()); + } + + ImmutableMap.Builder columnHandles = ImmutableMap.builder(); + + AtomicInteger index = new AtomicInteger(0); + AtomicInteger schemaNum = new AtomicInteger(0); + + pravegaStreamDescription.getEvent().ifPresent(events -> { + events.forEach(event -> { + List fields = event.getFields(); + if (fields != null) { + for (PravegaStreamFieldDescription pravegaStreamFieldDescription : fields) { + columnHandles.put(pravegaStreamFieldDescription.getName(), + pravegaStreamFieldDescription.getColumnHandle(connectorId, + false, + index.getAndIncrement(), + schemaNum.get())); + } + } + schemaNum.incrementAndGet(); + }); + }); + + for (PravegaInternalFieldDescription pravegaInternalFieldDescription : PravegaInternalFieldDescription.values()) { + columnHandles.put(pravegaInternalFieldDescription.getColumnName(), + pravegaInternalFieldDescription.getColumnHandle(connectorId, + index.getAndIncrement(), + hideInternalColumns)); + } + + return columnHandles.build(); + } + + @Override + public Map> listTableColumns(ConnectorSession session, SchemaTablePrefix prefix) + { + requireNonNull(prefix, "prefix is null"); + + ImmutableMap.Builder> columns = ImmutableMap.builder(); + + List tableNames; + if (prefix.getTableName() == null) { + tableNames = listTables(session, prefix.getSchemaName()); + } + else { + tableNames = ImmutableList.of(new SchemaTableName(prefix.getSchemaName(), prefix.getTableName())); + } + + for (SchemaTableName tableName : tableNames) { + try { + columns.put(tableName, getTableMetadata(tableName).getColumns()); + } + catch (TableNotFoundException e) { + // Normally it would mean the table disappeared during listing operation + throw new IllegalStateException(format("Table %s cannot be gone because tables are statically defined", tableName), e); + } + } + return columns.build(); + } + + @Override + public ColumnMetadata getColumnMetadata(ConnectorSession session, + ConnectorTableHandle tableHandle, + ColumnHandle columnHandle) + { + convertTableHandle(tableHandle); + return convertColumnHandle(columnHandle).getColumnMetadata(); + } + + @Override + public List getTableLayouts(ConnectorSession session, + ConnectorTableHandle table, + Constraint constraint, + Optional> desiredColumns) + { + PravegaTableHandle handle = convertTableHandle(table); + + ConnectorTableLayout layout = new ConnectorTableLayout(new PravegaTableLayoutHandle(handle)); + return ImmutableList.of(new ConnectorTableLayoutResult(layout, constraint.getSummary())); + } + + @Override + public ConnectorTableLayout getTableLayout(ConnectorSession session, ConnectorTableLayoutHandle handle) + { + return new ConnectorTableLayout(handle); + } + + @SuppressWarnings("ValueOfIncrementOrDecrementUsed") + private ConnectorTableMetadata getTableMetadata(SchemaTableName schemaTableName) + { + log.debug("getTableMetadata %s", schemaTableName); + PravegaStreamDescription table = tableDescSupplier.getTable(schemaTableName); + if (table == null) { + throw new TableNotFoundException(schemaTableName); + } + + ImmutableList.Builder builder = ImmutableList.builder(); + + table.getEvent().ifPresent(events -> { + events.forEach(event -> { + List fields = event.getFields(); + if (fields != null) { + for (PravegaStreamFieldDescription fieldDescription : fields) { + builder.add(fieldDescription.getColumnMetadata()); + } + } + }); + }); + + for (PravegaInternalFieldDescription fieldDescription : PravegaInternalFieldDescription.values()) { + builder.add(fieldDescription.getColumnMetadata(hideInternalColumns)); + } + + return new ConnectorTableMetadata(schemaTableName, builder.build()); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaObjectSchema.java b/src/main/java/com/facebook/presto/pravega/PravegaObjectSchema.java new file mode 100644 index 0000000..c8f47a1 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaObjectSchema.java @@ -0,0 +1,58 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Optional; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +public class PravegaObjectSchema +{ + private final String format; + + private final Optional schemaLocation; + + @JsonCreator + public PravegaObjectSchema(@JsonProperty("format") String format, + @JsonProperty("schemaLocation") Optional schemaLocation) + { + this.format = requireNonNull(format, "format is null"); + this.schemaLocation = requireNonNull(schemaLocation, "schemaLocation is null"); + } + + @JsonProperty + public String getFormat() + { + return format; + } + + @JsonProperty + public Optional getSchemaLocation() + { + return schemaLocation; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("format", format) + .add("schemaLocation", schemaLocation) + .toString(); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaPlugin.java b/src/main/java/com/facebook/presto/pravega/PravegaPlugin.java new file mode 100644 index 0000000..785e283 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaPlugin.java @@ -0,0 +1,53 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.spi.Plugin; +import com.facebook.presto.spi.connector.ConnectorFactory; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; + +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +/** + * Presto plugin to use Apache Pravega as a data source. + */ +public class PravegaPlugin + implements Plugin +{ + private Optional tableDescriptionSupplier = Optional.empty(); + + @VisibleForTesting + public synchronized void setTableDescriptionSupplier(PravegaTableDescriptionSupplier tableDescriptionSupplier) + { + this.tableDescriptionSupplier = Optional.of(requireNonNull(tableDescriptionSupplier, "tableDescriptionSupplier is null")); + } + + @Override + public synchronized Iterable getConnectorFactories() + { + return ImmutableList.of(new PravegaConnectorFactory(tableDescriptionSupplier, getClassLoader())); + } + + private static ClassLoader getClassLoader() + { + ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + if (classLoader == null) { + classLoader = PravegaPlugin.class.getClassLoader(); + } + return classLoader; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaProperties.java b/src/main/java/com/facebook/presto/pravega/PravegaProperties.java new file mode 100644 index 0000000..4739a76 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaProperties.java @@ -0,0 +1,93 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.session.PropertyMetadata; + +import java.util.LinkedList; +import java.util.List; + +public class PravegaProperties +{ + private static final String SESSION_READER_TYPE = "reader_type"; + + private static final String SESSION_CURSOR_DELIM_CHAR = "cursor_delim_char"; + + private static final String SESSION_GROUPED_EVENT_SPLITS = "grouped_event_splits"; + + private static final String SESSION_EVENT_READ_TIMEOUT_MS = "event_read_timeout_ms"; + + private final ConnectorSession session; + + public PravegaProperties(final ConnectorSession session) + { + this.session = session; + } + + public static List> buildSessionProperties() + { + List> propertyMetadataList = new LinkedList<>(); + + propertyMetadataList.add( + PropertyMetadata.stringProperty( + SESSION_CURSOR_DELIM_CHAR, + "character used as field separator for delimited formats", + ",", + false)); + + propertyMetadataList.add( + PropertyMetadata.stringProperty( + SESSION_READER_TYPE, + "reader type [event|grouped_event|segment_range|segment_range_per_split]", + "segment_range_per_split", + false)); + + propertyMetadataList.add( + PropertyMetadata.integerProperty( + SESSION_GROUPED_EVENT_SPLITS, + "number of splits when using grouped readers", + 63, + false)); + + propertyMetadataList.add( + PropertyMetadata.integerProperty( + SESSION_EVENT_READ_TIMEOUT_MS, + "timeout in ms to readNextEvent()", + 10000, + false)); + + return propertyMetadataList; + } + + public String getCursorDelimChar() + { + return session.getProperty(SESSION_CURSOR_DELIM_CHAR, String.class); + } + + public String getReaderType() + { + return session.getProperty(SESSION_READER_TYPE, String.class); + } + + public int getGroupedEventSplits() + { + return session.getProperty(SESSION_GROUPED_EVENT_SPLITS, Integer.class); + } + + public int getEventReadTimeoutMs() + { + return session.getProperty(SESSION_EVENT_READ_TIMEOUT_MS, Integer.class); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaRecordCursor.java b/src/main/java/com/facebook/presto/pravega/PravegaRecordCursor.java new file mode 100644 index 0000000..3533273 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaRecordCursor.java @@ -0,0 +1,175 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.airlift.log.Logger; +import com.facebook.presto.common.type.Type; +import com.facebook.presto.pravega.decoder.DecodableEvent; +import com.facebook.presto.pravega.decoder.EventDecoder; +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.RecordCursor; +import io.airlift.slice.Slice; + +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +import static com.facebook.presto.pravega.util.PravegaSchemaUtils.CSV; +import static com.google.common.base.Preconditions.checkArgument; + +public class PravegaRecordCursor + implements RecordCursor +{ + private static final Logger log = Logger.get(PravegaRecordCursor.class); + private long totalBytes; + private final Iterator iterator; + private final List columnHandles; + private final EventDecoder eventDecoder; + + private static final AtomicInteger CURSOR_ID = new AtomicInteger(); + private final int cursorId; + + private final int[] fieldToColumnIndex; + + private final PravegaRecordValue record; + + PravegaRecordCursor(Iterator iterator, + List columnHandles, + EventDecoder eventDecoder, + PravegaProperties properties, + String format) + { + this.iterator = iterator; + this.columnHandles = columnHandles; + this.eventDecoder = eventDecoder; + this.record = createRecordValue(format, properties, columnHandles); + this.cursorId = CURSOR_ID.getAndIncrement(); + log.debug("open record cursor " + cursorId); + + this.fieldToColumnIndex = new int[columnHandles.size()]; + + for (int i = 0; i < columnHandles.size(); i++) { + PravegaColumnHandle columnHandle = columnHandles.get(i); + this.fieldToColumnIndex[i] = columnHandle.getOrdinalPosition(); + } + } + + @Override + public long getCompletedBytes() + { + return totalBytes; + } + + @Override + public long getReadTimeNanos() + { + return 0; + } + + @Override + public Type getType(int field) + { + checkArgument(field < columnHandles.size(), "Invalid field index"); + return columnHandles.get(field).getType(); + } + + @Override + public boolean advanceNextPosition() + { + return iterator.hasNext() && nextRow(iterator.next()); + } + + private boolean nextRow(DecodableEvent event) + { + totalBytes += event.totalSize(); + + return eventDecoder.decodeEvent(event, record); + } + + @Override + public boolean getBoolean(int field) + { + checkField(field, boolean.class); + return record.getBoolean(field, fieldToColumnIndex[field]); + } + + @Override + public long getLong(int field) + { + checkField(field, long.class); + return record.getLong(field, fieldToColumnIndex[field]); + } + + @Override + public double getDouble(int field) + { + checkField(field, double.class); + return record.getDouble(field, fieldToColumnIndex[field]); + } + + @Override + public Slice getSlice(int field) + { + checkField(field, Slice.class); + return record.getSlice(field, fieldToColumnIndex[field]); + } + + @Override + public Object getObject(int field) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isNull(int field) + { + if (!record.decoded()) { + record.decode(); + } + return record.isNull(field, fieldToColumnIndex[field]); + } + + private void checkField(int field, Class expected) + { + checkArgument(field < columnHandles.size(), "Invalid field index"); + Class actual = getType(field).getJavaType(); + checkArgument(actual == expected, "Expected field %s to be type %s but is %s", field, expected, actual); + } + + @Override + public void close() + { + log.debug("cursorId " + cursorId + " closed"); + } + + // set decoded event into this object. access values for given fields. + static PravegaRecordValue createRecordValue(String format, + PravegaProperties properties, + List columnHandles) + { + if (format.equals(CSV)) { + return new DelimRecordValue(properties.getCursorDelimChar().charAt(0)); + } + else { + return new TypedRecordValue(columnHandles.stream().map( + PravegaRecordCursor::convertColumnHandle).collect(Collectors.toList())); + } + } + + static ColumnHandle convertColumnHandle(PravegaColumnHandle columnHandle) + { + return columnHandle; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaRecordSet.java b/src/main/java/com/facebook/presto/pravega/PravegaRecordSet.java new file mode 100644 index 0000000..3c8a181 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaRecordSet.java @@ -0,0 +1,103 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.airlift.log.Logger; +import com.facebook.presto.common.type.Type; +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.pravega.decoder.DecodableEvent; +import com.facebook.presto.pravega.decoder.EventDecoder; +import com.facebook.presto.spi.RecordCursor; +import com.facebook.presto.spi.RecordSet; +import com.google.common.collect.ImmutableList; +import io.pravega.client.batch.SegmentRange; + +import java.util.Iterator; +import java.util.List; + +import static com.facebook.presto.pravega.util.PravegaSerializationUtils.deserialize; +import static java.util.Objects.requireNonNull; + +public class PravegaRecordSet + implements RecordSet +{ + private static final Logger log = Logger.get(PravegaRecordSet.class); + + private final PravegaProperties properties; + + private final PravegaSplit split; + private final PravegaSegmentManager segmentManager; + + private final EventDecoder eventDecoder; + + private final List columnHandles; + private final List columnTypes; + + PravegaRecordSet(PravegaProperties properties, + PravegaSplit split, + PravegaSegmentManager segmentManager, + List columnHandles, + EventDecoder eventDecoder) + { + this.properties = requireNonNull(properties, "properties is null"); + + this.split = requireNonNull(split, "split is null"); + + this.segmentManager = requireNonNull(segmentManager, "segmentManager is null"); + + this.eventDecoder = requireNonNull(eventDecoder, "rowDecoder is null"); + + this.columnHandles = requireNonNull(columnHandles, "columnHandles is null"); + + ImmutableList.Builder typeBuilder = ImmutableList.builder(); + + for (DecoderColumnHandle handle : columnHandles) { + typeBuilder.add(handle.getType()); + } + + this.columnTypes = typeBuilder.build(); + } + + @Override + public List getColumnTypes() + { + return columnTypes; + } + + @Override + public RecordCursor cursor() + { + Iterator eventIterator; + + switch (split.getReaderType()) { + case EVENT_STREAM: + case SINGLE_GROUP_EVENT_STREAM: + eventIterator = new EventStreamIterator(segmentManager, deserialize(split.getReaderArgs(), ReaderArgs.class), properties); + break; + + case SEGMENT_RANGE: + eventIterator = new SegmentRangeIterator(segmentManager, deserialize(split.getReaderArgs(), ReaderArgs.class)); + break; + + case SEGMENT_RANGE_PER_SPLIT: + eventIterator = new SegmentEventIterator(segmentManager, deserialize(split.getReaderArgs(), SegmentRange.class)); + break; + + default: + throw new IllegalArgumentException("readerType " + split.getReaderType()); + } + + return new PravegaRecordCursor(eventIterator, columnHandles, eventDecoder, properties, split.getSchema().get(0).getFormat()); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaRecordSetProvider.java b/src/main/java/com/facebook/presto/pravega/PravegaRecordSetProvider.java new file mode 100644 index 0000000..2d7d6b5 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaRecordSetProvider.java @@ -0,0 +1,233 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.airlift.log.Logger; +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.decoder.DispatchingRowDecoderFactory; +import com.facebook.presto.decoder.RowDecoder; +import com.facebook.presto.pravega.decoder.AvroRowDecoder; +import com.facebook.presto.pravega.decoder.AvroSerializer; +import com.facebook.presto.pravega.decoder.BytesEventDecoder; +import com.facebook.presto.pravega.decoder.CsvRowDecoder; +import com.facebook.presto.pravega.decoder.CsvSerializer; +import com.facebook.presto.pravega.decoder.EventDecoder; +import com.facebook.presto.pravega.decoder.JsonRowDecoder; +import com.facebook.presto.pravega.decoder.JsonSerializer; +import com.facebook.presto.pravega.decoder.KVSerializer; +import com.facebook.presto.pravega.decoder.MultiSourceRowDecoder; +import com.facebook.presto.pravega.decoder.ProtobufRowDecoder; +import com.facebook.presto.pravega.decoder.ProtobufSerializer; +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.RecordSet; +import com.facebook.presto.spi.connector.ConnectorRecordSetProvider; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.pravega.schemaregistry.serializer.shared.impl.SerializerConfig; + +import javax.inject.Inject; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static com.facebook.presto.pravega.PravegaHandleResolver.convertSplit; +import static com.facebook.presto.pravega.util.PravegaSchemaUtils.AVRO; +import static com.facebook.presto.pravega.util.PravegaSchemaUtils.AVRO_INLINE; +import static com.facebook.presto.pravega.util.PravegaSchemaUtils.CSV; +import static com.facebook.presto.pravega.util.PravegaSchemaUtils.JSON; +import static com.facebook.presto.pravega.util.PravegaSchemaUtils.JSON_INLINE; +import static com.facebook.presto.pravega.util.PravegaSchemaUtils.PROTOBUF; +import static com.facebook.presto.pravega.util.PravegaSchemaUtils.PROTOBUF_INLINE; +import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static java.util.Objects.requireNonNull; + +/** + * Factory for Pravega specific {@link RecordSet} instances. + */ +public class PravegaRecordSetProvider + implements ConnectorRecordSetProvider +{ + private static final Logger log = Logger.get(PravegaRecordSetProvider.class); + private DispatchingRowDecoderFactory decoderFactory; + private final PravegaSegmentManager streamReaderManager; + private final PravegaConnectorConfig config; + + @Inject + public PravegaRecordSetProvider(DispatchingRowDecoderFactory decoderFactory, + PravegaSegmentManager streamReaderManager, + PravegaConnectorConfig config) + { + this.decoderFactory = requireNonNull(decoderFactory, "decoderFactory is null"); + this.streamReaderManager = requireNonNull(streamReaderManager, "streamReaderManager is null"); + this.config = requireNonNull(config, "config is null"); + } + + @Override + public RecordSet getRecordSet(ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorSplit split, + List columns) + { + final PravegaSplit pravegaSplit = convertSplit(split); + + List pravegaColumns = columns.stream() + .map(PravegaHandleResolver::convertColumnHandle) + .collect(ImmutableList.toImmutableList()); + + SerializerConfig serializerConfig = + streamReaderManager.serializerConfig(pravegaSplit.getschemaRegistryGroupId()); + + List> serializers = new ArrayList<>(2); + List eventDecoders = new ArrayList<>(2); + + // for stream there is 1 schema + // for kv table there are 2. 1 for key, 1 for value and (very likely) they are of different types + for (int i = 0; i < pravegaSplit.getSchema().size(); i++) { + int finalI = i; + PravegaObjectSchema schema = pravegaSplit.getSchema().get(i); + + // decoderColumnHandles will contain columns included only in current schema + Set decoderColumnHandles = + pravegaColumns.stream() + .filter(col -> !col.isInternal()) + .filter(col -> !col.isKeyDecoder()) + .filter(col -> col.getSchemaNum() == finalI) + .collect(toImmutableSet()); + + // serializer: de/serialize to/from object with given schema + // (KV table will have 2 serializers. 1 for key, 1 for value) + KVSerializer serializer = serializer(schema, serializerConfig); + serializers.add(serializer); + + // EventDecoder + // accepts an already deserialized object (DynamicMessage, GenericRecord, JsonNode) and decodes it as a row + // impl. for each of avro, protobuf, json + // + // BytesEventDecoder + // takes raw bytes from a source and deserializes + // + // when iterate from KV table it gives us a TableEntry with key+value already deserialize + // for this we give right to EventDecoder + // + // for stream it will come from raw bytes + // (2 flavors of this, SR serializerConfig or our own decoder w/ provided schema) + // for this wrap EventDecoder in BytesEventDecoder + EventDecoder eventDecoder = eventDecoder(schema, decoderColumnHandles); + + if (pravegaSplit.getObjectType() == ObjectType.KV_TABLE) { + // KV table API will give us back deserialized object, use + eventDecoders.add(eventDecoder); + } + else { + // stream API gives us bytes back + eventDecoders.add(new BytesEventDecoder(serializer, eventDecoder)); + } + } + + pravegaColumns.forEach(s -> log.debug("pravega column: %s", s)); + + switch (pravegaSplit.getObjectType()) { + case STREAM: + if (eventDecoders.size() != 1) { + throw new IllegalStateException("stream should have 1 event decoder (vs " + eventDecoders.size() + ")"); + } + + return new PravegaRecordSet(new PravegaProperties(session), + pravegaSplit, + streamReaderManager, + pravegaColumns, + eventDecoders.get(0)); + + case KV_TABLE: + return new PravegaKVRecordSet(new PravegaProperties(session), + pravegaSplit, + streamReaderManager, + pravegaColumns, + new MultiSourceRowDecoder(eventDecoders), + serializers); + default: + throw new IllegalArgumentException("unexpected split type: " + pravegaSplit.toString()); + } + } + + private KVSerializer serializer(PravegaObjectSchema schema, SerializerConfig serializerConfig) + { + switch (schema.getFormat()) { + case AVRO: + return new AvroSerializer(schema.getSchemaLocation().get()); + case AVRO_INLINE: + return new AvroSerializer(serializerConfig); + + case PROTOBUF: + return new ProtobufSerializer(schema.getSchemaLocation().get()); + case PROTOBUF_INLINE: + return new ProtobufSerializer(serializerConfig); + + case JSON: + return new JsonSerializer(); + case JSON_INLINE: + return new JsonSerializer(serializerConfig); + + case CSV: + return new CsvSerializer(); + + default: + throw new IllegalArgumentException(schema.toString()); + } + } + + private EventDecoder eventDecoder(PravegaObjectSchema schema, Set decoderColumnHandles) + { + switch (schema.getFormat()) { + case AVRO: + case AVRO_INLINE: + return new AvroRowDecoder(decoderColumnHandles); + + case PROTOBUF: + case PROTOBUF_INLINE: + return new ProtobufRowDecoder(decoderColumnHandles); + + case JSON: + case JSON_INLINE: { + RowDecoder rowDecoder = decoderFactory.create( + JSON, + getDecoderParameters(schema.getSchemaLocation()), + decoderColumnHandles); + if (!(rowDecoder instanceof com.facebook.presto.decoder.json.JsonRowDecoder)) { + throw new IllegalStateException(); + } + return new JsonRowDecoder((com.facebook.presto.decoder.json.JsonRowDecoder) rowDecoder); + } + + case CSV: { + return new CsvRowDecoder(); + } + default: + throw new IllegalArgumentException(schema.toString()); + } + } + + private static Map getDecoderParameters(Optional dataSchema) + { + ImmutableMap.Builder parameters = ImmutableMap.builder(); + dataSchema.ifPresent(schema -> parameters.put("dataSchema", schema)); + return parameters.build(); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaRecordValue.java b/src/main/java/com/facebook/presto/pravega/PravegaRecordValue.java new file mode 100644 index 0000000..437d982 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaRecordValue.java @@ -0,0 +1,33 @@ +/* + * 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 com.facebook.presto.pravega; + +import io.airlift.slice.Slice; + +public interface PravegaRecordValue +{ + void decode(); + + boolean decoded(); + + boolean isNull(int field, int ordinalPosition); + + long getLong(int field, int ordinalPosition); + + double getDouble(int field, int ordinalPosition); + + boolean getBoolean(int field, int ordinalPosition); + + Slice getSlice(int field, int ordinalPosition); +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaSegmentManager.java b/src/main/java/com/facebook/presto/pravega/PravegaSegmentManager.java new file mode 100644 index 0000000..c53535e --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaSegmentManager.java @@ -0,0 +1,184 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.airlift.log.Logger; +import io.pravega.client.BatchClientFactory; +import io.pravega.client.ClientConfig; +import io.pravega.client.EventStreamClientFactory; +import io.pravega.client.KeyValueTableFactory; +import io.pravega.client.admin.ReaderGroupManager; +import io.pravega.client.admin.StreamManager; +import io.pravega.client.batch.SegmentIterator; +import io.pravega.client.batch.SegmentRange; +import io.pravega.client.batch.StreamSegmentsIterator; +import io.pravega.client.stream.Stream; +import io.pravega.client.stream.StreamCut; +import io.pravega.client.stream.impl.ByteBufferSerializer; +import io.pravega.schemaregistry.client.SchemaRegistryClient; +import io.pravega.schemaregistry.client.SchemaRegistryClientConfig; +import io.pravega.schemaregistry.client.SchemaRegistryClientFactory; +import io.pravega.schemaregistry.serializer.shared.impl.SerializerConfig; + +import javax.inject.Inject; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import static java.util.Objects.requireNonNull; + +/** + * Manages segments and iterators. It uses the Pravega batch API to obtain iterators for + * a stream segment. One call obtains an iterator for segment ranges, where a segment + * range is a segment with start and end offsets. Given a segment range, a call to read + * the segment returns an iterator for the events between start and end offsets. + */ +public class PravegaSegmentManager +{ + private static final Logger log = Logger.get(PravegaSegmentManager.class); + private ClientConfig clientConfig; + private Map clientFactoryMap; + + private Map tableFactoryMap; + + private Map eventStreamClientFactoryMap; + + private Map scopedReaderGroupManagerMap; + + private SchemaRegistryClientConfig registryConfig; + private SchemaRegistryClient registryClient; + + @Inject + public PravegaSegmentManager(PravegaConnectorConfig connectorConfig) + { + requireNonNull(connectorConfig, "pravegaConnectorConfig is null"); + clientFactoryMap = new ConcurrentHashMap<>(); + tableFactoryMap = new ConcurrentHashMap<>(); + eventStreamClientFactoryMap = new ConcurrentHashMap<>(); + scopedReaderGroupManagerMap = new ConcurrentHashMap<>(); + clientConfig = ClientConfig.builder().controllerURI(connectorConfig.getControllerURI()).build(); + registryConfig = SchemaRegistryClientConfig.builder() + .schemaRegistryUri(connectorConfig.getSchemaRegistryURI()).build(); + registryClient = SchemaRegistryClientFactory.withDefaultNamespace(registryConfig); + } + + private BatchClientFactory batchClientFactory(String scope) + { + BatchClientFactory batchClientFactory = clientFactoryMap.get(scope); + if (batchClientFactory == null) { + synchronized (this) { + batchClientFactory = clientFactoryMap.get(scope); + if (batchClientFactory == null) { + batchClientFactory = BatchClientFactory.withScope(scope, clientConfig); + if (clientFactoryMap.putIfAbsent(scope, batchClientFactory) != null) { + throw new RuntimeException("unexpected concurrent create of batch client factory"); + } + } + } + } + return batchClientFactory; + } + + /** + * Returns a list of {@link SegmentRange} instances. + * + * @param scope scope where stream lives + * @param stream Stream to read from + * @param start The initial position in the stream + * @param end The end position in the stream + * @return A {@link SegmentRange} iterator + */ + StreamSegmentsIterator getSegments(String scope, String stream, StreamCut start, StreamCut end) + { + return batchClientFactory(scope).getSegments(Stream.of(scope, stream), start, end); + } + + /** + * Returns an iterator for the given segment range. It returns the bytes of the event + * and makes no attempt to deserialize it here. + * + * @param segmentRange The segment range to iterate over + * @param serializer The basic serializer to return the event bytes + * @return An iterator for the + */ + SegmentIterator getSegmentIterator(SegmentRange segmentRange, ByteBufferSerializer serializer) + { + return batchClientFactory(segmentRange.getScope()).readSegment(segmentRange, serializer); + } + + KeyValueTableFactory getKeyValueTableFactory(String scope) + { + KeyValueTableFactory factory = tableFactoryMap.get(scope); + if (factory == null) { + synchronized (this) { + factory = tableFactoryMap.get(scope); + if (factory == null) { + factory = KeyValueTableFactory.withScope(scope, clientConfig); + if (tableFactoryMap.putIfAbsent(scope, factory) != null) { + throw new RuntimeException("unexpected concurrent create of table factory"); + } + } + } + } + return factory; + } + + EventStreamClientFactory getEventStreamClientFactory(String scope) + { + EventStreamClientFactory factory = eventStreamClientFactoryMap.get(scope); + if (factory == null) { + synchronized (this) { + factory = eventStreamClientFactoryMap.get(scope); + if (factory == null) { + factory = EventStreamClientFactory.withScope(scope, clientConfig); + if (eventStreamClientFactoryMap.putIfAbsent(scope, factory) != null) { + throw new RuntimeException("unexpected concurrent create of event stream factory"); + } + } + } + } + return factory; + } + + ReaderGroupManager readerGroupManager(String scope) + { + ReaderGroupManager readerGroupManager = scopedReaderGroupManagerMap.get(scope); + if (readerGroupManager == null) { + synchronized (this) { + readerGroupManager = scopedReaderGroupManagerMap.get(scope); + if (readerGroupManager == null) { + readerGroupManager = ReaderGroupManager.withScope(scope, clientConfig); + if (scopedReaderGroupManagerMap.putIfAbsent(scope, readerGroupManager) != null) { + throw new RuntimeException("unexpected concurrent create of reader group manager"); + } + } + } + } + return readerGroupManager; + } + + SerializerConfig serializerConfig(String groupId) + { + return SerializerConfig.builder() + .registryClient(registryClient) + .groupId(groupId).build(); + } + + public boolean streamExists(String scope, String stream) + { + try (StreamManager streamManager = StreamManager.create(clientConfig.getControllerURI())) { + return streamManager.checkStreamExists(scope, stream); + } + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaSplit.java b/src/main/java/com/facebook/presto/pravega/PravegaSplit.java new file mode 100644 index 0000000..54eab76 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaSplit.java @@ -0,0 +1,126 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.HostAddress; +import com.facebook.presto.spi.schedule.NodeSelectionStrategy; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +import static com.facebook.presto.spi.schedule.NodeSelectionStrategy.NO_PREFERENCE; +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +/** + * Represents a Pravega specific {@link ConnectorSplit}. Each split is + * maps to a SegmentRange, which is a Pravega stream segment with a + * begin and end offsets. + */ +public class PravegaSplit + implements ConnectorSplit +{ + private final String connectorId; + private final ObjectType objectType; + private final List schema; + private final ReaderType readerType; + private final byte[] readerArgs; + private final String schemaRegistryGroupId; + + @JsonCreator + public PravegaSplit( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("objectType") ObjectType objectType, + @JsonProperty("schema") List schema, + @JsonProperty("readerType") ReaderType readerType, + @JsonProperty("readerArgs") byte[] readerArgs, + @JsonProperty("schemaRegistryGroupId") String schemaRegistryGroupId) + { + this.connectorId = requireNonNull(connectorId, "connector id is null"); + this.objectType = requireNonNull(objectType, "objectType is null"); + this.schema = requireNonNull(schema, "schema is null"); + this.readerType = requireNonNull(readerType, "readerType is null"); + this.readerArgs = requireNonNull(readerArgs, "readerArgs is null"); + this.schemaRegistryGroupId = requireNonNull(schemaRegistryGroupId, "schemaRegistryGroupId is null"); + } + + @JsonProperty + public String getConnectorId() + { + return connectorId; + } + + @JsonProperty + public ObjectType getObjectType() + { + return objectType; + } + + @JsonProperty + public ReaderType getReaderType() + { + return readerType; + } + + @JsonProperty + public byte[] getReaderArgs() + { + return readerArgs; + } + + @JsonProperty + public List getSchema() + { + return schema; + } + + @Override + public NodeSelectionStrategy getNodeSelectionStrategy() + { + return NO_PREFERENCE; + } + + @Override + public List getPreferredNodes(List sortedCandidates) + { + return ImmutableList.of(); + } + + @JsonProperty + public String getschemaRegistryGroupId() + { + return schemaRegistryGroupId; + } + + @Override + public Object getInfo() + { + return this; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("connectorId", connectorId) + .add("objectType", objectType) + .add("schema", schema) + .add("readerType", readerType) + .add("readerArgs", readerArgs) + .toString(); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaSplitManager.java b/src/main/java/com/facebook/presto/pravega/PravegaSplitManager.java new file mode 100644 index 0000000..3bb409b --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaSplitManager.java @@ -0,0 +1,289 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.airlift.log.Logger; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.ConnectorSplitSource; +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.facebook.presto.spi.FixedSplitSource; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.connector.ConnectorSplitManager; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.google.common.collect.ImmutableList; +import io.pravega.client.batch.SegmentRange; +import io.pravega.client.stream.ReaderGroupConfig; + +import javax.inject.Inject; + +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; + +import static com.facebook.presto.pravega.PravegaErrorCode.PRAVEGA_SPLIT_ERROR; +import static com.facebook.presto.pravega.PravegaHandleResolver.convertLayout; +import static com.facebook.presto.pravega.util.PravegaNameUtils.multiSourceStream; +import static com.facebook.presto.pravega.util.PravegaNameUtils.scopedName; +import static com.facebook.presto.pravega.util.PravegaSerializationUtils.serialize; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +/** + * Pravega specific implementation of {@link ConnectorSplitManager}. + */ +public class PravegaSplitManager + implements ConnectorSplitManager +{ + private static final Logger log = Logger.get(PravegaSegmentManager.class); + private final String connectorId; + private final PravegaConnectorConfig pravegaConnectorConfig; + private final PravegaSegmentManager streamReaderManager; + + @Inject + public PravegaSplitManager( + PravegaConnectorId connectorId, + PravegaConnectorConfig pravegaConnectorConfig, + PravegaSegmentManager streamReaderManager) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); + this.streamReaderManager = requireNonNull(streamReaderManager, "streamReaderManager is null"); + this.pravegaConnectorConfig = requireNonNull(pravegaConnectorConfig, "pravegaConnectorConfig is null"); + } + + @Override + public ConnectorSplitSource getSplits( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorTableLayoutHandle layout, + SplitSchedulingContext splitSchedulingContext) + { + PravegaTableHandle pravegaTableHandle = convertLayout(layout).getTable(); + ImmutableList.Builder splits = ImmutableList.builder(); + + try { + if (pravegaTableHandle.getObjectType() == ObjectType.KV_TABLE) { + buildKVSplits(pravegaTableHandle, splits); + } + else { + buildStreamSplits(new PravegaProperties(session), pravegaTableHandle, splits); + } + + return new FixedSplitSource(splits.build()); + } + catch (Exception e) { // Catch all exceptions because Pravega library is written in scala and checked exceptions are not declared in method signature. + if (e instanceof PrestoException) { + throw e; + } + throw new PrestoException(PRAVEGA_SPLIT_ERROR, + format("Cannot list splits for table '%s' reading stream '%s'", + pravegaTableHandle.getTableName(), pravegaTableHandle.getObjectName()), e); + } + } + + private static ReaderType readerType(PravegaProperties properties) + { + String type = properties.getReaderType(); + switch (type) { + case "event": + return ReaderType.EVENT_STREAM; + case "grouped_event": + return ReaderType.SINGLE_GROUP_EVENT_STREAM; + case "segment_range_per_split": + return ReaderType.SEGMENT_RANGE_PER_SPLIT; + default: + return ReaderType.SEGMENT_RANGE; + } + } + + private void buildKVSplits(PravegaTableHandle pravegaTableHandle, ImmutableList.Builder splits) + { + pravegaTableHandle.getOjectArgs().orElseThrow(() -> + new IllegalArgumentException("no KF defined for " + pravegaTableHandle)); + + for (String kf : pravegaTableHandle.getOjectArgs().get()) { + PravegaSplit split = + new PravegaSplit(connectorId, + ObjectType.KV_TABLE, + pravegaTableHandle.getSchema(), + ReaderType.KVT, + serialize(new PravegaKVTable(pravegaTableHandle.getSchemaName(), pravegaTableHandle.getObjectName(), kf)), + pravegaTableHandle.getSchemaRegistryGroupId()); + splits.add(split); + } + + log.info("created " + pravegaTableHandle.getOjectArgs().get().size() + " kv splits"); + } + + private void buildStreamSplits(final PravegaProperties properties, + PravegaTableHandle pravegaTableHandle, + ImmutableList.Builder splits) + { + // TODO: Enable begin and end cuts to be configurable: https://github.com/pravega/pravega-sql/issues/24 + List sourceStreams = multiSourceStream(pravegaTableHandle) + ? pravegaTableHandle.getOjectArgs().orElseThrow( + () -> new IllegalArgumentException("no args for multi source table found")) + : Collections.singletonList(pravegaTableHandle.getObjectName()); + + AtomicInteger splitCounter = new AtomicInteger(0); + ReaderType readerType = readerType(properties); + + sourceStreams.forEach(stream -> { + StreamCutSupplier streamCutSupplier = new StreamCutSupplier(streamReaderManager, pravegaTableHandle.getSchemaName(), stream); + + Supplier splitSupplier; + log.info("get split supplier for " + readerType); + switch (readerType) { + case EVENT_STREAM: + case SEGMENT_RANGE: + splitSupplier = splitSupplier(readerType, pravegaTableHandle, stream, streamCutSupplier); + break; + + case SINGLE_GROUP_EVENT_STREAM: + splitSupplier = groupedReaderSplitSupplier(readerType, pravegaTableHandle, stream, streamCutSupplier, properties.getGroupedEventSplits()); + break; + + case SEGMENT_RANGE_PER_SPLIT: + splitSupplier = segmentPerSplitSupplier(readerType, pravegaTableHandle, stream, streamCutSupplier); + break; + + default: + throw new IllegalArgumentException("" + readerType); + } + + PravegaSplit split = splitSupplier.get(); + do { + splits.add(split); + splitCounter.incrementAndGet(); + split = splitSupplier.get(); + } while (split != null); + }); + + log.info("created " + splitCounter.get() + " stream splits of type " + readerType); + } + + Supplier splitSupplier(final ReaderType readerType, + final PravegaTableHandle tableHandle, + final String stream, + final StreamCutSupplier streamCutSupplier) + { + return () -> { + StreamCutRange range = streamCutSupplier.get(); + if (range == null) { + return null; + } + + log.info(readerType + " split " + range); + + return new PravegaSplit( + connectorId, + ObjectType.STREAM, + Collections.singletonList(tableHandle.getSchema().get(0)), + readerType, + serialize(new ReaderArgs(tableHandle.getSchemaName(), stream, range, null)), + tableHandle.getSchemaRegistryGroupId()); + }; + } + + Supplier groupedReaderSplitSupplier(final ReaderType readerType, + final PravegaTableHandle tableHandle, + final String stream, + final StreamCutSupplier streamCutSupplier, + final int numSplits) + { + StreamCutRange first = streamCutSupplier.get(); + StreamCutRange last = null; + do { + StreamCutRange range = streamCutSupplier.get(); + if (range == null) { + break; + } + last = range; + } while (true); + + if (last == null) { + throw new IllegalStateException("no end split"); + } + StreamCutRange range = new StreamCutRange(first.getStart(), last.getEnd()); + + log.info(readerType + " split " + range); + String readerGroup = UUID.randomUUID().toString(); + + final ReaderArgs readerArgs = + new ReaderArgs(tableHandle.getSchemaName(), stream, range, readerGroup); + ReaderGroupConfig config = + ReaderGroupConfig.builder() + .stream(scopedName(readerArgs.getScope(), readerArgs.getStream()), + readerArgs.getStreamCutRange().getStart(), + readerArgs.getStreamCutRange().getEnd()) + .build(); + + log.info("create reader group " + readerGroup); + streamReaderManager.readerGroupManager( + tableHandle.getSchemaName()).createReaderGroup(readerGroup, config); + + final AtomicInteger splitCounter = new AtomicInteger(); + + return () -> { + if (splitCounter.getAndIncrement() == numSplits) { + return null; + } + return new PravegaSplit( + connectorId, + ObjectType.STREAM, + Collections.singletonList(tableHandle.getSchema().get(0)), + readerType, + serialize(readerArgs), + tableHandle.getSchemaRegistryGroupId()); + }; + } + + Supplier segmentPerSplitSupplier(final ReaderType readerType, + final PravegaTableHandle tableHandle, + final String stream, + final StreamCutSupplier streamCutSupplier) + { + final AtomicReference> iterator = new AtomicReference<>(); + + return () -> { + if (iterator.get() == null || !iterator.get().hasNext()) { + StreamCutRange range = streamCutSupplier.get(); + if (range == null) { + return null; + } + log.info(readerType + " split " + range); + iterator.set(streamReaderManager.getSegments(tableHandle.getSchemaName(), stream, range.getStart(), range.getEnd()).getIterator()); + if (iterator.get() == null || !iterator.get().hasNext()) { + log.info("no more splits"); + return null; + } + } + + SegmentRange segmentRange = iterator.get().next(); + log.info(readerType + " split " + segmentRange); + + return new PravegaSplit( + connectorId, + ObjectType.STREAM, + Collections.singletonList(tableHandle.getSchema().get(0)), + readerType, + serialize(segmentRange), + tableHandle.getSchemaRegistryGroupId()); + }; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaStreamDescription.java b/src/main/java/com/facebook/presto/pravega/PravegaStreamDescription.java new file mode 100644 index 0000000..bede43a --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaStreamDescription.java @@ -0,0 +1,105 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Optional; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static java.util.Objects.requireNonNull; + +/** + * Json description to parse a row on a Pravega stream. + */ +public class PravegaStreamDescription +{ + private final String tableName; + private final Optional schemaName; + private final String objectName; + private final ObjectType objectType; + private final Optional> objectArgs; + private final Optional> event; + + @JsonCreator + public PravegaStreamDescription( + @JsonProperty("tableName") String tableName, + @JsonProperty("schemaName") Optional schemaName, + @JsonProperty("objectName") String objectName, + @JsonProperty("objectType") Optional objectType, + @JsonProperty("objectArgs") Optional> objectArgs, + @JsonProperty("event") Optional> event) + { + checkArgument(!isNullOrEmpty(tableName), "tableName is null or is empty"); + this.tableName = tableName; + this.objectName = objectName; + this.objectType = objectType.orElse(ObjectType.STREAM); + this.objectArgs = requireNonNull(objectArgs, "objectArgs is null"); + this.schemaName = requireNonNull(schemaName, "schemaName is null"); + this.event = requireNonNull(event, "message is null"); + } + + @JsonProperty + public Optional getSchemaName() + { + return schemaName; + } + + @JsonProperty + public String getTableName() + { + return tableName; + } + + @JsonProperty + public String getObjectName() + { + return objectName; + } + + @JsonProperty + public ObjectType getObjectType() + { + return objectType; + } + + @JsonProperty + public Optional> getObjectArgs() + { + return objectArgs; + } + + @JsonProperty + public Optional> getEvent() + { + return event; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("schemaName", schemaName) + .add("tableName", tableName) + .add("objectName", objectName) + .add("objectType", objectType) + .add("objectArgs", objectArgs) + .add("event", event) + .toString(); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldDescription.java b/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldDescription.java new file mode 100644 index 0000000..b2377d2 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldDescription.java @@ -0,0 +1,160 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.common.type.Type; +import com.facebook.presto.spi.ColumnMetadata; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static java.util.Objects.requireNonNull; + +/** + * Json description to parse a single field from a Pravega stream message. See {@link com.facebook.presto.pravega.PravegaStreamDescription} for more details. + */ +public final class PravegaStreamFieldDescription +{ + private final String name; + private final Type type; + private final String mapping; + private final String comment; + private final String dataFormat; + private final String formatHint; + private final boolean hidden; + + @JsonCreator + public PravegaStreamFieldDescription( + @JsonProperty("name") String name, + @JsonProperty("type") Type type, + @JsonProperty("mapping") String mapping, + @JsonProperty("comment") String comment, + @JsonProperty("dataFormat") String dataFormat, + @JsonProperty("formatHint") String formatHint, + @JsonProperty("hidden") boolean hidden) + { + checkArgument(!isNullOrEmpty(name), "name is null or is empty"); + this.name = name; + this.type = requireNonNull(type, "type is null"); + this.mapping = mapping; + this.comment = comment; + this.dataFormat = dataFormat; + this.formatHint = formatHint; + this.hidden = hidden; + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public Type getType() + { + return type; + } + + @JsonProperty + public String getMapping() + { + return mapping; + } + + @JsonProperty + public String getComment() + { + return comment; + } + + @JsonProperty + public String getDataFormat() + { + return dataFormat; + } + + @JsonProperty + public String getFormatHint() + { + return formatHint; + } + + @JsonProperty + public boolean isHidden() + { + return hidden; + } + + PravegaColumnHandle getColumnHandle(String connectorId, boolean keyDecoder, int index, int schemaNum) + { + return new PravegaColumnHandle(connectorId, + index, + getName(), + getType(), + getMapping(), + getDataFormat(), + getFormatHint(), + keyDecoder, + isHidden(), + false, + schemaNum); + } + + ColumnMetadata getColumnMetadata() + { + return new ColumnMetadata(getName(), getType(), getComment(), isHidden()); + } + + @Override + public int hashCode() + { + return Objects.hash(name, type, mapping, dataFormat, formatHint, hidden); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + PravegaStreamFieldDescription other = (PravegaStreamFieldDescription) obj; + return Objects.equals(this.name, other.name) && + Objects.equals(this.type, other.type) && + Objects.equals(this.mapping, other.mapping) && + Objects.equals(this.dataFormat, other.dataFormat) && + Objects.equals(this.formatHint, other.formatHint) && + Objects.equals(this.hidden, other.hidden); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("name", name) + .add("type", type) + .add("mapping", mapping) + .add("dataFormat", dataFormat) + .add("formatHint", formatHint) + .add("hidden", hidden) + .toString(); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldGroup.java b/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldGroup.java new file mode 100644 index 0000000..6ca3a5f --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldGroup.java @@ -0,0 +1,82 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Optional; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +/** + * Groups the field descriptions for message or key. + */ +public class PravegaStreamFieldGroup +{ + private final String dataFormat; + private final Optional dataSchema; + private final Optional> fields; + private final Optional mapping; + + @JsonCreator + public PravegaStreamFieldGroup( + @JsonProperty("dataFormat") String dataFormat, + @JsonProperty("mapping") Optional mapping, + @JsonProperty("dataSchema") Optional dataSchema, + @JsonProperty("fields") Optional> fields) + { + this.dataFormat = requireNonNull(dataFormat, "dataFormat is null"); + this.mapping = mapping; + this.dataSchema = requireNonNull(dataSchema, "dataSchema is null"); + this.fields = fields; + } + + @JsonProperty + public String getDataFormat() + { + return dataFormat; + } + + @JsonProperty + public Optional getMapping() + { + return mapping; + } + + @JsonProperty + public List getFields() + { + return fields.orElse(null); + } + + @JsonProperty + public Optional getDataSchema() + { + return dataSchema; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("dataFormat", dataFormat) + .add("mapping", mapping) + .add("dataSchema", dataSchema) + .add("fields", fields) + .toString(); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaTableDescriptionSupplier.java b/src/main/java/com/facebook/presto/pravega/PravegaTableDescriptionSupplier.java new file mode 100644 index 0000000..9a12ad2 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaTableDescriptionSupplier.java @@ -0,0 +1,784 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.airlift.json.JsonCodec; +import com.facebook.airlift.log.Logger; +import com.facebook.presto.common.type.Type; +import com.facebook.presto.spi.SchemaTableName; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Streams; +import com.google.protobuf.Descriptors; +import io.pravega.client.ClientConfig; +import io.pravega.client.admin.StreamManager; +import io.pravega.client.stream.Stream; +import io.pravega.schemaregistry.client.SchemaRegistryClient; +import io.pravega.schemaregistry.client.SchemaRegistryClientConfig; +import io.pravega.schemaregistry.client.SchemaRegistryClientFactory; +import io.pravega.schemaregistry.contract.data.GroupProperties; +import io.pravega.schemaregistry.contract.data.SchemaWithVersion; +import io.pravega.schemaregistry.contract.data.SerializationFormat; +import io.pravega.schemaregistry.serializer.json.schemas.JSONSchema; +import org.everit.json.schema.BooleanSchema; +import org.everit.json.schema.NumberSchema; +import org.everit.json.schema.ObjectSchema; +import org.everit.json.schema.Schema; +import org.everit.json.schema.StringSchema; + +import javax.inject.Inject; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Locale; +import java.util.Optional; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import static com.facebook.presto.common.type.BigintType.BIGINT; +import static com.facebook.presto.common.type.BooleanType.BOOLEAN; +import static com.facebook.presto.common.type.DoubleType.DOUBLE; +import static com.facebook.presto.common.type.VarbinaryType.VARBINARY; +import static com.facebook.presto.common.type.VarcharType.createUnboundedVarcharType; +import static com.facebook.presto.pravega.ProtobufCommon.encodeSchema; +import static com.facebook.presto.pravega.util.PravegaNameUtils.groupId; +import static com.facebook.presto.pravega.util.PravegaNameUtils.kvFieldMapping; +import static com.facebook.presto.pravega.util.PravegaNameUtils.kvTable; +import static com.facebook.presto.pravega.util.PravegaNameUtils.multiSourceStream; +import static com.facebook.presto.pravega.util.PravegaNameUtils.temp_streamNameToTableName; +import static com.facebook.presto.pravega.util.PravegaNameUtils.temp_tableNameToStreamName; +import static com.facebook.presto.pravega.util.PravegaSchemaUtils.AVRO; +import static com.facebook.presto.pravega.util.PravegaSchemaUtils.GROUP_PROPERTIES_INLINE_KEY; +import static com.facebook.presto.pravega.util.PravegaSchemaUtils.GROUP_PROPERTIES_INLINE_KV_KEY; +import static com.facebook.presto.pravega.util.PravegaSchemaUtils.GROUP_PROPERTIES_INLINE_KV_VALUE; +import static com.facebook.presto.pravega.util.PravegaSchemaUtils.INLINE_SUFFIX; +import static com.facebook.presto.pravega.util.PravegaSchemaUtils.NESTED_RECORD_SEPARATOR; +import static com.facebook.presto.pravega.util.PravegaSchemaUtils.readSchema; +import static java.nio.file.Files.readAllBytes; +import static java.util.Arrays.asList; +import static java.util.Objects.requireNonNull; +import static org.apache.avro.Schema.Type.RECORD; + +// pravega scope is a namespace for streams. stream is unique within scope. +// presto schema is like a database, with collection of tables. +// we will map scope->schema and stream->table +// scope will be our database and streams will be tables in the database +// +// a stream schema could be "local". local definitions take precedence. +// .json file in known directory with naming format ..json +// there could be any number of local schemas. +// +// additionally a local stream schema could be a composite table. this is called 'multi-source'. +// stream name will be a regex. it can match 1 or more source streams. when this single table is +// queried we will consider events from all source streams +public class PravegaTableDescriptionSupplier +{ + private static final Logger log = Logger.get(PravegaTableDescriptionSupplier.class); + + private final PravegaConnectorConfig pravegaConnectorConfig; + + private Cache schemaCache; + + private Cache> tableCache; + + private JsonCodec streamDescriptionCodec; + + // "inline" means that event was written using schema registry wrapped client and schema encoding id + // is within the raw event data in pravega + @Inject + PravegaTableDescriptionSupplier(PravegaConnectorConfig pravegaConnectorConfig, + JsonCodec streamDescriptionCodec) + { + requireNonNull(pravegaConnectorConfig, "pravegaConfig is null"); + this.pravegaConnectorConfig = pravegaConnectorConfig; + this.streamDescriptionCodec = streamDescriptionCodec; + + // there will be many successive calls to listSchemas + listTables in short time period + // do not reach out to pravega each time as it is unlikely things would have changed + // enhancement issue - can we determine if there are changes/removals and selectively update? + // https://github.com/StreamingDataPlatform/pravega-sql/issues/101 + this.schemaCache = CacheBuilder.newBuilder() + .expireAfterWrite(pravegaConnectorConfig.getTableCacheExpireSecs(), TimeUnit.SECONDS) + .build(); + + this.tableCache = CacheBuilder.newBuilder() + .expireAfterWrite(pravegaConnectorConfig.getTableCacheExpireSecs(), TimeUnit.SECONDS) + .build(); + } + + public List listSchemas() + { + // if any expired, retrieve list again from pravega + // they are inserted to cache at same time so will all be same state + final List schemas = schemaCache.asMap().keySet().stream().collect(Collectors.toList()); + if (schemas.isEmpty()) { + listLocalSchemas().forEach(schema -> schemaCache.put(schema, new Object())); + + try (StreamManager streamManager = + StreamManager.create(ClientConfig.builder().controllerURI( + pravegaConnectorConfig.getControllerURI()).build())) { + Streams.stream(streamManager.listScopes()).filter(s -> !internalObject(s)).forEach(schema -> { + schemas.add(schema); + schemaCache.put(schema, new Object()); + }); + } + } + else { + log.info("serving listSchemas() from cache"); + } + return schemas; + } + + public List listTables(Optional schema) + { + List schemas = schema.isPresent() ? Collections.singletonList(schema.get()) : listSchemas(); + + StreamManager streamManager = null; + + try { + List tableList = new ArrayList<>(); + + for (String s : schemas) { + List tableListForSchema = + tableCache.asMap().keySet().stream() + .filter(streamDesc -> streamDesc.getSchemaTableName().getSchemaName().startsWith(s)) + .collect(Collectors.toList()); + + // not all tables inserted to cache at same time + if (tableListForSchema.isEmpty()) { + if (streamManager == null) { + streamManager = StreamManager.create( + ClientConfig.builder().controllerURI(pravegaConnectorConfig.getControllerURI()).build()); + } + + List compositeStreams = new ArrayList<>(); + + // local takes precedence. list before pravega. ifAbsent used later to not clobber. + listLocalTables(s).forEach(table -> { + PravegaTableName pravegaTableName = new PravegaTableName(s, table); + + // don't clobber existing entry + if (tableCache.getIfPresent(pravegaTableName) == null || + !tableCache.getIfPresent(pravegaTableName).isPresent()) { + tableCache.put(pravegaTableName, Optional.empty()); + } + + // load .json def to get stream name in order to determine type + PravegaStreamDescription localTable = getLocalTable(pravegaTableName.getSchemaTableName()); + if (multiSourceStream(localTable)) { + compositeStreams.add(Pattern.compile(localTable.getObjectName())); + } + }); + + // (underlying streams used by kv table are seen as internal and thus are skipped) + Streams.stream(streamManager.listStreams(s)) + .filter(stream -> !internalStream(stream)) + .forEach(stream -> { + boolean hidden = + compositeStreams.stream().anyMatch(p -> p.matcher(stream.getStreamName()).matches()); + // ifAbsent - don't clobber table description if we have it + PravegaTableName tableName = new PravegaTableName(s, temp_streamNameToTableName(stream.getStreamName()), hidden); + if (tableCache.getIfPresent(tableName) == null || + !tableCache.getIfPresent(tableName).isPresent()) { + tableCache.put(tableName, Optional.empty()); + } + }); + } + else { + log.info("serving listTables(%s) from cache", s); + } + + tableList.addAll(tableCache.asMap().keySet().stream() + .filter(pravegaStreamDescription -> + pravegaStreamDescription.getSchemaTableName().getSchemaName().startsWith(s)) + .collect(Collectors.toList())); + } + return tableList; + } + finally { + if (streamManager != null) { + streamManager.close(); + } + } + } + + public PravegaStreamDescription getTable(SchemaTableName schemaTableName) + { + PravegaTableName pravegaTableName = new PravegaTableName(schemaTableName); + Optional cachedTable = tableCache.getIfPresent(pravegaTableName); + if (cachedTable != null && cachedTable.isPresent()) { + log.info("serving getTable(%s) from cache", schemaTableName); + return cachedTable.get(); + } + + PravegaStreamDescription table = getLocalTable(schemaTableName); + if (table != null) { + log.info("found local schema for '%s'", schemaTableName); + + // kv this is list of key family (defined in local schema file) + // for multi source stream this is list of composite streams (empty here, to be filled in later) + Optional> objectArgs = table.getObjectArgs(); + + // field definitions can come from 1 of 4 places + // (1) defined in local .json schema ("event/fields") + // (2) uri in "dataSchema" field + // (3) lookup from a source stream (if multi source stream) + // (4) lookup directly in schema registry (if kv table) + + Optional> fieldGroups = Optional.empty(); + + if (fieldsDefined(table)) { + // case (1) - no-op + log.info("fields defined in schema file %s", schemaTableName); + fieldGroups = Optional.of(new LinkedList<>(table.getEvent().get())); + } + else if (table.getEvent().isPresent() && + table.getEvent().get().get(0).getDataSchema().isPresent()) { + fieldGroups = Optional.of(new LinkedList<>()); + + // case (2) uri containing schema + List finalFieldGroups = fieldGroups.get(); + for (int i = 0; i < table.getEvent().get().size(); i++) { + PravegaStreamFieldGroup event = table.getEvent().get().get(i); + String colPrefix = event.getMapping().orElse( + table.getEvent().get().size() > 1 ? kvFieldMapping(i) : ""); + Optional dataSchema = Optional.of(readSchema(event.getDataSchema().get())); + PravegaStreamFieldGroup fieldGroup = + new PravegaStreamFieldGroup(event.getDataFormat(), + Optional.empty(), + dataSchema, + Optional.of( + mapFieldsFromSchema(colPrefix, event.getDataFormat(), dataSchema.get()))); + finalFieldGroups.add(fieldGroup); + } + } + else if (kvTable(table)) { + fieldGroups = fieldGroupsFromSchemaRegistry(schemaTableName); + } + + if (multiSourceStream(table)) { + // stream name will be some regex. + // find all the possible source streams. + Pattern pattern = Pattern.compile(table.getObjectName()); + + List sourceTableNames = + listTables(Optional.of(schemaTableName.getSchemaName())).stream() + .filter(t -> pattern.matcher(t.getSchemaTableName().getTableName()).matches()) + .collect(Collectors.toList()); + + objectArgs = Optional.of(sourceTableNames.stream() + .map(PravegaTableName::getSchemaTableName) + .map(SchemaTableName::getTableName) + .collect(Collectors.toList())); + + if (!fieldGroups.isPresent()) { + // case (3) schema not already defined, look one up + // lookup actual schema from any of them - implies all sources are the same + PravegaStreamDescription sourceTable = sourceTableNames.isEmpty() + ? null + : getTable(sourceTableNames.get(0).getSchemaTableName()); + if (sourceTable == null) { + throw new IllegalArgumentException("no stream found for multi source"); + } + fieldGroups = Optional.of(new LinkedList<>()); + fieldGroups.get().add(new PravegaStreamFieldGroup( + sourceTable.getEvent().get().get(0).getDataFormat(), + Optional.empty(), + sourceTable.getEvent().get().get(0).getDataSchema(), + Optional.of(sourceTable.getEvent().get().get(0).getFields()))); + } + } + + fieldGroups.orElseThrow(() -> + new IllegalArgumentException("unable to determine schema for " + schemaTableName)); + + // our final table definition. use schema that we looked up, and set all source stream names here + table = new PravegaStreamDescription(schemaTableName.getTableName(), + Optional.of(schemaTableName.getSchemaName()), + table.getObjectName(), + Optional.of(table.getObjectType()), + objectArgs, + fieldGroups); + + tableCache.put(pravegaTableName, Optional.of(table)); + return table; + } + + Optional> fieldGroups = fieldGroupsFromSchemaRegistry(schemaTableName); + + table = new PravegaStreamDescription( + schemaTableName.getTableName(), + Optional.of(schemaTableName.getSchemaName()), + temp_tableNameToStreamName(schemaTableName.getTableName()), + Optional.of(ObjectType.STREAM), + Optional.empty() /* args */, + fieldGroups); + tableCache.put(pravegaTableName, Optional.of(table)); + return table; + } + + private Optional> fieldGroupsFromSchemaRegistry(final SchemaTableName schemaTableName) + { + log.info("look up description of '%s' from pravega", schemaTableName); + String groupName = groupId(schemaTableName.getSchemaName(), temp_tableNameToStreamName(schemaTableName.getTableName())); + + SchemaRegistryClientConfig registryConfig = + SchemaRegistryClientConfig.builder() + .schemaRegistryUri(pravegaConnectorConfig.getSchemaRegistryURI()).build(); + SchemaRegistryClient registryClient = SchemaRegistryClientFactory.withDefaultNamespace(registryConfig); + + List fieldGroups = new ArrayList<>(2); + + GroupProperties properties = + registryClient.getGroupProperties(groupName); + + List schemas = + registryClient.getSchemas(groupName); + if (schemas.size() == 0 || schemas.size() > 2) { + throw new IllegalStateException(schemaTableName + " has " + schemas.size() + " registered schemas. expecting either 1 or 2"); + } + + for (int i = 0; i < schemas.size(); i++) { + SerializationFormat format = schemas.get(i).getSchemaInfo().getSerializationFormat(); + + // kv table will have > 1 schema. key+value likely different types + // colPrefix used for display so can differentiate between fields from key or value + boolean kv = schemas.size() > 1; + String colPrefix = kv ? kvFieldMapping(i) : ""; + fieldGroups.add(new PravegaStreamFieldGroup( + dataFormat(properties.getProperties(), format, kv, i), + Optional.of(colPrefix), + dataSchema(format, schemas.get(i)), + Optional.of(mapFieldsFromSchema(colPrefix, format, schemas.get(i))))); + } + + return Optional.of(fieldGroups); + } + + private static boolean fieldsDefined(PravegaStreamDescription table) + { + // event is optional, fields within event is also optional + // for kv table - 0 or 2 schemas. so fine to just check for 1. + return table.getEvent().isPresent() && (table.getEvent().get().get(0).getFields() != null); + } + + private List listLocalSchemas() + { + return localSchemaStream() + .map(file -> file.getName().split("\\.")[0]) + .collect(Collectors.toList()); + } + + // scope.stream -> schema.table + private List listLocalTables(String schema) + { + return localSchemaStream() + .filter(file -> file.getName().endsWith(".json")) + .filter(file -> file.getName().startsWith(schema)) + .filter(file -> file.getName().split("\\.").length == 3) + .map(file -> file.getName().split("\\.")[1]) + .collect(Collectors.toList()); + } + + private PravegaStreamDescription getLocalTable(SchemaTableName schemaTableName) + { + try { + File file = new File(pravegaConnectorConfig.getTableDescriptionDir(), + String.format("%s.%s.json", schemaTableName.getSchemaName(), schemaTableName.getTableName())); + if (!file.exists()) { + return null; + } + return streamDescriptionCodec.fromJson(readAllBytes(file.toPath())); + } + catch (IOException e) { + log.error("%s", e); + throw new UncheckedIOException(e); + } + catch (RuntimeException e) { + log.error("%s", e); + throw e; + } + } + + private java.util.stream.Stream localSchemaStream() + { + return listFiles(pravegaConnectorConfig.getTableDescriptionDir()).stream() + .filter(file -> file.isFile() && file.getName().endsWith(".json")); + } + + private static List listFiles(File dir) + { + if ((dir != null) && dir.isDirectory()) { + File[] files = dir.listFiles(); + if (files != null) { + log.debug("Considering files: %s", asList(files)); + return ImmutableList.copyOf(files); + } + } + return ImmutableList.of(); + } + + private static String dataFormat(ImmutableMap groupProperties, + SerializationFormat format, + boolean kvTable, + int kvIdx) + { + /* + TODO: auto-detect https://github.com/pravega/pravega-sql/issues/58 + + (1) no schema registry. + (2) Register and evolve schemas in registry but do not use registry client while writing data + (3) Register schemas in the registry and use registry client to encode schema Id with payload + "inline" is for #3. for e.g. "avro" -> "avro-inline". PravegaRecordSetProvider is interested in this + + hopefully this can all go away (see linked issue 58 above) + + but for now the following is our convention + if "inline" exists in our properties, all data uses SR + else if it is a kv table key+value may be different. both, neither, or either may use SR + look for "inlinekey" / "inlinevalue" + */ + + String key = GROUP_PROPERTIES_INLINE_KEY; + + if (kvTable && !groupProperties.containsKey(key)) { + key = kvIdx == 0 ? GROUP_PROPERTIES_INLINE_KV_KEY : GROUP_PROPERTIES_INLINE_KV_VALUE; + } + + String finalFormat = format == SerializationFormat.Custom + ? format.getFullTypeName().toLowerCase(Locale.ENGLISH) + : format.name().toLowerCase(Locale.ENGLISH); + return finalFormat + (groupProperties.containsKey(key) ? INLINE_SUFFIX : ""); + } + + /** + * map protobuf java type -> presto sql type + * + * @param fieldDescriptor + * @return + */ + private static Type typeFromSchema(Descriptors.FieldDescriptor fieldDescriptor) + { + switch (fieldDescriptor.getJavaType()) { + case STRING: + return createUnboundedVarcharType(); + + case INT: + case LONG: + return BIGINT; + + case FLOAT: + case DOUBLE: + return DOUBLE; + + case BOOLEAN: + return BOOLEAN; + + case BYTE_STRING: + return VARBINARY; + + default: + throw new RuntimeException("unsupported type " + fieldDescriptor); + } + } + + /** + * map json schema type -> presto sql type + * + * @param schema + * @return + */ + private static Type typeFromSchema(Schema schema) + { + if (schema instanceof NumberSchema) { + return ((NumberSchema) schema).requiresInteger() + ? BIGINT + : DOUBLE; + } + else if (schema instanceof BooleanSchema) { + return BOOLEAN; + } + else if (schema instanceof StringSchema) { + return createUnboundedVarcharType(); + } + else { + throw new RuntimeException("unsupported schema " + schema); + } + } + + /** + * map avro schema type to presto sql type + * + * @param schema + * @return + */ + private static Type typeFromSchema(org.apache.avro.Schema schema) + { + // refer to AvroColumnDecoder#isSupportedType + + switch (schema.getType()) { + case FIXED: + case STRING: + return createUnboundedVarcharType(); + + case INT: + case LONG: + return BIGINT; + + case FLOAT: + case DOUBLE: + return DOUBLE; + + case BOOLEAN: + return BOOLEAN; + + case BYTES: + return VARBINARY; + + case MAP: + case ARRAY: + // TODO: ^^ handle these https://github.com/pravega/pravega-sql/issues/65 + + case RECORD: + case ENUM: + case UNION: + default: + throw new RuntimeException("unexpected type " + schema); + } + } + + /** + * return lists of common field definitions + * uses list of fields from provided schema; schema is different depending on serialization format + * + * @param format + * @param schemaWithVersion + * @return + */ + private static List mapFieldsFromSchema( + String namePrefix, + SerializationFormat format, + SchemaWithVersion schemaWithVersion) + { + switch (format) { + case Json: + ObjectSchema objectSchema = + (ObjectSchema) JSONSchema.from(schemaWithVersion.getSchemaInfo()).getSchema(); + return mapTable(namePrefix, new JsonSchema(objectSchema)); + + case Avro: + case Custom: // re: Custom - definition for schema itself Custom is always Avro (only custom impl. is csv) + org.apache.avro.Schema schema = + new org.apache.avro.Schema.Parser().parse( + new String(schemaWithVersion.getSchemaInfo().getSchemaData().array(), StandardCharsets.UTF_8)); + return mapTable(namePrefix, new AvroSchema(schema, format == SerializationFormat.Custom)); + + case Protobuf: + return mapTable(namePrefix, new ProtobufSchema(ProtobufCommon.descriptorFor(schemaWithVersion))); + + default: + throw new IllegalArgumentException("unexpected format " + format); + } + } + + private static List mapFieldsFromSchema(String namePrefix, String format, String schemaString) + { + // schemaString defined as human-readable string in local file. only avro supported now. + switch (format) { + case AVRO: + org.apache.avro.Schema schema = + new org.apache.avro.Schema.Parser().parse(schemaString); + return mapTable(namePrefix, new AvroSchema(schema, false)); + + default: + throw new UnsupportedOperationException("unexpected format " + format); + } + } + + private static class SchemaColumn + { + String name; + String mapping; + Type type; + + SchemaColumn(String name, String mapping, Type type) + { + this.name = name; + this.mapping = mapping; + this.type = type; + } + } + + static class SchemaWrapper + { + List fields = new ArrayList<>(); + } + + static class SchemaField + { + String name; + Type type; + boolean record; + SchemaWrapper schema; + int ordinalPosition; + + SchemaField(String name, Type type, boolean record, SchemaWrapper schema) + { + this(name, type, record, schema, -1); + } + + SchemaField(String name, Type type, boolean record, SchemaWrapper schema, int ordinalPosition) + { + this.name = name; + this.type = type; + this.record = record; + this.schema = schema; + this.ordinalPosition = ordinalPosition; + } + } + + static class JsonSchema + extends SchemaWrapper + { + JsonSchema(ObjectSchema schema) + { + schema.getPropertySchemas().forEach((key, value) -> { + boolean record = value instanceof ObjectSchema; + fields.add(new SchemaField(key, + record ? null : typeFromSchema(value), + record, + record ? new JsonSchema((ObjectSchema) value) : null)); + }); + } + } + + static class ProtobufSchema + extends SchemaWrapper + { + ProtobufSchema(Descriptors.Descriptor schema) + { + schema.getFields().forEach(f -> { + boolean record = f.getJavaType() == Descriptors.FieldDescriptor.JavaType.MESSAGE; + fields.add(new SchemaField(f.getJsonName(), + record ? null : typeFromSchema(f), + record, + record ? new ProtobufSchema(f.getMessageType()) : null)); + }); + } + } + + static class AvroSchema + extends SchemaWrapper + { + AvroSchema(org.apache.avro.Schema schema, boolean customCsv) + { + final AtomicInteger position = new AtomicInteger(); + schema.getFields().forEach(f -> { + boolean record = f.schema().getType() == RECORD; + fields.add(new SchemaField(f.name(), + record ? null : typeFromSchema(f.schema()), + record, + record ? new AvroSchema(f.schema(), customCsv) : null, + customCsv ? position.getAndIncrement() : -1)); + }); + } + } + + private static List mapTable(String namePrefix, SchemaWrapper schema) + { + return mapFieldsFromSchema(mapColumns(namePrefix, null /* mappingPrefix */, schema)); + } + + private static List mapColumns(String namePrefix, String mappingPrefix, SchemaWrapper schema) + { + List columnList = new ArrayList<>(); + schema.fields.forEach(field -> { + String name = nestedPrefixFor(namePrefix, field.name); + // for csv we use only position. for avro, json, etc, can be path into nested object + String mapping = field.ordinalPosition >= 0 + ? String.valueOf(field.ordinalPosition) + : nestedPrefixFor(mappingPrefix, field.name); + if (field.record) { + columnList.addAll(mapColumns(name, mapping, field.schema)); + } + else { + columnList.add(new SchemaColumn(name, mapping, field.type)); + } + }); + return columnList; + } + + private static String nestedPrefixFor(String prefix, String name) + { + // (record1, field1) -> record1/field1 + return prefix == null || prefix.isEmpty() + ? name + : prefix + NESTED_RECORD_SEPARATOR + name; + } + + /** + * create field description from list of name,mapping,type tuples. each pair is a field in the schema. + * @param schemaColumns + * @return + */ + static List mapFieldsFromSchema(List schemaColumns) + { + List fields = new ArrayList<>(); + schemaColumns.forEach(sc -> { + fields.add(new PravegaStreamFieldDescription(sc.name, + sc.type, + sc.mapping, + "", + null, + null, + false)); + }); + return fields; + } + + private static Optional dataSchema(SerializationFormat format, SchemaWithVersion schemaWithVersion) + { + // it is intentional that nothing is returned for Custom + // pass schema to row decoders. refer to PravegaRecordSetProvider + switch (format) { + case Protobuf: + return Optional.of(encodeSchema(schemaWithVersion)); + case Avro: + return Optional.of(new String(schemaWithVersion.getSchemaInfo().getSchemaData().array(), StandardCharsets.UTF_8)); + default: + return Optional.empty(); + } + } + + private static boolean internalStream(Stream stream) + { + return internalObject(stream.getStreamName()); + } + + private static boolean internalObject(String object) + { + return object.startsWith("_") /* pravega internal */ || + object.endsWith("-SC") /* application internal - stream cuts */; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaTableHandle.java b/src/main/java/com/facebook/presto/pravega/PravegaTableHandle.java new file mode 100644 index 0000000..c79e769 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaTableHandle.java @@ -0,0 +1,173 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.spi.ConnectorTableHandle; +import com.facebook.presto.spi.SchemaTableName; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +/** + * Pravega specific {@link ConnectorTableHandle}. + */ +public final class PravegaTableHandle + implements ConnectorTableHandle +{ + /** + * connector id + */ + private final String connectorId; + + /** + * The schema name for this table. Is set through configuration and read + */ + private final String schemaName; + + private final ObjectType objectType; + + /** + * The table name used by presto. + */ + private final String tableName; + + /** + * The stream or kv table name that is read from Pravega. + */ + private final String objectName; + + private final Optional> objectArgs; + + private final List schema; + + private final String schemaRegistryGroupId; + + @JsonCreator + public PravegaTableHandle( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("schemaName") String schemaName, + @JsonProperty("tableName") String tableName, + @JsonProperty("objectName") String objectName, + @JsonProperty("objectType") ObjectType objectType, + @JsonProperty("objectArgs") Optional> objectArgs, + @JsonProperty("schema") List schema, + @JsonProperty("schemaRegistryGroupId") String schemaRegistryGroupId) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null"); + this.schemaName = requireNonNull(schemaName, "schemaName is null"); + this.tableName = requireNonNull(tableName, "tableName is null"); + this.objectName = requireNonNull(objectName, "objectName is null"); + this.objectType = requireNonNull(objectType, "objectType is null"); + this.objectArgs = requireNonNull(objectArgs, "objectArgs is null"); + this.schema = requireNonNull(schema, "schema is null"); + this.schemaRegistryGroupId = requireNonNull(schemaRegistryGroupId, "schemaRegistryGroupId is null"); + } + + @JsonProperty + public String getConnectorId() + { + return connectorId; + } + + @JsonProperty + public String getSchemaName() + { + return schemaName; + } + + @JsonProperty + public String getTableName() + { + return tableName; + } + + @JsonProperty + public ObjectType getObjectType() + { + return objectType; + } + + @JsonProperty + public String getObjectName() + { + return objectName; + } + + @JsonProperty + public Optional> getOjectArgs() + { + return objectArgs; + } + + @JsonProperty + public List getSchema() + { + return schema; + } + + @JsonProperty + public String getSchemaRegistryGroupId() + { + return schemaRegistryGroupId; + } + + public SchemaTableName toSchemaTableName() + { + return new SchemaTableName(schemaName, tableName); + } + + @Override + public int hashCode() + { + return Objects.hash(connectorId, schemaName, tableName, objectName, objectType, schema); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + PravegaTableHandle other = (PravegaTableHandle) obj; + return Objects.equals(this.connectorId, other.connectorId) + && Objects.equals(this.schemaName, other.schemaName) + && Objects.equals(this.tableName, other.tableName) + && Objects.equals(this.objectName, other.objectName) + && Objects.equals(this.objectType, other.objectType) + && Objects.equals(this.schema, other.schema); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("connectorId", connectorId) + .add("schemaName", schemaName) + .add("tableName", tableName) + .add("objectName", objectName) + .add("objectType", objectType) + .add("schema", schema) + .toString(); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaTableLayoutHandle.java b/src/main/java/com/facebook/presto/pravega/PravegaTableLayoutHandle.java new file mode 100644 index 0000000..ca8184e --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaTableLayoutHandle.java @@ -0,0 +1,48 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +public class PravegaTableLayoutHandle + implements ConnectorTableLayoutHandle +{ + private final PravegaTableHandle table; + + @JsonCreator + public PravegaTableLayoutHandle( + @JsonProperty("table") PravegaTableHandle table) + { + this.table = requireNonNull(table, "table is null"); + } + + @JsonProperty + public PravegaTableHandle getTable() + { + return table; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("table", table.toString()) + .toString(); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaTableName.java b/src/main/java/com/facebook/presto/pravega/PravegaTableName.java new file mode 100644 index 0000000..c522f6d --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaTableName.java @@ -0,0 +1,81 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.spi.SchemaTableName; + +import java.util.Objects; + +public class PravegaTableName +{ + private final SchemaTableName schemaTableName; + private final boolean hidden; + + public PravegaTableName(String schema, String table) + { + this(new SchemaTableName(schema, table), false); + } + + // ADR: string 'stream name' for mapping table name to stream. can be == + public PravegaTableName(String schema, String table, boolean hidden) + { + this(new SchemaTableName(schema, table), hidden); + } + + public PravegaTableName(SchemaTableName schemaTableName) + { + this(schemaTableName, false); + } + + public PravegaTableName(SchemaTableName schemaTableName, boolean hidden) + { + this.schemaTableName = schemaTableName; + this.hidden = hidden; + } + + public SchemaTableName getSchemaTableName() + { + return schemaTableName; + } + + public boolean getHidden() + { + return hidden; + } + + @Override + public int hashCode() + { + return schemaTableName.hashCode(); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final PravegaTableName other = (PravegaTableName) obj; + return Objects.equals(this.schemaTableName, other.schemaTableName); + } + + @Override + public String toString() + { + return schemaTableName.toString() + "(" + hidden + ")"; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/PravegaTransactionHandle.java b/src/main/java/com/facebook/presto/pravega/PravegaTransactionHandle.java new file mode 100644 index 0000000..8f4dd7f --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/PravegaTransactionHandle.java @@ -0,0 +1,22 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; + +public enum PravegaTransactionHandle + implements ConnectorTransactionHandle +{ + INSTANCE +} diff --git a/src/main/java/com/facebook/presto/pravega/ProtobufCommon.java b/src/main/java/com/facebook/presto/pravega/ProtobufCommon.java new file mode 100644 index 0000000..2efd897 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/ProtobufCommon.java @@ -0,0 +1,117 @@ +/* + * 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. + */ + +/* + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * 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 + */ +package com.facebook.presto.pravega; + +import com.google.common.base.Strings; +import com.google.protobuf.DescriptorProtos; +import com.google.protobuf.Descriptors; +import io.pravega.schemaregistry.common.NameUtil; +import io.pravega.schemaregistry.contract.data.SchemaWithVersion; +import org.apache.commons.lang3.SerializationException; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Base64; + +public class ProtobufCommon +{ + private ProtobufCommon() + { + } + + public static String encodeSchema(final SchemaWithVersion schemaWithVersion) + { + return schemaWithVersion.getSchemaInfo().getType() + "|" + + Base64.getEncoder().encodeToString(schemaWithVersion.getSchemaInfo().getSchemaData().array()); + } + + public static Pair decodeSchema(String encodedSchema) + { + String[] parts = encodedSchema.split("\\|"); + return new ImmutablePair<>(parts[0], ByteBuffer.wrap(Base64.getDecoder().decode((parts[1])))); + } + + public static Descriptors.Descriptor descriptorFor(final SchemaWithVersion schemaWithVersion) + { + return descriptorFor(schemaWithVersion.getSchemaInfo().getType(), + schemaWithVersion.getSchemaInfo().getSchemaData()); + } + + // TODO: following code block is from schema registry/serializers. license added above + // looking into possibly making this common/exposed in SR libs. + public static Descriptors.Descriptor descriptorFor(String type, final ByteBuffer buffer) + { + DescriptorProtos.FileDescriptorSet descriptorSet; + try { + descriptorSet = DescriptorProtos.FileDescriptorSet.parseFrom(buffer); + } + catch (IOException e) { + throw new IllegalArgumentException(e); + } + + int count = descriptorSet.getFileCount(); + String[] tokens = NameUtil.extractNameAndQualifier(type); + String name = tokens[0]; + String pckg = tokens[1]; + DescriptorProtos.FileDescriptorProto mainDescriptor = null; + for (DescriptorProtos.FileDescriptorProto x : descriptorSet.getFileList()) { + boolean packageMatch; + if (x.getPackage() == null) { + packageMatch = Strings.isNullOrEmpty(pckg); + } + else { + packageMatch = x.getPackage().equals(pckg); + } + if (packageMatch && x.getMessageTypeList().stream().anyMatch(y -> y.getName().equals(name))) { + mainDescriptor = x; + break; + } + } + if (mainDescriptor == null) { + throw new IllegalArgumentException("FileDescriptorSet doesn't contain the schema for the object type."); + } + + Descriptors.FileDescriptor[] dependencyArray = new Descriptors.FileDescriptor[count]; + Descriptors.FileDescriptor fd; + try { + for (int i = 0; i < count; i++) { + fd = Descriptors.FileDescriptor.buildFrom( + descriptorSet.getFile(i), + new Descriptors.FileDescriptor[]{}); + dependencyArray[i] = fd; + } + fd = Descriptors.FileDescriptor.buildFrom(mainDescriptor, dependencyArray); + } + catch (Descriptors.DescriptorValidationException e) { + throw new IllegalArgumentException("Invalid protobuf schema."); + } + + return fd.getMessageTypes().stream() + .filter(x -> x.getName().equals(name)).findAny() + .orElseThrow(() -> + new SerializationException(String.format("schema for %s not found", type))); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/ReaderArgs.java b/src/main/java/com/facebook/presto/pravega/ReaderArgs.java new file mode 100644 index 0000000..93ce705 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/ReaderArgs.java @@ -0,0 +1,76 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.Serializable; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +public class ReaderArgs + implements Serializable +{ + private final String scope; + private final String stream; + private final StreamCutRange streamCutRange; + private final String readerGroup; + + public ReaderArgs(@JsonProperty("scope") String scope, + @JsonProperty("stream") String stream, + @JsonProperty("streamCutRange") StreamCutRange streamCutRange, + @JsonProperty("readerGroup") String readerGroup) + { + this.scope = requireNonNull(scope, "scope is null"); + this.stream = requireNonNull(stream, "stream is null"); + this.streamCutRange = requireNonNull(streamCutRange, "streamCutRange is null"); + this.readerGroup = readerGroup; // may be null + } + + @JsonProperty + public String getScope() + { + return scope; + } + + @JsonProperty + public String getStream() + { + return stream; + } + + @JsonProperty + public StreamCutRange getStreamCutRange() + { + return streamCutRange; + } + + @JsonProperty + public String getReaderGroup() + { + return readerGroup; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("scope", scope) + .add("stream", stream) + .add("streamCutRange", streamCutRange) + .add("readerGroup", readerGroup) + .toString(); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/ReaderType.java b/src/main/java/com/facebook/presto/pravega/ReaderType.java new file mode 100644 index 0000000..eabac02 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/ReaderType.java @@ -0,0 +1,23 @@ +/* + * 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 com.facebook.presto.pravega; + +public enum ReaderType +{ + SEGMENT_RANGE /* 1 split handles all segments within a stream cut */, + SEGMENT_RANGE_PER_SPLIT /* segments for stream cut are given out to different splits */, + EVENT_STREAM /* stream oriented reading (vs. segments) */, + SINGLE_GROUP_EVENT_STREAM /* stream oriented reading (vs. segments) all readers in same group */, + KVT /* key value table */, +} diff --git a/src/main/java/com/facebook/presto/pravega/SegmentEventIterator.java b/src/main/java/com/facebook/presto/pravega/SegmentEventIterator.java new file mode 100644 index 0000000..1250645 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/SegmentEventIterator.java @@ -0,0 +1,50 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.airlift.log.Logger; +import com.facebook.presto.pravega.decoder.BytesEvent; +import com.facebook.presto.pravega.decoder.DecodableEvent; +import io.pravega.client.batch.SegmentIterator; +import io.pravega.client.batch.SegmentRange; +import io.pravega.client.stream.impl.ByteBufferSerializer; + +import java.nio.ByteBuffer; +import java.util.Iterator; + +public class SegmentEventIterator + implements Iterator +{ + private static final Logger log = Logger.get(SegmentEventIterator.class); + + private final SegmentIterator segmentEventIterator; + + public SegmentEventIterator(PravegaSegmentManager segmentManager, SegmentRange segmentRange) + { + log.info("open iterator for " + segmentRange); + this.segmentEventIterator = segmentManager.getSegmentIterator(segmentRange, new ByteBufferSerializer()); + } + + @Override + public boolean hasNext() + { + return segmentEventIterator.hasNext(); + } + + @Override + public DecodableEvent next() + { + return new BytesEvent(segmentEventIterator.next()); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/SegmentRangeIterator.java b/src/main/java/com/facebook/presto/pravega/SegmentRangeIterator.java new file mode 100644 index 0000000..ad8ea86 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/SegmentRangeIterator.java @@ -0,0 +1,102 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.airlift.log.Logger; +import com.facebook.presto.pravega.decoder.BytesEvent; +import com.facebook.presto.pravega.decoder.DecodableEvent; +import io.pravega.client.batch.SegmentIterator; +import io.pravega.client.batch.SegmentRange; +import io.pravega.client.stream.impl.ByteBufferSerializer; + +import java.nio.ByteBuffer; +import java.util.Iterator; + +public class SegmentRangeIterator + implements Iterator +{ + private static final Logger log = Logger.get(SegmentRangeIterator.class); + + private final PravegaSegmentManager segmentManager; + + private final Iterator segmentIterator; + + private SegmentIterator segmentEventIterator; + + private ByteBuffer event; + + private final StreamCutRange streamCutRange; + + private int fullSegments; + + private int emptySegments; + + private int events; + + public SegmentRangeIterator(PravegaSegmentManager segmentManager, ReaderArgs readerArgs) + { + this.segmentManager = segmentManager; + + this.streamCutRange = readerArgs.getStreamCutRange(); + + log.info("open iterator for " + streamCutRange); + this.segmentIterator = + segmentManager.getSegments(readerArgs.getScope(), + readerArgs.getStream(), + readerArgs.getStreamCutRange().getStart(), + readerArgs.getStreamCutRange().getEnd()).getIterator(); + } + + private ByteBuffer _next() + { + if (segmentEventIterator != null && segmentEventIterator.hasNext()) { + events++; + return segmentEventIterator.next(); + } + + do { + if (!segmentIterator.hasNext()) { + log.info("done with " + streamCutRange + "; full: " + fullSegments + ", empty: " + emptySegments + ", events: " + events); + return null; + } + + segmentEventIterator = segmentManager.getSegmentIterator(segmentIterator.next(), new ByteBufferSerializer()); + log.info("next segment " + streamCutRange + " has event? " + segmentEventIterator.hasNext()); + if (segmentEventIterator.hasNext()) { + fullSegments++; + events++; + return segmentEventIterator.next(); + } + emptySegments++; + // maybe segment was empty, continue + } while (true); + } + + @Override + public boolean hasNext() + { + if (event == null) { + event = _next(); + } + return event != null; + } + + @Override + public DecodableEvent next() + { + ByteBuffer toReturn = event; + event = null; + return new BytesEvent(toReturn); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/StreamCutRange.java b/src/main/java/com/facebook/presto/pravega/StreamCutRange.java new file mode 100644 index 0000000..ee44a9a --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/StreamCutRange.java @@ -0,0 +1,63 @@ +/* + * 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 com.facebook.presto.pravega; + +import io.pravega.client.stream.StreamCut; + +import java.io.Serializable; + +import static com.google.common.base.MoreObjects.toStringHelper; + +public class StreamCutRange + implements Serializable +{ + public static final StreamCutRange NULL_PAIR = new StreamCutRange(null, null); + + private final StreamCut start; + + private final StreamCut end; + + public StreamCutRange(StreamCut start, StreamCut end) + { + this.start = start; + this.end = end; + } + + public StreamCut getStart() + { + return start; + } + + public StreamCut getEnd() + { + return end; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("start", start == null ? "(null)" : start.asText()) + .add("startPositions", + start == null || start.asImpl() == null + ? "(null)" + : start.asImpl().getPositions()) + .add("end", end == null ? "(null)" : end.asText()) + .add("endPositions", + end == null || end.asImpl() == null + ? "(null)" + : end.asImpl().getPositions()) + .toString(); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/StreamCutSupplier.java b/src/main/java/com/facebook/presto/pravega/StreamCutSupplier.java new file mode 100644 index 0000000..a0b2488 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/StreamCutSupplier.java @@ -0,0 +1,102 @@ +/* + * 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 com.facebook.presto.pravega; + +import io.pravega.client.batch.SegmentIterator; +import io.pravega.client.batch.SegmentRange; +import io.pravega.client.stream.StreamCut; +import io.pravega.client.stream.impl.ByteBufferSerializer; + +import java.nio.ByteBuffer; +import java.util.Iterator; + +import static com.facebook.presto.pravega.util.PravegaNameUtils.streamCutName; + +public class StreamCutSupplier + implements AutoCloseable +{ + private PravegaSegmentManager segmentManager; + + private Iterator rangeIterator; + + private SegmentIterator segmentIterator; + + private StreamCut previous; + + private boolean empty; + + public StreamCutSupplier(PravegaSegmentManager segmentManager, String scope, String stream) + { + if (segmentManager.streamExists(scope, streamCutName(stream))) { + // for now, read stream cuts from internal stream + // https://github.com/pravega/pravega-sql/issues/24 + this.segmentManager = segmentManager; + + this.rangeIterator = segmentManager.getSegments(scope, streamCutName(stream), null, null).getIterator(); + // init fist stream cut + this.previous = nextStreamCut(); + } + + if (this.previous == null) { + // either stream doesn't exist or no stream cuts logged + this.empty = true; + } + } + + private StreamCut nextStreamCut() + { + do { + if (segmentIterator != null && segmentIterator.hasNext()) { + return StreamCut.fromBytes(segmentIterator.next()); + } + + if (!rangeIterator.hasNext()) { + return null; + } + + segmentIterator = segmentManager.getSegmentIterator(rangeIterator.next(), + new ByteBufferSerializer()); + } while (true); + } + + private StreamCutRange next() + { + if (previous == null) { + return null; + } + + StreamCut start = previous; + StreamCut end = nextStreamCut(); + previous = end; + + // looking for explicitly defined start+end stream cuts + // so we return null when we have no end (vs. start->UNBOUNDED) + return previous == null ? null : new StreamCutRange(start, end); + } + + public StreamCutRange get() + { + if (empty) { + StreamCutRange range = StreamCutRange.NULL_PAIR; + empty = false; + return range; + } + return next(); + } + + @Override + public void close() + { + } +} diff --git a/src/main/java/com/facebook/presto/pravega/TypedRecordValue.java b/src/main/java/com/facebook/presto/pravega/TypedRecordValue.java new file mode 100644 index 0000000..6b44a13 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/TypedRecordValue.java @@ -0,0 +1,104 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.decoder.FieldValueProvider; +import com.facebook.presto.spi.ColumnHandle; +import com.google.common.base.Preconditions; +import io.airlift.slice.Slice; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +// avro, protobuf, json +public class TypedRecordValue + implements PravegaRecordValue +{ + private final List columnHandles; + + private final FieldValueProvider[] currentRowValues; + + private final Map currentRowValuesMap = new HashMap<>(); + + private Optional> decodedValue; + + private boolean decoded; + + public TypedRecordValue(List columnHandles) + { + this.columnHandles = columnHandles; + this.currentRowValues = new FieldValueProvider[columnHandles.size()]; + } + + public void setDecodedValue(Optional> decodedValue) + { + this.decodedValue = decodedValue; + this.decoded = false; + } + + public Optional> getDecodedValue() + { + return decodedValue; + } + + public void decode() + { + currentRowValuesMap.clear(); + decodedValue.ifPresent(currentRowValuesMap::putAll); + + for (int i = 0; i < columnHandles.size(); i++) { + ColumnHandle columnHandle = columnHandles.get(i); + currentRowValues[i] = currentRowValuesMap.get(columnHandle); + } + + decoded = true; + } + + public boolean decoded() + { + return decoded; + } + + public boolean isNull(int field, int ordinalPosition) + { + return currentRowValues[field] == null || currentRowValues[field].isNull(); + } + + public long getLong(int field, int ordinalPosition) + { + Preconditions.checkState(decoded); + return currentRowValues[field].getLong(); + } + + public double getDouble(int field, int ordinalPosition) + { + Preconditions.checkState(decoded); + return currentRowValues[field].getDouble(); + } + + public boolean getBoolean(int field, int ordinalPosition) + { + Preconditions.checkState(decoded); + return currentRowValues[field].getBoolean(); + } + + public Slice getSlice(int field, int ordinalPosition) + { + Preconditions.checkState(decoded); + return currentRowValues[field].getSlice(); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/decoder/AvroEvent.java b/src/main/java/com/facebook/presto/pravega/decoder/AvroEvent.java new file mode 100644 index 0000000..447d4a8 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/decoder/AvroEvent.java @@ -0,0 +1,38 @@ +/* + * 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 com.facebook.presto.pravega.decoder; + +import org.apache.avro.generic.GenericRecord; + +public class AvroEvent + extends DecodableEvent +{ + private final GenericRecord record; + + public AvroEvent(Object object) + { + this.record = (GenericRecord) object; + } + + public GenericRecord asAvro() + { + return record; + } + + @Override + protected int size() + { + return 0; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/decoder/AvroRowDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/AvroRowDecoder.java new file mode 100644 index 0000000..791a8db --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/decoder/AvroRowDecoder.java @@ -0,0 +1,52 @@ +/* + * 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 com.facebook.presto.pravega.decoder; + +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.decoder.avro.AvroColumnDecoder; +import com.facebook.presto.pravega.PravegaRecordValue; +import com.facebook.presto.pravega.TypedRecordValue; + +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static com.google.common.base.Functions.identity; +import static com.google.common.collect.ImmutableMap.toImmutableMap; + +public class AvroRowDecoder + implements EventDecoder +{ + private final Map columnDecoders; + + public AvroRowDecoder(Set columns) + { + columnDecoders = columns.stream().collect(toImmutableMap(identity(), this::createColumnDecoder)); + } + + private AvroColumnDecoder createColumnDecoder(DecoderColumnHandle columnHandle) + { + return new AvroColumnDecoder(columnHandle); + } + + @Override + public boolean decodeEvent(DecodableEvent event, PravegaRecordValue record) + { + ((TypedRecordValue) record).setDecodedValue( + Optional.of(columnDecoders.entrySet().stream() + .collect(toImmutableMap(Map.Entry::getKey, entry -> + entry.getValue().decodeField(event.asAvro()))))); + return true; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/decoder/AvroSerializer.java b/src/main/java/com/facebook/presto/pravega/decoder/AvroSerializer.java new file mode 100644 index 0000000..354c38f --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/decoder/AvroSerializer.java @@ -0,0 +1,103 @@ +/* + * 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 com.facebook.presto.pravega.decoder; + +import com.facebook.presto.pravega.util.ByteBufferInputStream; +import com.google.protobuf.DynamicMessage; +import io.pravega.client.stream.Serializer; +import io.pravega.schemaregistry.serializer.shared.impl.SerializerConfig; +import io.pravega.schemaregistry.serializers.SerializerFactory; +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileStream; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.DatumReader; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; + +// deserialize using externally provided schema or using SR+SerializerConfig +public class AvroSerializer + extends KVSerializer +{ + private static class GenericRecordSerializer + implements Serializer + { + private final DatumReader datumReader; + + private final Schema schema; + + GenericRecordSerializer(Schema schema) + { + this.datumReader = new GenericDatumReader(schema); + this.schema = schema; + } + + @Override + public ByteBuffer serialize(Object value) + { + return ByteBuffer.wrap(((DynamicMessage) value).toByteArray()); + } + + @Override + public GenericRecord deserialize(ByteBuffer serializedValue) + { + try (DataFileStream dataFileReader = + new DataFileStream<>(new ByteBufferInputStream(serializedValue), datumReader)) { + // TODO: need to figure out how to auto-detect format of avro data + // for e.g, is schema provided for every row? (this is how the normal presto avro decoder takes it) + // i would think more typically case would be that schema defined once and thus schema not provided + // in every rows data + // + // for now we will do it the "presto way" + return dataFileReader.next(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + } + + private final Serializer delegate; + + public AvroSerializer(SerializerConfig config) + { + this.delegate = SerializerFactory.genericDeserializer(config); + } + + public AvroSerializer(String encodedSchema) + { + Schema schema = (new Schema.Parser()).parse(encodedSchema); + this.delegate = new GenericRecordSerializer(schema); + } + + @Override + public ByteBuffer serialize(GenericRecord value) + { + return delegate.serialize(value); + } + + @Override + public GenericRecord deserialize(ByteBuffer serializedValue) + { + return (GenericRecord) delegate.deserialize(serializedValue); + } + + @Override + public DecodableEvent toEvent(Object obj) + { + return new AvroEvent(obj); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/decoder/BytesEvent.java b/src/main/java/com/facebook/presto/pravega/decoder/BytesEvent.java new file mode 100644 index 0000000..3a85469 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/decoder/BytesEvent.java @@ -0,0 +1,38 @@ +/* + * 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 com.facebook.presto.pravega.decoder; + +import java.nio.ByteBuffer; + +public class BytesEvent + extends DecodableEvent +{ + private ByteBuffer buffer; + + public BytesEvent(ByteBuffer buffer) + { + this.buffer = buffer; + } + + public ByteBuffer asBytes() + { + return buffer; + } + + @Override + protected int size() + { + return buffer.capacity(); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/decoder/BytesEventDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/BytesEventDecoder.java new file mode 100644 index 0000000..75b5b06 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/decoder/BytesEventDecoder.java @@ -0,0 +1,37 @@ +/* + * 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 com.facebook.presto.pravega.decoder; + +import com.facebook.presto.pravega.PravegaRecordValue; + +// raw byte source. deserialize and then pass off to decoder +public class BytesEventDecoder + implements EventDecoder +{ + private final KVSerializer kvSerializer; + + private final EventDecoder delegate; + + public BytesEventDecoder(KVSerializer kvSerializer, EventDecoder delegate) + { + this.kvSerializer = kvSerializer; + this.delegate = delegate; + } + + @Override + public boolean decodeEvent(DecodableEvent event, PravegaRecordValue record) + { + return delegate.decodeEvent(kvSerializer.toEvent(kvSerializer.deserialize(event.asBytes())), record); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/decoder/CsvRowDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/CsvRowDecoder.java new file mode 100644 index 0000000..52d5111 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/decoder/CsvRowDecoder.java @@ -0,0 +1,28 @@ +/* + * 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 com.facebook.presto.pravega.decoder; + +import com.facebook.presto.pravega.DelimRecordValue; +import com.facebook.presto.pravega.PravegaRecordValue; + +public class CsvRowDecoder + implements EventDecoder +{ + @Override + public boolean decodeEvent(DecodableEvent event, PravegaRecordValue record) + { + ((DelimRecordValue) record).setBuf(event.asBytes()); + return true; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/decoder/CsvSerializer.java b/src/main/java/com/facebook/presto/pravega/decoder/CsvSerializer.java new file mode 100644 index 0000000..ac736b6 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/decoder/CsvSerializer.java @@ -0,0 +1,44 @@ +/* + * 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 com.facebook.presto.pravega.decoder; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; + +public class CsvSerializer + extends KVSerializer +{ + public CsvSerializer() + { + } + + @Override + public ByteBuffer serialize(String value) + { + return ByteBuffer.wrap(value.getBytes(StandardCharsets.UTF_8)); + } + + @Override + public String deserialize(ByteBuffer serializedValue) + { + return new String(serializedValue.array(), + serializedValue.arrayOffset() + serializedValue.position(), + serializedValue.remaining()); + } + + @Override + public DecodableEvent toEvent(Object obj) + { + return new BytesEvent(ByteBuffer.wrap(((String) obj).getBytes(StandardCharsets.UTF_8))); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/decoder/DecodableEvent.java b/src/main/java/com/facebook/presto/pravega/decoder/DecodableEvent.java new file mode 100644 index 0000000..dda18da --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/decoder/DecodableEvent.java @@ -0,0 +1,62 @@ +/* + * 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 com.facebook.presto.pravega.decoder; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.protobuf.DynamicMessage; +import org.apache.avro.generic.GenericRecord; + +import java.nio.ByteBuffer; + +public abstract class DecodableEvent +{ + private DecodableEvent next; + + public void setNext(DecodableEvent next) + { + this.next = next; + } + + public DecodableEvent next() + { + return next; + } + + public DynamicMessage asProtobuf() + { + throw new UnsupportedOperationException("not protobuf format"); + } + + public GenericRecord asAvro() + { + throw new UnsupportedOperationException("not avro format"); + } + + public JsonNode asJson() + { + throw new UnsupportedOperationException("not json format"); + } + + public ByteBuffer asBytes() + { + throw new UnsupportedOperationException("not bytes format"); + } + + protected abstract int size(); + + public int totalSize() + { + return size() + (next == null ? 0 : next.totalSize()); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/decoder/EventDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/EventDecoder.java new file mode 100644 index 0000000..f3e6ae5 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/decoder/EventDecoder.java @@ -0,0 +1,21 @@ +/* + * 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 com.facebook.presto.pravega.decoder; + +import com.facebook.presto.pravega.PravegaRecordValue; + +public interface EventDecoder +{ + boolean decodeEvent(DecodableEvent event, PravegaRecordValue record); +} diff --git a/src/main/java/com/facebook/presto/pravega/decoder/JsonEvent.java b/src/main/java/com/facebook/presto/pravega/decoder/JsonEvent.java new file mode 100644 index 0000000..d1a5763 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/decoder/JsonEvent.java @@ -0,0 +1,38 @@ +/* + * 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 com.facebook.presto.pravega.decoder; + +import com.fasterxml.jackson.databind.JsonNode; + +public class JsonEvent + extends DecodableEvent +{ + private final JsonNode tree; + + public JsonEvent(Object object) + { + this.tree = (JsonNode) object; + } + + public JsonNode asJson() + { + return tree; + } + + @Override + protected int size() + { + return tree.size(); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/decoder/JsonRowDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/JsonRowDecoder.java new file mode 100644 index 0000000..bda699e --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/decoder/JsonRowDecoder.java @@ -0,0 +1,35 @@ +/* + * 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 com.facebook.presto.pravega.decoder; + +import com.facebook.presto.pravega.PravegaRecordValue; +// import com.facebook.presto.pravega.TypedRecordValue; + +public class JsonRowDecoder + implements EventDecoder +{ + private final com.facebook.presto.decoder.json.JsonRowDecoder jsonRowDecoder; + + public JsonRowDecoder(com.facebook.presto.decoder.json.JsonRowDecoder jsonRowDecoder) + { + this.jsonRowDecoder = jsonRowDecoder; + } + + @Override + public boolean decodeEvent(DecodableEvent event, PravegaRecordValue record) + { + // ((TypedRecordValue) record).setDecodedValue(jsonRowDecoder.decodeTree(event.asJson())); + return true; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/decoder/JsonSerializer.java b/src/main/java/com/facebook/presto/pravega/decoder/JsonSerializer.java new file mode 100644 index 0000000..635f21d --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/decoder/JsonSerializer.java @@ -0,0 +1,88 @@ +/* + * 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 com.facebook.presto.pravega.decoder; + +import com.facebook.presto.pravega.util.ByteBufferInputStream; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.pravega.client.stream.Serializer; +import io.pravega.schemaregistry.serializer.shared.impl.SerializerConfig; +import io.pravega.schemaregistry.serializers.SerializerFactory; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; + +// deserialize using externally provided schema or using SR+SerializerConfig +public class JsonSerializer + extends KVSerializer +{ + private static class JsonTreeSerializer + implements Serializer + { + private final ObjectMapper objectMapper = new ObjectMapper(); + + @Override + public ByteBuffer serialize(Object value) + { + try { + return ByteBuffer.wrap(objectMapper.writeValueAsBytes(value)); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public JsonNode deserialize(ByteBuffer serializedValue) + { + try { + return objectMapper.readTree(new ByteBufferInputStream(serializedValue)); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + } + + private final Serializer delegate; + + public JsonSerializer(SerializerConfig config) + { + this.delegate = SerializerFactory.genericDeserializer(config); + } + + public JsonSerializer() + { + this.delegate = new JsonTreeSerializer(); + } + + @Override + public ByteBuffer serialize(JsonNode value) + { + return delegate.serialize(value); + } + + @Override + public JsonNode deserialize(ByteBuffer serializedValue) + { + return (JsonNode) delegate.deserialize(serializedValue); + } + + @Override + public DecodableEvent toEvent(Object obj) + { + return new JsonEvent(obj); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/decoder/KVSerializer.java b/src/main/java/com/facebook/presto/pravega/decoder/KVSerializer.java new file mode 100644 index 0000000..c6c0256 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/decoder/KVSerializer.java @@ -0,0 +1,24 @@ +/* + * 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 com.facebook.presto.pravega.decoder; + +import io.pravega.client.stream.Serializer; + +// deserialize using externally provided schema or using SR+SerializerConfig +public abstract class KVSerializer + implements Serializer +{ + // create an event that can be passed down to decoders + public abstract DecodableEvent toEvent(Object obj); +} diff --git a/src/main/java/com/facebook/presto/pravega/decoder/MultiSourceRowDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/MultiSourceRowDecoder.java new file mode 100644 index 0000000..e83d776 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/decoder/MultiSourceRowDecoder.java @@ -0,0 +1,60 @@ +/* + * 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 com.facebook.presto.pravega.decoder; + +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.decoder.FieldValueProvider; +import com.facebook.presto.pravega.PravegaRecordValue; +import com.facebook.presto.pravega.TypedRecordValue; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +// row comprised of N sources +// (kv table, where we have a key + value which are dealt with separately) +public class MultiSourceRowDecoder + implements EventDecoder +{ + private final List rowDecoders; + + public MultiSourceRowDecoder(List rowDecoders) + { + this.rowDecoders = rowDecoders; + } + + @Override + public boolean decodeEvent(DecodableEvent event, PravegaRecordValue record) + { + // rowDecoder[0] <-> event + // rowDecoder[1] <-> event.next() + // rowDecoder[N] <-> .. + + TypedRecordValue legacyRecord = (TypedRecordValue) record; + Map result = new HashMap<>(); + + for (EventDecoder rowDecoder : rowDecoders) { + if (event == null) { + throw new IllegalArgumentException("no more events, decoder set of " + rowDecoders.size()); + } + rowDecoder.decodeEvent(event, legacyRecord); + legacyRecord.getDecodedValue().ifPresent(result::putAll); + event = event.next(); + } + + legacyRecord.setDecodedValue(Optional.of(result)); + return true; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufColumnDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufColumnDecoder.java new file mode 100644 index 0000000..540e6ca --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufColumnDecoder.java @@ -0,0 +1,172 @@ +/* + * 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 com.facebook.presto.pravega.decoder; + +import com.facebook.presto.common.type.Type; +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.decoder.FieldValueProvider; +import com.facebook.presto.spi.PrestoException; +import com.google.protobuf.ByteString; +import com.google.protobuf.Descriptors; +import com.google.protobuf.DynamicMessage; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; + +import java.util.Map; + +import static com.facebook.presto.common.type.StandardTypes.VARBINARY; +import static com.facebook.presto.common.type.StandardTypes.VARCHAR; +import static com.facebook.presto.common.type.Varchars.truncateToLength; +import static com.facebook.presto.decoder.DecoderErrorCode.DECODER_CONVERSION_NOT_SUPPORTED; +import static com.facebook.presto.pravega.util.PravegaSchemaUtils.NESTED_RECORD_SEPARATOR; +import static io.airlift.slice.Slices.utf8Slice; +import static java.lang.String.format; + +public class ProtobufColumnDecoder +{ + private final Type type; + private final String name; + private final String[] mapping; + + public ProtobufColumnDecoder(DecoderColumnHandle columnHandle) + { + this.type = columnHandle.getType(); + this.name = columnHandle.getName(); + this.mapping = columnHandle.getMapping().split(NESTED_RECORD_SEPARATOR); + } + + public FieldValueProvider decodeField(DynamicMessage dynamicMessage) + { + return decodeField(0, dynamicMessage); + } + + /* + protobuf schema: + Address { + string street; + string city; + string state; + } + + Person { + string first; + string last; + Address address; + } + + our list of columns will be: + first, last, address/street, address/city, address/state + + Address here is nested. if this column is "address/city": + this.name = "address/city" + this.mapping[0] = "address" + this.mapping[1] = "city" + + starting out with mapping level 0 we look for mapping[0] / "address" + iterating fields of Person, we will get field name "address" of type MESSAGE + here we recurse into decodeField bumping the mapping level to look for mapping[1] / "city" + */ + + private FieldValueProvider decodeField(int level, DynamicMessage dynamicMessage) + { + for (Map.Entry entry : dynamicMessage.getAllFields().entrySet()) { + if (entry.getKey().getJsonName().equals(this.mapping[level])) { + if (entry.getKey().getType() == Descriptors.FieldDescriptor.Type.MESSAGE) { + if (level == this.mapping.length - 1) { + throw new IllegalArgumentException("unexpected end to mapping " + name); + } + return decodeField(level + 1, + (DynamicMessage) entry.getValue()); + } + else { + return new ProtobufFieldValueProvider(this.type, this.name, entry.getValue()); + } + } + } + // record does not have this field. will return null. + return new ProtobufFieldValueProvider(this.type, this.name, null); + } + + private static class ProtobufFieldValueProvider + extends FieldValueProvider + { + private final Type type; + private final String name; + private final Object object; + + ProtobufFieldValueProvider(Type type, String name, Object object) + { + this.type = type; + this.name = name; + this.object = object; + } + + @Override + public boolean getBoolean() + { + return (boolean) object; + } + + @Override + public long getLong() + { + if (object instanceof Integer) { + return (long) (int) object; + } + else { + return (long) object; + } + } + + @Override + public double getDouble() + { + if (object instanceof Float) { + return (double) (float) object; + } + else { + return (double) object; + } + } + + @Override + public Slice getSlice() + { + return getSlice(object, type, name); + } + + @Override + public boolean isNull() + { + return object == null; + } + + private static Slice getSlice(Object value, Type type, String columnName) + { + switch (type.getTypeSignature().getBase()) { + case VARCHAR: + if (value instanceof String) { + return truncateToLength(utf8Slice(value.toString()), type); + } + case VARBINARY: + if (value instanceof ByteString) { + return Slices.wrappedBuffer(((ByteString) value).asReadOnlyByteBuffer()); + } + default: + throw new PrestoException(DECODER_CONVERSION_NOT_SUPPORTED, + format("cannot decode object of '%s' as '%s' for column '%s'", value.getClass(), type, columnName)); + } + } + } +} diff --git a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufEvent.java b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufEvent.java new file mode 100644 index 0000000..d67dd4e --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufEvent.java @@ -0,0 +1,38 @@ +/* + * 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 com.facebook.presto.pravega.decoder; + +import com.google.protobuf.DynamicMessage; + +public class ProtobufEvent + extends DecodableEvent +{ + private final DynamicMessage message; + + public ProtobufEvent(Object object) + { + this.message = (DynamicMessage) object; + } + + public DynamicMessage asProtobuf() + { + return message; + } + + @Override + protected int size() + { + return 0; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufRowDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufRowDecoder.java new file mode 100644 index 0000000..991899c --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufRowDecoder.java @@ -0,0 +1,51 @@ +/* + * 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 com.facebook.presto.pravega.decoder; + +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.pravega.PravegaRecordValue; +import com.facebook.presto.pravega.TypedRecordValue; + +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static com.google.common.base.Functions.identity; +import static com.google.common.collect.ImmutableMap.toImmutableMap; + +public class ProtobufRowDecoder + implements EventDecoder +{ + private final Map columnDecoders; + + public ProtobufRowDecoder(Set columns) + { + columnDecoders = columns.stream().collect(toImmutableMap(identity(), this::createColumnDecoder)); + } + + private ProtobufColumnDecoder createColumnDecoder(DecoderColumnHandle columnHandle) + { + return new ProtobufColumnDecoder(columnHandle); + } + + @Override + public boolean decodeEvent(DecodableEvent event, PravegaRecordValue record) + { + ((TypedRecordValue) record).setDecodedValue( + Optional.of(columnDecoders.entrySet().stream() + .collect(toImmutableMap(Map.Entry::getKey, entry -> + entry.getValue().decodeField(event.asProtobuf()))))); + return true; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufSerializer.java b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufSerializer.java new file mode 100644 index 0000000..7056211 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufSerializer.java @@ -0,0 +1,94 @@ +/* + * 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 com.facebook.presto.pravega.decoder; + +import com.facebook.presto.pravega.util.ByteBufferInputStream; +import com.google.protobuf.Descriptors; +import com.google.protobuf.DynamicMessage; +import io.pravega.client.stream.Serializer; +import io.pravega.schemaregistry.serializer.shared.impl.SerializerConfig; +import io.pravega.schemaregistry.serializers.SerializerFactory; +import org.apache.commons.lang3.tuple.Pair; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; + +import static com.facebook.presto.pravega.ProtobufCommon.decodeSchema; +import static com.facebook.presto.pravega.ProtobufCommon.descriptorFor; + +// deserialize using externally provided schema or using SR+SerializerConfig +public class ProtobufSerializer + extends KVSerializer +{ + private static class DynamicMessageSerializer + implements Serializer + { + private final Descriptors.Descriptor descriptor; + + DynamicMessageSerializer(Descriptors.Descriptor descriptor) + { + this.descriptor = descriptor; + } + + @Override + public ByteBuffer serialize(Object value) + { + return ByteBuffer.wrap(((DynamicMessage) value).toByteArray()); + } + + @Override + public DynamicMessage deserialize(ByteBuffer serializedValue) + { + try { + return DynamicMessage.parseFrom(descriptor, + new ByteBufferInputStream(serializedValue)); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + } + + private final Serializer delegate; + + public ProtobufSerializer(SerializerConfig config) + { + this.delegate = SerializerFactory.genericDeserializer(config); + } + + public ProtobufSerializer(String encodedSchema) + { + Pair pair = decodeSchema(encodedSchema); + this.delegate = new DynamicMessageSerializer(descriptorFor(pair.getLeft(), pair.getRight())); + } + + @Override + public ByteBuffer serialize(DynamicMessage value) + { + return delegate.serialize(value); + } + + @Override + public DynamicMessage deserialize(ByteBuffer serializedValue) + { + return (DynamicMessage) delegate.deserialize(serializedValue); + } + + @Override + public DecodableEvent toEvent(Object obj) + { + return new ProtobufEvent(obj); + } +} diff --git a/src/main/java/com/facebook/presto/pravega/util/ByteBufferInputStream.java b/src/main/java/com/facebook/presto/pravega/util/ByteBufferInputStream.java new file mode 100644 index 0000000..c7ffc69 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/util/ByteBufferInputStream.java @@ -0,0 +1,46 @@ +/* + * 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 com.facebook.presto.pravega.util; + +import java.io.InputStream; +import java.nio.ByteBuffer; + +public class ByteBufferInputStream + extends InputStream +{ + private final ByteBuffer buffer; + + public ByteBufferInputStream(ByteBuffer buffer) + { + this.buffer = buffer; + } + + @Override + public int read() + { + return buffer.hasRemaining() ? buffer.get() & 0xff : -1; + } + + @Override + public int read(byte[] bytes, int offset, int length) + { + if (!buffer.hasRemaining()) { + return -1; + } + + length = Math.min(length, buffer.remaining()); + buffer.get(bytes, offset, length); + return length; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/util/PravegaNameUtils.java b/src/main/java/com/facebook/presto/pravega/util/PravegaNameUtils.java new file mode 100644 index 0000000..1a32d29 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/util/PravegaNameUtils.java @@ -0,0 +1,108 @@ +/* + * 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 com.facebook.presto.pravega.util; + +import com.facebook.presto.pravega.ObjectType; +import com.facebook.presto.pravega.PravegaStreamDescription; +import com.facebook.presto.pravega.PravegaTableHandle; +import io.pravega.shared.NameUtils; + +import java.util.HashMap; +import java.util.Map; + +public class PravegaNameUtils +{ + private PravegaNameUtils() + { + } + + public static final String STREAM_CUT_PREFIX = "-SC"; + + // used for prefixing field names when presenting them in presto + // will default to these prefixes for kv table fields unless specified in user config + static Map kvFieldNamePrefixMap = new HashMap<>(); + static + { + kvFieldNamePrefixMap.put(0, "key"); + kvFieldNamePrefixMap.put(1, "value"); + } + + public static String scopedName(String scope, String stream) + { + return scope + "/" + stream; + } + + public static String groupId(String scope, String stream) + { + return scope + "." + stream; + } + + // test stream name - if not valid pravega stream name assume it is regex for multi source + public static boolean multiSourceStream(PravegaStreamDescription object) + { + return object.getObjectType() == ObjectType.STREAM && + multiSourceStream(object.getObjectName()); + } + + public static boolean multiSourceStream(PravegaTableHandle object) + { + return object.getObjectType() == ObjectType.STREAM && + multiSourceStream(object.getObjectName()); + } + + private static boolean multiSourceStream(String stream) + { + try { + // test pattern for stream names pravega will allow + NameUtils.validateUserStreamName(stream); + return false; + } + catch (IllegalArgumentException e) { + // if not valid, we take it as multi source w/ regex + return true; + } + } + + public static boolean kvTable(PravegaStreamDescription object) + { + return object.getObjectType() == ObjectType.KV_TABLE; + } + + public static String kvFieldMapping(int index) + { + // should only see 0, 1 + return kvFieldNamePrefixMap.getOrDefault(index, "UNEXPECTED INDEX"); + } + + // need to allow user/application to specify what table name is + // https://github.com/StreamingDataPlatform/pravega-sql/issues/91 + public static String temp_tableNameToStreamName(String tableName) + { + // stream name does not allow '_' so we change to '-' + // but tpc test + scripts require '_' + return tableName.replaceAll("_", "-"); + } + + public static String temp_streamNameToTableName(String streamName) + { + // stream name does not allow '_' so we change to '-' + // but tpc test + scripts require '_' + return streamName.replaceAll("-", "_"); + } + + public static String streamCutName(String stream) + { + return stream + STREAM_CUT_PREFIX; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/util/PravegaSchemaUtils.java b/src/main/java/com/facebook/presto/pravega/util/PravegaSchemaUtils.java new file mode 100644 index 0000000..832d545 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/util/PravegaSchemaUtils.java @@ -0,0 +1,103 @@ +/* + * 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 com.facebook.presto.pravega.util; + +import com.facebook.airlift.log.Logger; +import com.facebook.presto.spi.PrestoException; +import com.google.common.io.CharStreams; + +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.net.MalformedURLException; +import java.net.URI; +import java.net.URL; + +import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; +import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.Locale.ENGLISH; + +public class PravegaSchemaUtils +{ + private PravegaSchemaUtils() + { + } + + private static final Logger log = Logger.get(PravegaSchemaUtils.class); + + public static final String AVRO_INLINE = "avro-inline"; + public static final String PROTOBUF_INLINE = "protobuf-inline"; + public static final String JSON_INLINE = "json-inline"; + public static final String INLINE_SUFFIX = "-inline"; + public static final String GROUP_PROPERTIES_INLINE_KEY = "inline"; + public static final String GROUP_PROPERTIES_INLINE_KV_KEY = "inlinekey"; + public static final String GROUP_PROPERTIES_INLINE_KV_VALUE = "inlinevalue"; + public static final String AVRO = "avro"; + public static final String PROTOBUF = "protobuf"; + public static final String JSON = "json"; + public static final String CSV = "csv"; + + public static final String NESTED_RECORD_SEPARATOR = "/"; + + public static String readSchema(String dataSchemaLocation) + { + InputStream inputStream = null; + try { + if (isURI(dataSchemaLocation.trim().toLowerCase(ENGLISH))) { + try { + inputStream = new URL(dataSchemaLocation).openStream(); + } + catch (MalformedURLException e) { + // try again before failing + log.warn("invalid URL: " + dataSchemaLocation); + inputStream = new FileInputStream(dataSchemaLocation); + } + } + else { + inputStream = new FileInputStream(dataSchemaLocation); + } + return CharStreams.toString(new InputStreamReader(inputStream, UTF_8)); + } + catch (IOException e) { + throw new PrestoException(GENERIC_INTERNAL_ERROR, + "Could not parse the schema at: " + dataSchemaLocation, e); + } + finally { + closeQuietly(inputStream); + } + } + + private static void closeQuietly(InputStream stream) + { + try { + if (stream != null) { + stream.close(); + } + } + catch (IOException ignored) { + } + } + + private static boolean isURI(String location) + { + try { + URI.create(location); + } + catch (Exception e) { + return false; + } + return true; + } +} diff --git a/src/main/java/com/facebook/presto/pravega/util/PravegaSerializationUtils.java b/src/main/java/com/facebook/presto/pravega/util/PravegaSerializationUtils.java new file mode 100644 index 0000000..b3af0d1 --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/util/PravegaSerializationUtils.java @@ -0,0 +1,57 @@ +/* + * 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 com.facebook.presto.pravega.util; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.io.UncheckedIOException; + +public class PravegaSerializationUtils +{ + private PravegaSerializationUtils() + { + } + + public static byte[] serialize(Serializable s) + { + try { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(baos); + oos.writeObject(s); + return baos.toByteArray(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + public static T deserialize(byte[] bytes, Class clazz) + { + try { + ByteArrayInputStream bais = new ByteArrayInputStream(bytes); + ObjectInputStream bis = new ObjectInputStream(bais); + return (T) bis.readObject(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + catch (ClassNotFoundException e) { + throw new IllegalArgumentException(e); + } + } +} diff --git a/src/main/resources/META-INF/services/com.facebook.presto.spi.Plugin b/src/main/resources/META-INF/services/com.facebook.presto.spi.Plugin new file mode 100644 index 0000000..d773c3c --- /dev/null +++ b/src/main/resources/META-INF/services/com.facebook.presto.spi.Plugin @@ -0,0 +1 @@ +com.facebook.presto.pravega.PravegaPlugin diff --git a/src/modernizer/violations.xml b/src/modernizer/violations.xml new file mode 100644 index 0000000..5cc8cd4 --- /dev/null +++ b/src/modernizer/violations.xml @@ -0,0 +1,32 @@ + + + + java/lang/Class.newInstance:()Ljava/lang/Object; + 1.1 + Prefer Class.getConstructor().newInstance() + + + + java/lang/String.toLowerCase:()Ljava/lang/String; + 1.1 + Prefer String.toLowerCase(java.util.Locale) + + + + com/google/common/primitives/Ints.checkedCast:(J)I + 1.8 + Prefer Math.toIntExact(long) + + + + org/testng/Assert.assertEquals:(Ljava/lang/Iterable;Ljava/lang/Iterable;)V + 1.8 + Use com.facebook.presto.testing.assertions.Assert.assertEquals due to TestNG #543 + + + + org/testng/Assert.assertEquals:(Ljava/lang/Iterable;Ljava/lang/Iterable;Ljava/lang/String;)V + 1.8 + Use com.facebook.presto.testing.assertions.Assert.assertEquals due to TestNG #543 + + diff --git a/src/test/java/com/facebook/presto/pravega/TestPravegaPlugin.java b/src/test/java/com/facebook/presto/pravega/TestPravegaPlugin.java new file mode 100644 index 0000000..39cb82e --- /dev/null +++ b/src/test/java/com/facebook/presto/pravega/TestPravegaPlugin.java @@ -0,0 +1,43 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.spi.connector.Connector; +import com.facebook.presto.spi.connector.ConnectorFactory; +import com.facebook.presto.testing.TestingConnectorContext; +import com.google.common.collect.ImmutableMap; + +import static com.facebook.airlift.testing.Assertions.assertInstanceOf; +import static com.google.common.collect.Iterables.getOnlyElement; +import static org.testng.Assert.assertNotNull; + +public class TestPravegaPlugin +{ + public void testSpinup() + { + PravegaPlugin plugin = new PravegaPlugin(); + + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + assertInstanceOf(factory, PravegaConnectorFactory.class); + + Connector c = factory.create( + "test-connector", + ImmutableMap.builder() + .put("pravega.table-names", "test") + .put("pravega.nodes", "localhost:9092") + .build(), + new TestingConnectorContext()); + assertNotNull(c); + } +} From 881093fac45133c152fb5d67e4e4710e09a20e36 Mon Sep 17 00:00:00 2001 From: Chip Maurer Date: Fri, 5 Mar 2021 13:19:30 -0500 Subject: [PATCH 2/8] Added checkstyle plugin. Kept the presto version of checkstyle.xml Signed-off-by: Chip Maurer --- build.gradle | 14 +--------- checkstyle.gradle | 26 +++++++++++++++++++ .../checkstyle/checkstyle.xml | 3 +++ 3 files changed, 30 insertions(+), 13 deletions(-) create mode 100644 checkstyle.gradle rename src/checkstyle/presto-checks.xml => config/checkstyle/checkstyle.xml (98%) diff --git a/build.gradle b/build.gradle index 5e1092d..f153396 100644 --- a/build.gradle +++ b/build.gradle @@ -5,6 +5,7 @@ plugins { id 'java' id 'distribution' + id 'checkstyle' } repositories { @@ -22,11 +23,6 @@ repositories { } } -configurations { - prestoDB - prestoSQL -} - dependencies { compile 'com.facebook.airlift:bootstrap:0.191' compile 'com.facebook.airlift:json:0.191' @@ -82,14 +78,6 @@ task getHomeDir { } } -jar { - from project.configurations.runtime, configurations.prestoDB -} - -task prestoSQLJar(type: Jar) { - from project.configurations.runtime, configurations.prestoSQL -} - plugins.withType(DistributionPlugin) { distTar { compression = Compression.GZIP diff --git a/checkstyle.gradle b/checkstyle.gradle new file mode 100644 index 0000000..9a8830f --- /dev/null +++ b/checkstyle.gradle @@ -0,0 +1,26 @@ +/** + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * 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 + * + */ +plugins.withId('checkstyle') { + checkstyle { + toolVersion = checkstyleToolVersion + + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + configProperties = [importControlFile: "$rootDir/checkstyle/import-control.xml", + suppressionsFile: "$rootDir/checkstyle/suppressions.xml"] + } + + plugins.withId('java') { + checkstyle { + // Exclude generated code from Checkstyle checks + checkstyleMain.exclude '**/rest/generated/**' + } + } +} diff --git a/src/checkstyle/presto-checks.xml b/config/checkstyle/checkstyle.xml similarity index 98% rename from src/checkstyle/presto-checks.xml rename to config/checkstyle/checkstyle.xml index 96b8e42..d5791b2 100644 --- a/src/checkstyle/presto-checks.xml +++ b/config/checkstyle/checkstyle.xml @@ -3,6 +3,9 @@ "-//Puppy Crawl//DTD Check Configuration 1.3//EN" "http://checkstyle.sourceforge.net/dtds/configuration_1_3.dtd"> + From 9a627fb1b2c77e0de33d1f050e27efb1000ff5b5 Mon Sep 17 00:00:00 2001 From: Chip Maurer Date: Fri, 5 Mar 2021 13:38:39 -0500 Subject: [PATCH 3/8] Add github action workflow file for build. Add more when tests become available Signed-off-by: Chip Maurer --- .github/workflows/pravega-build.yml | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) create mode 100644 .github/workflows/pravega-build.yml diff --git a/.github/workflows/pravega-build.yml b/.github/workflows/pravega-build.yml new file mode 100644 index 0000000..6dd4780 --- /dev/null +++ b/.github/workflows/pravega-build.yml @@ -0,0 +1,16 @@ +name: Java CI + +on: [push] + +jobs: + build: + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v2 + - name: Set up JDK 1.8 + uses: actions/setup-java@v1 + with: + java-version: 1.8 + - name: Build with Gradle + run: ./gradlew build From 4f9d14fabf6ab0f492ad97f653c584f422573c69 Mon Sep 17 00:00:00 2001 From: Chip Maurer Date: Tue, 9 Mar 2021 13:34:14 -0500 Subject: [PATCH 4/8] Added correct headers and HeaderExp check in checkstyle Signed-off-by: Chip Maurer --- build.gradle | 2 ++ config/checkstyle/checkstyle.xml | 6 +++--- config/checkstyle/suppressions.xml | 14 ++++++++++++++ gradle.properties | 11 +++++++++++ checkstyle.gradle => gradle/checkstyle.gradle | 5 ++--- .../pravega/ClassLoaderSafeRecordSetProvider.java | 5 ++++- .../facebook/presto/pravega/DelimRecordValue.java | 5 ++++- .../presto/pravega/EventStreamIterator.java | 5 ++++- .../com/facebook/presto/pravega/ObjectType.java | 5 ++++- .../presto/pravega/PravegaColumnHandle.java | 5 ++++- .../facebook/presto/pravega/PravegaConnector.java | 5 ++++- .../presto/pravega/PravegaConnectorConfig.java | 5 ++++- .../presto/pravega/PravegaConnectorFactory.java | 5 ++++- .../presto/pravega/PravegaConnectorId.java | 5 ++++- .../presto/pravega/PravegaConnectorModule.java | 5 ++++- .../facebook/presto/pravega/PravegaErrorCode.java | 5 ++++- .../presto/pravega/PravegaHandleResolver.java | 5 ++++- .../pravega/PravegaInternalFieldDescription.java | 5 ++++- .../presto/pravega/PravegaKVRecordSet.java | 5 ++++- .../facebook/presto/pravega/PravegaKVTable.java | 5 ++++- .../facebook/presto/pravega/PravegaMetadata.java | 5 ++++- .../presto/pravega/PravegaObjectSchema.java | 5 ++++- .../com/facebook/presto/pravega/PravegaPlugin.java | 5 ++++- .../facebook/presto/pravega/PravegaProperties.java | 5 ++++- .../presto/pravega/PravegaRecordCursor.java | 5 ++++- .../facebook/presto/pravega/PravegaRecordSet.java | 5 ++++- .../presto/pravega/PravegaRecordSetProvider.java | 5 ++++- .../presto/pravega/PravegaRecordValue.java | 5 ++++- .../presto/pravega/PravegaSegmentManager.java | 5 ++++- .../com/facebook/presto/pravega/PravegaSplit.java | 5 ++++- .../presto/pravega/PravegaSplitManager.java | 5 ++++- .../presto/pravega/PravegaStreamDescription.java | 5 ++++- .../pravega/PravegaStreamFieldDescription.java | 5 ++++- .../presto/pravega/PravegaStreamFieldGroup.java | 5 ++++- .../pravega/PravegaTableDescriptionSupplier.java | 5 ++++- .../presto/pravega/PravegaTableHandle.java | 5 ++++- .../presto/pravega/PravegaTableLayoutHandle.java | 5 ++++- .../facebook/presto/pravega/PravegaTableName.java | 5 ++++- .../presto/pravega/PravegaTransactionHandle.java | 5 ++++- .../facebook/presto/pravega/ProtobufCommon.java | 13 +++---------- .../com/facebook/presto/pravega/ReaderArgs.java | 5 ++++- .../com/facebook/presto/pravega/ReaderType.java | 5 ++++- .../presto/pravega/SegmentEventIterator.java | 5 ++++- .../presto/pravega/SegmentRangeIterator.java | 5 ++++- .../facebook/presto/pravega/StreamCutRange.java | 5 ++++- .../facebook/presto/pravega/StreamCutSupplier.java | 5 ++++- .../facebook/presto/pravega/TypedRecordValue.java | 5 ++++- .../facebook/presto/pravega/decoder/AvroEvent.java | 5 ++++- .../presto/pravega/decoder/AvroRowDecoder.java | 5 ++++- .../presto/pravega/decoder/AvroSerializer.java | 5 ++++- .../presto/pravega/decoder/BytesEvent.java | 5 ++++- .../presto/pravega/decoder/BytesEventDecoder.java | 5 ++++- .../presto/pravega/decoder/CsvRowDecoder.java | 5 ++++- .../presto/pravega/decoder/CsvSerializer.java | 5 ++++- .../presto/pravega/decoder/DecodableEvent.java | 5 ++++- .../presto/pravega/decoder/EventDecoder.java | 5 ++++- .../facebook/presto/pravega/decoder/JsonEvent.java | 5 ++++- .../presto/pravega/decoder/JsonRowDecoder.java | 5 ++++- .../presto/pravega/decoder/JsonSerializer.java | 5 ++++- .../presto/pravega/decoder/KVSerializer.java | 5 ++++- .../pravega/decoder/MultiSourceRowDecoder.java | 5 ++++- .../pravega/decoder/ProtobufColumnDecoder.java | 5 ++++- .../presto/pravega/decoder/ProtobufEvent.java | 5 ++++- .../presto/pravega/decoder/ProtobufRowDecoder.java | 5 ++++- .../presto/pravega/decoder/ProtobufSerializer.java | 5 ++++- .../presto/pravega/util/ByteBufferInputStream.java | 5 ++++- .../presto/pravega/util/PravegaNameUtils.java | 5 ++++- .../presto/pravega/util/PravegaSchemaUtils.java | 5 ++++- .../pravega/util/PravegaSerializationUtils.java | 5 ++++- .../facebook/presto/pravega/TestPravegaPlugin.java | 5 ++++- 70 files changed, 291 insertions(+), 80 deletions(-) create mode 100644 config/checkstyle/suppressions.xml create mode 100644 gradle.properties rename checkstyle.gradle => gradle/checkstyle.gradle (71%) diff --git a/build.gradle b/build.gradle index f153396..8e3d17c 100644 --- a/build.gradle +++ b/build.gradle @@ -8,6 +8,8 @@ plugins { id 'checkstyle' } +apply from: "$rootDir/gradle/checkstyle.gradle" + repositories { mavenLocal() maven { diff --git a/config/checkstyle/checkstyle.xml b/config/checkstyle/checkstyle.xml index d5791b2..4d8eccf 100644 --- a/config/checkstyle/checkstyle.xml +++ b/config/checkstyle/checkstyle.xml @@ -3,9 +3,9 @@ "-//Puppy Crawl//DTD Check Configuration 1.3//EN" "http://checkstyle.sourceforge.net/dtds/configuration_1_3.dtd"> - + + + diff --git a/config/checkstyle/suppressions.xml b/config/checkstyle/suppressions.xml new file mode 100644 index 0000000..f43a6bb --- /dev/null +++ b/config/checkstyle/suppressions.xml @@ -0,0 +1,14 @@ + + + + + + + + + + + + diff --git a/gradle.properties b/gradle.properties new file mode 100644 index 0000000..e63dd93 --- /dev/null +++ b/gradle.properties @@ -0,0 +1,11 @@ +# +# Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. +# +# 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 +# +checkstyleToolVersion=8.23 + diff --git a/checkstyle.gradle b/gradle/checkstyle.gradle similarity index 71% rename from checkstyle.gradle rename to gradle/checkstyle.gradle index 9a8830f..a6b662a 100644 --- a/checkstyle.gradle +++ b/gradle/checkstyle.gradle @@ -12,9 +12,8 @@ plugins.withId('checkstyle') { checkstyle { toolVersion = checkstyleToolVersion - configFile = new File(rootDir, "checkstyle/checkstyle.xml") - configProperties = [importControlFile: "$rootDir/checkstyle/import-control.xml", - suppressionsFile: "$rootDir/checkstyle/suppressions.xml"] + configFile = new File(rootDir, "config/checkstyle/checkstyle.xml") + configProperties = [suppressionsFile: "$rootDir/config/checkstyle/suppressions.xml"] } plugins.withId('java') { diff --git a/src/main/java/com/facebook/presto/pravega/ClassLoaderSafeRecordSetProvider.java b/src/main/java/com/facebook/presto/pravega/ClassLoaderSafeRecordSetProvider.java index e989994..09acd69 100644 --- a/src/main/java/com/facebook/presto/pravega/ClassLoaderSafeRecordSetProvider.java +++ b/src/main/java/com/facebook/presto/pravega/ClassLoaderSafeRecordSetProvider.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.presto.spi.ColumnHandle; diff --git a/src/main/java/com/facebook/presto/pravega/DelimRecordValue.java b/src/main/java/com/facebook/presto/pravega/DelimRecordValue.java index 91f666e..478db73 100644 --- a/src/main/java/com/facebook/presto/pravega/DelimRecordValue.java +++ b/src/main/java/com/facebook/presto/pravega/DelimRecordValue.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.google.common.base.Preconditions; diff --git a/src/main/java/com/facebook/presto/pravega/EventStreamIterator.java b/src/main/java/com/facebook/presto/pravega/EventStreamIterator.java index 7105c75..98410ef 100644 --- a/src/main/java/com/facebook/presto/pravega/EventStreamIterator.java +++ b/src/main/java/com/facebook/presto/pravega/EventStreamIterator.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.airlift.log.Logger; diff --git a/src/main/java/com/facebook/presto/pravega/ObjectType.java b/src/main/java/com/facebook/presto/pravega/ObjectType.java index f54c1de..ff66823 100644 --- a/src/main/java/com/facebook/presto/pravega/ObjectType.java +++ b/src/main/java/com/facebook/presto/pravega/ObjectType.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; public enum ObjectType diff --git a/src/main/java/com/facebook/presto/pravega/PravegaColumnHandle.java b/src/main/java/com/facebook/presto/pravega/PravegaColumnHandle.java index 0f4d136..fe6b4c0 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaColumnHandle.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaColumnHandle.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.presto.common.type.Type; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaConnector.java b/src/main/java/com/facebook/presto/pravega/PravegaConnector.java index 5a2fd0f..8f617c2 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaConnector.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaConnector.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.airlift.bootstrap.LifeCycleManager; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaConnectorConfig.java b/src/main/java/com/facebook/presto/pravega/PravegaConnectorConfig.java index 45b4f26..8bb0ead 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaConnectorConfig.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaConnectorConfig.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.airlift.configuration.Config; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaConnectorFactory.java b/src/main/java/com/facebook/presto/pravega/PravegaConnectorFactory.java index 8153659..7f8af58 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaConnectorFactory.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaConnectorFactory.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.airlift.bootstrap.Bootstrap; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaConnectorId.java b/src/main/java/com/facebook/presto/pravega/PravegaConnectorId.java index 8646321..f85aa6c 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaConnectorId.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaConnectorId.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import java.util.Objects; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaConnectorModule.java b/src/main/java/com/facebook/presto/pravega/PravegaConnectorModule.java index 78c2323..b2692bf 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaConnectorModule.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaConnectorModule.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.airlift.configuration.AbstractConfigurationAwareModule; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaErrorCode.java b/src/main/java/com/facebook/presto/pravega/PravegaErrorCode.java index 10137f7..2cb24b1 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaErrorCode.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaErrorCode.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.presto.spi.ErrorCode; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaHandleResolver.java b/src/main/java/com/facebook/presto/pravega/PravegaHandleResolver.java index 7f2fa6b..d11d85e 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaHandleResolver.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaHandleResolver.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.presto.spi.ColumnHandle; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaInternalFieldDescription.java b/src/main/java/com/facebook/presto/pravega/PravegaInternalFieldDescription.java index ce154ce..89c0c39 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaInternalFieldDescription.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaInternalFieldDescription.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.presto.common.type.BigintType; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaKVRecordSet.java b/src/main/java/com/facebook/presto/pravega/PravegaKVRecordSet.java index e5aae8d..777132c 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaKVRecordSet.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaKVRecordSet.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.airlift.log.Logger; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaKVTable.java b/src/main/java/com/facebook/presto/pravega/PravegaKVTable.java index 5b68ac6..25f9cf4 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaKVTable.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaKVTable.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaMetadata.java b/src/main/java/com/facebook/presto/pravega/PravegaMetadata.java index 012fd77..3314b88 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaMetadata.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaMetadata.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.airlift.log.Logger; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaObjectSchema.java b/src/main/java/com/facebook/presto/pravega/PravegaObjectSchema.java index c8f47a1..aa9feee 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaObjectSchema.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaObjectSchema.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaPlugin.java b/src/main/java/com/facebook/presto/pravega/PravegaPlugin.java index 785e283..6b160e4 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaPlugin.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaPlugin.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.presto.spi.Plugin; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaProperties.java b/src/main/java/com/facebook/presto/pravega/PravegaProperties.java index 4739a76..6922917 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaProperties.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaProperties.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.presto.spi.ConnectorSession; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaRecordCursor.java b/src/main/java/com/facebook/presto/pravega/PravegaRecordCursor.java index 3533273..a0a5661 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaRecordCursor.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaRecordCursor.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.airlift.log.Logger; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaRecordSet.java b/src/main/java/com/facebook/presto/pravega/PravegaRecordSet.java index 3c8a181..1d53c47 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaRecordSet.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaRecordSet.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.airlift.log.Logger; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaRecordSetProvider.java b/src/main/java/com/facebook/presto/pravega/PravegaRecordSetProvider.java index 2d7d6b5..cb658e8 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaRecordSetProvider.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaRecordSetProvider.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.airlift.log.Logger; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaRecordValue.java b/src/main/java/com/facebook/presto/pravega/PravegaRecordValue.java index 437d982..ca7a5c5 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaRecordValue.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaRecordValue.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import io.airlift.slice.Slice; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaSegmentManager.java b/src/main/java/com/facebook/presto/pravega/PravegaSegmentManager.java index c53535e..bb3604d 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaSegmentManager.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaSegmentManager.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.airlift.log.Logger; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaSplit.java b/src/main/java/com/facebook/presto/pravega/PravegaSplit.java index 54eab76..afced9b 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaSplit.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaSplit.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.presto.spi.ConnectorSplit; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaSplitManager.java b/src/main/java/com/facebook/presto/pravega/PravegaSplitManager.java index 3bb409b..c2454fe 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaSplitManager.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaSplitManager.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.airlift.log.Logger; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaStreamDescription.java b/src/main/java/com/facebook/presto/pravega/PravegaStreamDescription.java index bede43a..c27032f 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaStreamDescription.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaStreamDescription.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldDescription.java b/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldDescription.java index b2377d2..7818cf9 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldDescription.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldDescription.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.presto.common.type.Type; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldGroup.java b/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldGroup.java index 6ca3a5f..01a3ec7 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldGroup.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldGroup.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaTableDescriptionSupplier.java b/src/main/java/com/facebook/presto/pravega/PravegaTableDescriptionSupplier.java index 9a12ad2..3d0ffdf 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaTableDescriptionSupplier.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaTableDescriptionSupplier.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.airlift.json.JsonCodec; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaTableHandle.java b/src/main/java/com/facebook/presto/pravega/PravegaTableHandle.java index c79e769..46b7eb7 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaTableHandle.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaTableHandle.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.presto.spi.ConnectorTableHandle; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaTableLayoutHandle.java b/src/main/java/com/facebook/presto/pravega/PravegaTableLayoutHandle.java index ca8184e..8d5505d 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaTableLayoutHandle.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaTableLayoutHandle.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.presto.spi.ConnectorTableLayoutHandle; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaTableName.java b/src/main/java/com/facebook/presto/pravega/PravegaTableName.java index c522f6d..511e87f 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaTableName.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaTableName.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.presto.spi.SchemaTableName; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaTransactionHandle.java b/src/main/java/com/facebook/presto/pravega/PravegaTransactionHandle.java index 8f4dd7f..fcfcf70 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaTransactionHandle.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaTransactionHandle.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.presto.spi.connector.ConnectorTransactionHandle; diff --git a/src/main/java/com/facebook/presto/pravega/ProtobufCommon.java b/src/main/java/com/facebook/presto/pravega/ProtobufCommon.java index 2efd897..ecd521f 100644 --- a/src/main/java/com/facebook/presto/pravega/ProtobufCommon.java +++ b/src/main/java/com/facebook/presto/pravega/ProtobufCommon.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,18 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ -/* - * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. - * - * 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 - */ package com.facebook.presto.pravega; import com.google.common.base.Strings; diff --git a/src/main/java/com/facebook/presto/pravega/ReaderArgs.java b/src/main/java/com/facebook/presto/pravega/ReaderArgs.java index 93ce705..268da07 100644 --- a/src/main/java/com/facebook/presto/pravega/ReaderArgs.java +++ b/src/main/java/com/facebook/presto/pravega/ReaderArgs.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/src/main/java/com/facebook/presto/pravega/ReaderType.java b/src/main/java/com/facebook/presto/pravega/ReaderType.java index eabac02..736566f 100644 --- a/src/main/java/com/facebook/presto/pravega/ReaderType.java +++ b/src/main/java/com/facebook/presto/pravega/ReaderType.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; public enum ReaderType diff --git a/src/main/java/com/facebook/presto/pravega/SegmentEventIterator.java b/src/main/java/com/facebook/presto/pravega/SegmentEventIterator.java index 1250645..947fcaa 100644 --- a/src/main/java/com/facebook/presto/pravega/SegmentEventIterator.java +++ b/src/main/java/com/facebook/presto/pravega/SegmentEventIterator.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.airlift.log.Logger; diff --git a/src/main/java/com/facebook/presto/pravega/SegmentRangeIterator.java b/src/main/java/com/facebook/presto/pravega/SegmentRangeIterator.java index ad8ea86..bbb5a6f 100644 --- a/src/main/java/com/facebook/presto/pravega/SegmentRangeIterator.java +++ b/src/main/java/com/facebook/presto/pravega/SegmentRangeIterator.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.airlift.log.Logger; diff --git a/src/main/java/com/facebook/presto/pravega/StreamCutRange.java b/src/main/java/com/facebook/presto/pravega/StreamCutRange.java index ee44a9a..faeb984 100644 --- a/src/main/java/com/facebook/presto/pravega/StreamCutRange.java +++ b/src/main/java/com/facebook/presto/pravega/StreamCutRange.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import io.pravega.client.stream.StreamCut; diff --git a/src/main/java/com/facebook/presto/pravega/StreamCutSupplier.java b/src/main/java/com/facebook/presto/pravega/StreamCutSupplier.java index a0b2488..8a8eeb2 100644 --- a/src/main/java/com/facebook/presto/pravega/StreamCutSupplier.java +++ b/src/main/java/com/facebook/presto/pravega/StreamCutSupplier.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import io.pravega.client.batch.SegmentIterator; diff --git a/src/main/java/com/facebook/presto/pravega/TypedRecordValue.java b/src/main/java/com/facebook/presto/pravega/TypedRecordValue.java index 6b44a13..b5ad563 100644 --- a/src/main/java/com/facebook/presto/pravega/TypedRecordValue.java +++ b/src/main/java/com/facebook/presto/pravega/TypedRecordValue.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.presto.decoder.DecoderColumnHandle; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/AvroEvent.java b/src/main/java/com/facebook/presto/pravega/decoder/AvroEvent.java index 447d4a8..29e7ecf 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/AvroEvent.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/AvroEvent.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.decoder; import org.apache.avro.generic.GenericRecord; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/AvroRowDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/AvroRowDecoder.java index 791a8db..6e41e40 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/AvroRowDecoder.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/AvroRowDecoder.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.decoder; import com.facebook.presto.decoder.DecoderColumnHandle; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/AvroSerializer.java b/src/main/java/com/facebook/presto/pravega/decoder/AvroSerializer.java index 354c38f..9e8fbfb 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/AvroSerializer.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/AvroSerializer.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.decoder; import com.facebook.presto.pravega.util.ByteBufferInputStream; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/BytesEvent.java b/src/main/java/com/facebook/presto/pravega/decoder/BytesEvent.java index 3a85469..181d8a0 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/BytesEvent.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/BytesEvent.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.decoder; import java.nio.ByteBuffer; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/BytesEventDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/BytesEventDecoder.java index 75b5b06..ade109f 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/BytesEventDecoder.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/BytesEventDecoder.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.decoder; import com.facebook.presto.pravega.PravegaRecordValue; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/CsvRowDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/CsvRowDecoder.java index 52d5111..40b78ba 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/CsvRowDecoder.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/CsvRowDecoder.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.decoder; import com.facebook.presto.pravega.DelimRecordValue; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/CsvSerializer.java b/src/main/java/com/facebook/presto/pravega/decoder/CsvSerializer.java index ac736b6..06a57d5 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/CsvSerializer.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/CsvSerializer.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.decoder; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/DecodableEvent.java b/src/main/java/com/facebook/presto/pravega/decoder/DecodableEvent.java index dda18da..6c2704c 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/DecodableEvent.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/DecodableEvent.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.decoder; import com.fasterxml.jackson.databind.JsonNode; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/EventDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/EventDecoder.java index f3e6ae5..6e08b15 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/EventDecoder.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/EventDecoder.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.decoder; import com.facebook.presto.pravega.PravegaRecordValue; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/JsonEvent.java b/src/main/java/com/facebook/presto/pravega/decoder/JsonEvent.java index d1a5763..3394b17 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/JsonEvent.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/JsonEvent.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.decoder; import com.fasterxml.jackson.databind.JsonNode; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/JsonRowDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/JsonRowDecoder.java index bda699e..d3b1254 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/JsonRowDecoder.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/JsonRowDecoder.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.decoder; import com.facebook.presto.pravega.PravegaRecordValue; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/JsonSerializer.java b/src/main/java/com/facebook/presto/pravega/decoder/JsonSerializer.java index 635f21d..69af84c 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/JsonSerializer.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/JsonSerializer.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.decoder; import com.facebook.presto.pravega.util.ByteBufferInputStream; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/KVSerializer.java b/src/main/java/com/facebook/presto/pravega/decoder/KVSerializer.java index c6c0256..70d9362 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/KVSerializer.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/KVSerializer.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.decoder; import io.pravega.client.stream.Serializer; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/MultiSourceRowDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/MultiSourceRowDecoder.java index e83d776..f0475f1 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/MultiSourceRowDecoder.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/MultiSourceRowDecoder.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.decoder; import com.facebook.presto.decoder.DecoderColumnHandle; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufColumnDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufColumnDecoder.java index 540e6ca..8f84566 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufColumnDecoder.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufColumnDecoder.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.decoder; import com.facebook.presto.common.type.Type; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufEvent.java b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufEvent.java index d67dd4e..b2dc81d 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufEvent.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufEvent.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.decoder; import com.google.protobuf.DynamicMessage; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufRowDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufRowDecoder.java index 991899c..310ce09 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufRowDecoder.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufRowDecoder.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.decoder; import com.facebook.presto.decoder.DecoderColumnHandle; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufSerializer.java b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufSerializer.java index 7056211..b4980cd 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufSerializer.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufSerializer.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.decoder; import com.facebook.presto.pravega.util.ByteBufferInputStream; diff --git a/src/main/java/com/facebook/presto/pravega/util/ByteBufferInputStream.java b/src/main/java/com/facebook/presto/pravega/util/ByteBufferInputStream.java index c7ffc69..4208592 100644 --- a/src/main/java/com/facebook/presto/pravega/util/ByteBufferInputStream.java +++ b/src/main/java/com/facebook/presto/pravega/util/ByteBufferInputStream.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.util; import java.io.InputStream; diff --git a/src/main/java/com/facebook/presto/pravega/util/PravegaNameUtils.java b/src/main/java/com/facebook/presto/pravega/util/PravegaNameUtils.java index 1a32d29..33f3e77 100644 --- a/src/main/java/com/facebook/presto/pravega/util/PravegaNameUtils.java +++ b/src/main/java/com/facebook/presto/pravega/util/PravegaNameUtils.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.util; import com.facebook.presto.pravega.ObjectType; diff --git a/src/main/java/com/facebook/presto/pravega/util/PravegaSchemaUtils.java b/src/main/java/com/facebook/presto/pravega/util/PravegaSchemaUtils.java index 832d545..f82d97e 100644 --- a/src/main/java/com/facebook/presto/pravega/util/PravegaSchemaUtils.java +++ b/src/main/java/com/facebook/presto/pravega/util/PravegaSchemaUtils.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.util; import com.facebook.airlift.log.Logger; diff --git a/src/main/java/com/facebook/presto/pravega/util/PravegaSerializationUtils.java b/src/main/java/com/facebook/presto/pravega/util/PravegaSerializationUtils.java index b3af0d1..2a09ca9 100644 --- a/src/main/java/com/facebook/presto/pravega/util/PravegaSerializationUtils.java +++ b/src/main/java/com/facebook/presto/pravega/util/PravegaSerializationUtils.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega.util; import java.io.ByteArrayInputStream; diff --git a/src/test/java/com/facebook/presto/pravega/TestPravegaPlugin.java b/src/test/java/com/facebook/presto/pravega/TestPravegaPlugin.java index 39cb82e..a6ea06c 100644 --- a/src/test/java/com/facebook/presto/pravega/TestPravegaPlugin.java +++ b/src/test/java/com/facebook/presto/pravega/TestPravegaPlugin.java @@ -1,4 +1,6 @@ /* + * Copyright (c) Pravega Authors. + * * 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 @@ -9,8 +11,9 @@ * 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. + * limitations under the License.Copyright (c) Pravega Authors. */ + package com.facebook.presto.pravega; import com.facebook.presto.spi.connector.Connector; From 24c700933897d8e6c40caa09e2e15808158c9f9f Mon Sep 17 00:00:00 2001 From: Chip Maurer Date: Tue, 9 Mar 2021 14:07:40 -0500 Subject: [PATCH 5/8] Remove extra words in header. Replace checkstyle to compare with HEADER. Add comments to HEADER --- HEADER | 28 ++++++++++--------- config/checkstyle/checkstyle.xml | 4 +-- .../ClassLoaderSafeRecordSetProvider.java | 2 +- .../presto/pravega/DelimRecordValue.java | 2 +- .../presto/pravega/EventStreamIterator.java | 2 +- .../facebook/presto/pravega/ObjectType.java | 2 +- .../presto/pravega/PravegaColumnHandle.java | 2 +- .../presto/pravega/PravegaConnector.java | 2 +- .../pravega/PravegaConnectorConfig.java | 2 +- .../pravega/PravegaConnectorFactory.java | 2 +- .../presto/pravega/PravegaConnectorId.java | 2 +- .../pravega/PravegaConnectorModule.java | 2 +- .../presto/pravega/PravegaErrorCode.java | 2 +- .../presto/pravega/PravegaHandleResolver.java | 2 +- .../PravegaInternalFieldDescription.java | 2 +- .../presto/pravega/PravegaKVRecordSet.java | 2 +- .../presto/pravega/PravegaKVTable.java | 2 +- .../presto/pravega/PravegaMetadata.java | 2 +- .../presto/pravega/PravegaObjectSchema.java | 2 +- .../presto/pravega/PravegaPlugin.java | 2 +- .../presto/pravega/PravegaProperties.java | 2 +- .../presto/pravega/PravegaRecordCursor.java | 2 +- .../presto/pravega/PravegaRecordSet.java | 2 +- .../pravega/PravegaRecordSetProvider.java | 2 +- .../presto/pravega/PravegaRecordValue.java | 2 +- .../presto/pravega/PravegaSegmentManager.java | 2 +- .../facebook/presto/pravega/PravegaSplit.java | 2 +- .../presto/pravega/PravegaSplitManager.java | 2 +- .../pravega/PravegaStreamDescription.java | 2 +- .../PravegaStreamFieldDescription.java | 2 +- .../pravega/PravegaStreamFieldGroup.java | 2 +- .../PravegaTableDescriptionSupplier.java | 2 +- .../presto/pravega/PravegaTableHandle.java | 2 +- .../pravega/PravegaTableLayoutHandle.java | 2 +- .../presto/pravega/PravegaTableName.java | 2 +- .../pravega/PravegaTransactionHandle.java | 2 +- .../presto/pravega/ProtobufCommon.java | 2 +- .../facebook/presto/pravega/ReaderArgs.java | 2 +- .../facebook/presto/pravega/ReaderType.java | 2 +- .../presto/pravega/SegmentEventIterator.java | 2 +- .../presto/pravega/SegmentRangeIterator.java | 2 +- .../presto/pravega/StreamCutRange.java | 2 +- .../presto/pravega/StreamCutSupplier.java | 2 +- .../presto/pravega/TypedRecordValue.java | 2 +- .../presto/pravega/decoder/AvroEvent.java | 2 +- .../pravega/decoder/AvroRowDecoder.java | 2 +- .../pravega/decoder/AvroSerializer.java | 2 +- .../presto/pravega/decoder/BytesEvent.java | 2 +- .../pravega/decoder/BytesEventDecoder.java | 2 +- .../presto/pravega/decoder/CsvRowDecoder.java | 2 +- .../presto/pravega/decoder/CsvSerializer.java | 2 +- .../pravega/decoder/DecodableEvent.java | 2 +- .../presto/pravega/decoder/EventDecoder.java | 2 +- .../presto/pravega/decoder/JsonEvent.java | 2 +- .../pravega/decoder/JsonRowDecoder.java | 2 +- .../pravega/decoder/JsonSerializer.java | 2 +- .../presto/pravega/decoder/KVSerializer.java | 2 +- .../decoder/MultiSourceRowDecoder.java | 2 +- .../decoder/ProtobufColumnDecoder.java | 2 +- .../presto/pravega/decoder/ProtobufEvent.java | 2 +- .../pravega/decoder/ProtobufRowDecoder.java | 2 +- .../pravega/decoder/ProtobufSerializer.java | 2 +- .../pravega/util/ByteBufferInputStream.java | 2 +- .../presto/pravega/util/PravegaNameUtils.java | 2 +- .../pravega/util/PravegaSchemaUtils.java | 2 +- .../util/PravegaSerializationUtils.java | 2 +- .../presto/pravega/TestPravegaPlugin.java | 2 +- 67 files changed, 82 insertions(+), 80 deletions(-) diff --git a/HEADER b/HEADER index 74ffe5c..cc5300b 100644 --- a/HEADER +++ b/HEADER @@ -1,13 +1,15 @@ -Copyright (c) Pravega Authors. - -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. +/* + * Copyright (c) Pravega Authors. + * + * 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. + */ diff --git a/config/checkstyle/checkstyle.xml b/config/checkstyle/checkstyle.xml index 4d8eccf..a649adb 100644 --- a/config/checkstyle/checkstyle.xml +++ b/config/checkstyle/checkstyle.xml @@ -3,8 +3,8 @@ "-//Puppy Crawl//DTD Check Configuration 1.3//EN" "http://checkstyle.sourceforge.net/dtds/configuration_1_3.dtd"> - - + + diff --git a/src/main/java/com/facebook/presto/pravega/ClassLoaderSafeRecordSetProvider.java b/src/main/java/com/facebook/presto/pravega/ClassLoaderSafeRecordSetProvider.java index 09acd69..c1fe93e 100644 --- a/src/main/java/com/facebook/presto/pravega/ClassLoaderSafeRecordSetProvider.java +++ b/src/main/java/com/facebook/presto/pravega/ClassLoaderSafeRecordSetProvider.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/DelimRecordValue.java b/src/main/java/com/facebook/presto/pravega/DelimRecordValue.java index 478db73..e1c0ad3 100644 --- a/src/main/java/com/facebook/presto/pravega/DelimRecordValue.java +++ b/src/main/java/com/facebook/presto/pravega/DelimRecordValue.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/EventStreamIterator.java b/src/main/java/com/facebook/presto/pravega/EventStreamIterator.java index 98410ef..1645219 100644 --- a/src/main/java/com/facebook/presto/pravega/EventStreamIterator.java +++ b/src/main/java/com/facebook/presto/pravega/EventStreamIterator.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/ObjectType.java b/src/main/java/com/facebook/presto/pravega/ObjectType.java index ff66823..5f4e594 100644 --- a/src/main/java/com/facebook/presto/pravega/ObjectType.java +++ b/src/main/java/com/facebook/presto/pravega/ObjectType.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaColumnHandle.java b/src/main/java/com/facebook/presto/pravega/PravegaColumnHandle.java index fe6b4c0..4a1fcd1 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaColumnHandle.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaColumnHandle.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaConnector.java b/src/main/java/com/facebook/presto/pravega/PravegaConnector.java index 8f617c2..9ea858c 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaConnector.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaConnector.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaConnectorConfig.java b/src/main/java/com/facebook/presto/pravega/PravegaConnectorConfig.java index 8bb0ead..9894fb9 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaConnectorConfig.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaConnectorConfig.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaConnectorFactory.java b/src/main/java/com/facebook/presto/pravega/PravegaConnectorFactory.java index 7f8af58..cf20ff3 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaConnectorFactory.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaConnectorFactory.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaConnectorId.java b/src/main/java/com/facebook/presto/pravega/PravegaConnectorId.java index f85aa6c..0ccbd5e 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaConnectorId.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaConnectorId.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaConnectorModule.java b/src/main/java/com/facebook/presto/pravega/PravegaConnectorModule.java index b2692bf..6f67026 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaConnectorModule.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaConnectorModule.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaErrorCode.java b/src/main/java/com/facebook/presto/pravega/PravegaErrorCode.java index 2cb24b1..b18994b 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaErrorCode.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaErrorCode.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaHandleResolver.java b/src/main/java/com/facebook/presto/pravega/PravegaHandleResolver.java index d11d85e..8e554f8 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaHandleResolver.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaHandleResolver.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaInternalFieldDescription.java b/src/main/java/com/facebook/presto/pravega/PravegaInternalFieldDescription.java index 89c0c39..1746c3a 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaInternalFieldDescription.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaInternalFieldDescription.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaKVRecordSet.java b/src/main/java/com/facebook/presto/pravega/PravegaKVRecordSet.java index 777132c..699900b 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaKVRecordSet.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaKVRecordSet.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaKVTable.java b/src/main/java/com/facebook/presto/pravega/PravegaKVTable.java index 25f9cf4..6250dc9 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaKVTable.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaKVTable.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaMetadata.java b/src/main/java/com/facebook/presto/pravega/PravegaMetadata.java index 3314b88..fa1d4f1 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaMetadata.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaMetadata.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaObjectSchema.java b/src/main/java/com/facebook/presto/pravega/PravegaObjectSchema.java index aa9feee..4e44d3c 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaObjectSchema.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaObjectSchema.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaPlugin.java b/src/main/java/com/facebook/presto/pravega/PravegaPlugin.java index 6b160e4..ec927c6 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaPlugin.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaPlugin.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaProperties.java b/src/main/java/com/facebook/presto/pravega/PravegaProperties.java index 6922917..3ef3cd5 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaProperties.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaProperties.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaRecordCursor.java b/src/main/java/com/facebook/presto/pravega/PravegaRecordCursor.java index a0a5661..480c4f9 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaRecordCursor.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaRecordCursor.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaRecordSet.java b/src/main/java/com/facebook/presto/pravega/PravegaRecordSet.java index 1d53c47..f25b739 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaRecordSet.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaRecordSet.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaRecordSetProvider.java b/src/main/java/com/facebook/presto/pravega/PravegaRecordSetProvider.java index cb658e8..e3efcb0 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaRecordSetProvider.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaRecordSetProvider.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaRecordValue.java b/src/main/java/com/facebook/presto/pravega/PravegaRecordValue.java index ca7a5c5..0f861c7 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaRecordValue.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaRecordValue.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaSegmentManager.java b/src/main/java/com/facebook/presto/pravega/PravegaSegmentManager.java index bb3604d..2faa718 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaSegmentManager.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaSegmentManager.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaSplit.java b/src/main/java/com/facebook/presto/pravega/PravegaSplit.java index afced9b..930460b 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaSplit.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaSplit.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaSplitManager.java b/src/main/java/com/facebook/presto/pravega/PravegaSplitManager.java index c2454fe..e47dc41 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaSplitManager.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaSplitManager.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaStreamDescription.java b/src/main/java/com/facebook/presto/pravega/PravegaStreamDescription.java index c27032f..503e107 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaStreamDescription.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaStreamDescription.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldDescription.java b/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldDescription.java index 7818cf9..148e95e 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldDescription.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldDescription.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldGroup.java b/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldGroup.java index 01a3ec7..6a6137a 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldGroup.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldGroup.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaTableDescriptionSupplier.java b/src/main/java/com/facebook/presto/pravega/PravegaTableDescriptionSupplier.java index 3d0ffdf..e8fc00c 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaTableDescriptionSupplier.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaTableDescriptionSupplier.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaTableHandle.java b/src/main/java/com/facebook/presto/pravega/PravegaTableHandle.java index 46b7eb7..f8b5534 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaTableHandle.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaTableHandle.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaTableLayoutHandle.java b/src/main/java/com/facebook/presto/pravega/PravegaTableLayoutHandle.java index 8d5505d..31367af 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaTableLayoutHandle.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaTableLayoutHandle.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaTableName.java b/src/main/java/com/facebook/presto/pravega/PravegaTableName.java index 511e87f..42007c9 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaTableName.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaTableName.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/PravegaTransactionHandle.java b/src/main/java/com/facebook/presto/pravega/PravegaTransactionHandle.java index fcfcf70..e47c7c8 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaTransactionHandle.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaTransactionHandle.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/ProtobufCommon.java b/src/main/java/com/facebook/presto/pravega/ProtobufCommon.java index ecd521f..8f0e7f4 100644 --- a/src/main/java/com/facebook/presto/pravega/ProtobufCommon.java +++ b/src/main/java/com/facebook/presto/pravega/ProtobufCommon.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/ReaderArgs.java b/src/main/java/com/facebook/presto/pravega/ReaderArgs.java index 268da07..fbfbc0b 100644 --- a/src/main/java/com/facebook/presto/pravega/ReaderArgs.java +++ b/src/main/java/com/facebook/presto/pravega/ReaderArgs.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/ReaderType.java b/src/main/java/com/facebook/presto/pravega/ReaderType.java index 736566f..174035f 100644 --- a/src/main/java/com/facebook/presto/pravega/ReaderType.java +++ b/src/main/java/com/facebook/presto/pravega/ReaderType.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/SegmentEventIterator.java b/src/main/java/com/facebook/presto/pravega/SegmentEventIterator.java index 947fcaa..67876c6 100644 --- a/src/main/java/com/facebook/presto/pravega/SegmentEventIterator.java +++ b/src/main/java/com/facebook/presto/pravega/SegmentEventIterator.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/SegmentRangeIterator.java b/src/main/java/com/facebook/presto/pravega/SegmentRangeIterator.java index bbb5a6f..408eba0 100644 --- a/src/main/java/com/facebook/presto/pravega/SegmentRangeIterator.java +++ b/src/main/java/com/facebook/presto/pravega/SegmentRangeIterator.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/StreamCutRange.java b/src/main/java/com/facebook/presto/pravega/StreamCutRange.java index faeb984..60859c3 100644 --- a/src/main/java/com/facebook/presto/pravega/StreamCutRange.java +++ b/src/main/java/com/facebook/presto/pravega/StreamCutRange.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/StreamCutSupplier.java b/src/main/java/com/facebook/presto/pravega/StreamCutSupplier.java index 8a8eeb2..940b306 100644 --- a/src/main/java/com/facebook/presto/pravega/StreamCutSupplier.java +++ b/src/main/java/com/facebook/presto/pravega/StreamCutSupplier.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/TypedRecordValue.java b/src/main/java/com/facebook/presto/pravega/TypedRecordValue.java index b5ad563..6bebfe5 100644 --- a/src/main/java/com/facebook/presto/pravega/TypedRecordValue.java +++ b/src/main/java/com/facebook/presto/pravega/TypedRecordValue.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/AvroEvent.java b/src/main/java/com/facebook/presto/pravega/decoder/AvroEvent.java index 29e7ecf..a866a88 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/AvroEvent.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/AvroEvent.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.decoder; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/AvroRowDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/AvroRowDecoder.java index 6e41e40..5fc481f 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/AvroRowDecoder.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/AvroRowDecoder.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.decoder; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/AvroSerializer.java b/src/main/java/com/facebook/presto/pravega/decoder/AvroSerializer.java index 9e8fbfb..9e78611 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/AvroSerializer.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/AvroSerializer.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.decoder; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/BytesEvent.java b/src/main/java/com/facebook/presto/pravega/decoder/BytesEvent.java index 181d8a0..a145ca8 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/BytesEvent.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/BytesEvent.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.decoder; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/BytesEventDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/BytesEventDecoder.java index ade109f..988bb77 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/BytesEventDecoder.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/BytesEventDecoder.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.decoder; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/CsvRowDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/CsvRowDecoder.java index 40b78ba..5fd78e1 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/CsvRowDecoder.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/CsvRowDecoder.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.decoder; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/CsvSerializer.java b/src/main/java/com/facebook/presto/pravega/decoder/CsvSerializer.java index 06a57d5..de64f98 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/CsvSerializer.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/CsvSerializer.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.decoder; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/DecodableEvent.java b/src/main/java/com/facebook/presto/pravega/decoder/DecodableEvent.java index 6c2704c..ebcb574 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/DecodableEvent.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/DecodableEvent.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.decoder; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/EventDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/EventDecoder.java index 6e08b15..3d234cb 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/EventDecoder.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/EventDecoder.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.decoder; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/JsonEvent.java b/src/main/java/com/facebook/presto/pravega/decoder/JsonEvent.java index 3394b17..e99835a 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/JsonEvent.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/JsonEvent.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.decoder; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/JsonRowDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/JsonRowDecoder.java index d3b1254..fb3cee3 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/JsonRowDecoder.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/JsonRowDecoder.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.decoder; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/JsonSerializer.java b/src/main/java/com/facebook/presto/pravega/decoder/JsonSerializer.java index 69af84c..46047a0 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/JsonSerializer.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/JsonSerializer.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.decoder; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/KVSerializer.java b/src/main/java/com/facebook/presto/pravega/decoder/KVSerializer.java index 70d9362..ef090b5 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/KVSerializer.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/KVSerializer.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.decoder; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/MultiSourceRowDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/MultiSourceRowDecoder.java index f0475f1..b30e2fb 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/MultiSourceRowDecoder.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/MultiSourceRowDecoder.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.decoder; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufColumnDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufColumnDecoder.java index 8f84566..d0f36f7 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufColumnDecoder.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufColumnDecoder.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.decoder; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufEvent.java b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufEvent.java index b2dc81d..c0ff397 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufEvent.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufEvent.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.decoder; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufRowDecoder.java b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufRowDecoder.java index 310ce09..92de896 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufRowDecoder.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufRowDecoder.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.decoder; diff --git a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufSerializer.java b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufSerializer.java index b4980cd..be4e77f 100644 --- a/src/main/java/com/facebook/presto/pravega/decoder/ProtobufSerializer.java +++ b/src/main/java/com/facebook/presto/pravega/decoder/ProtobufSerializer.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.decoder; diff --git a/src/main/java/com/facebook/presto/pravega/util/ByteBufferInputStream.java b/src/main/java/com/facebook/presto/pravega/util/ByteBufferInputStream.java index 4208592..aee9743 100644 --- a/src/main/java/com/facebook/presto/pravega/util/ByteBufferInputStream.java +++ b/src/main/java/com/facebook/presto/pravega/util/ByteBufferInputStream.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.util; diff --git a/src/main/java/com/facebook/presto/pravega/util/PravegaNameUtils.java b/src/main/java/com/facebook/presto/pravega/util/PravegaNameUtils.java index 33f3e77..19bea11 100644 --- a/src/main/java/com/facebook/presto/pravega/util/PravegaNameUtils.java +++ b/src/main/java/com/facebook/presto/pravega/util/PravegaNameUtils.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.util; diff --git a/src/main/java/com/facebook/presto/pravega/util/PravegaSchemaUtils.java b/src/main/java/com/facebook/presto/pravega/util/PravegaSchemaUtils.java index f82d97e..645f34e 100644 --- a/src/main/java/com/facebook/presto/pravega/util/PravegaSchemaUtils.java +++ b/src/main/java/com/facebook/presto/pravega/util/PravegaSchemaUtils.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.util; diff --git a/src/main/java/com/facebook/presto/pravega/util/PravegaSerializationUtils.java b/src/main/java/com/facebook/presto/pravega/util/PravegaSerializationUtils.java index 2a09ca9..70e58ea 100644 --- a/src/main/java/com/facebook/presto/pravega/util/PravegaSerializationUtils.java +++ b/src/main/java/com/facebook/presto/pravega/util/PravegaSerializationUtils.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega.util; diff --git a/src/test/java/com/facebook/presto/pravega/TestPravegaPlugin.java b/src/test/java/com/facebook/presto/pravega/TestPravegaPlugin.java index a6ea06c..e308be8 100644 --- a/src/test/java/com/facebook/presto/pravega/TestPravegaPlugin.java +++ b/src/test/java/com/facebook/presto/pravega/TestPravegaPlugin.java @@ -11,7 +11,7 @@ * 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.Copyright (c) Pravega Authors. + * limitations under the License. */ package com.facebook.presto.pravega; From ad6be4eaa7eb0eb34263e19d79c06514609c8a37 Mon Sep 17 00:00:00 2001 From: Chip Maurer Date: Tue, 9 Mar 2021 14:13:11 -0500 Subject: [PATCH 6/8] Making minor change because I forgot -s on last commit Signed-off-by: Chip Maurer --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 8e3d17c..ef7c098 100644 --- a/build.gradle +++ b/build.gradle @@ -1,5 +1,5 @@ /* - * This file was generated by the Gradle 'init' task. + * Build gradle file for the Pravega connector */ plugins { From 955164db55874be994d115eebc7a6a6e779e5369 Mon Sep 17 00:00:00 2001 From: Chip Maurer Date: Wed, 10 Mar 2021 11:10:30 -0500 Subject: [PATCH 7/8] Updated README to document build and development process Signed-off-by: Chip Maurer --- README.md | 79 ++++++++++++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 78 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index e53115e..ff0abe6 100644 --- a/README.md +++ b/README.md @@ -1,2 +1,79 @@ # presto-connector -Pravega connector for Presto +Pravega Presto connector + +Presto is a distributed SQL query engine for big data. Presto uses connectors to query storage from different storage sources. This repository contains the code for a connector (the Pravega Presto connector) to query storage from Pravega streams. To learn more about PrestoDB, visit https://prestodb.io + +Pravega is an open source distributed storage service implementing Streams. It offers Stream as the main primitive for the foundation of reliable storage systems: a high-performance, durable, elastic, and unlimited append-only byte stream with strict ordering and consistency. To learn more about Pravega, visit https://pravega.io + +See the [User Manual](https://prestodb.github.io/docs/current/) for Presto deployment instructions and end user documentation. + +## Requirements + +To build and run the Pravega Presto connector, you must meet the following requirements: + +* Linux +* Java 8 Update 151 or higher (8u151+), 64-bit. Both Oracle JDK and OpenJDK are supported. +* Gradle 6.5.1+ (for building) +* Python 2.7+ (for running with the launcher script) +* Pravega version 0.9.0 or higher +* Pravega Schema Registry version 0.2.0 or higher + +## Building Presto + +Pravega Presto connector is a standard Gradle project. Simply run the following command from the project root directory: + + ./gradlew clean build + +On the first build, Gradle will download all the dependencies from various locations of the internet and cache them in the local repository (`~/.gradle/caches`), which can take a considerable amount of time. Subsequent builds will be faster. + +Pravega Presto connector has a set of unit tests that can take several minutes to run. You can run the tests using this command: + + ./gradlew test + +## Running Presto + +The plugin file that gets created during the build process is: ./build/distributions/pravega-.tar.gz. This file can be untar'd in the /usr/lib/presto/lib/plugins directory of a running Presto installation. Like all Presto connectors, the Pravega Presto connector uses a properties files to point to the storage provider (e.g. Pravega controller). Create a properties file similar to below, but replace the # characters with the appropriate IP address of the Pravega Controller and the Pravega Schema Registry server of your configuration. + +[root@lrmk226 ~]# cat /etc/presto/catalog/pravega.properties +connector.name=pravega +pravega.controller=tcp://##.###.###.###:9090 +pravega.schema-registry=http://##.###.###.###:9092 + +## Running Presto in your IDE + +After building Presto for the first time, you can load the project into your IDE and run the server in your IDE. We recommend using [IntelliJ IDEA](http://www.jetbrains.com/idea/). Because Pravega Presto connectoris a standard Gradle project, you can import it into your IDE. In IntelliJ, choose Import Project from the Quick Start box and point it to the root of the source tree. IntelliJ will identify the *.gradle files and prompt you to confirm. + +After opening the project in IntelliJ, double check that the Java SDK is properly configured for the project: + +* Open the File menu and select Project Structure +* In the SDKs section, ensure that a 1.8 JDK is selected (create one if none exist) +* In the Project section, ensure the Project language level is set to 8.0 as Presto makes use of several Java 8 language features + +Use the following options to create a run configuration that runs the Presto server using the Pravega Presto connector: + +* Main Class: 'com.facebook.presto.server.PrestoServer' +* VM Options: '-ea -XX:+UseG1GC -XX:G1HeapRegionSize=32M -XX:+UseGCOverheadLimit -XX:+ExplicitGCInvokesConcurrent -Xmx2G -Dconfig=etc/config.properties -Dcom.sun.xml.bind.v2.bytecode.ClassTailor.noOptimize=true -Dlog.levels-file=etc/log.properties' +* Working directory: '/root/presto' +* Use classpath of module: 'pravega.main' + +The working directory should be manually created and set to where the configuration properties files are located on your host. + +[root@lrmk226 ~]# find /root/presto -ls +537429121 0 drwxr-xr-x 4 root root 28 Mar 10 10:40 /root/presto +537429123 0 drwxr-xr-x 4 root root 97 Mar 2 17:12 /root/presto/etc +805328026 0 drwxr-xr-x 2 root root 32 Mar 3 15:58 /root/presto/etc/catalog +816318053 4 -rw-r--r-- 1 root root 119 Mar 3 15:58 /root/presto/etc/catalog/pravega.properties +272367596 0 drwxr-xr-x 2 root root 6 Mar 2 17:04 /root/presto/etc/ecs +537429402 4 -rw-r--r-- 1 root root 854 Mar 2 17:10 /root/presto/etc/config.properties +537429389 4 -rw-r--r-- 1 root root 351 Mar 2 17:11 /root/presto/etc/jvm.config +537435775 4 -rw-r--r-- 1 root root 378 Mar 2 17:12 /root/presto/etc/log.properties +575879 0 drwxr-xr-x 3 root root 17 Mar 2 17:17 /root/presto/var +268833640 0 drwxr-xr-x 2 root root 30 Mar 10 10:42 /root/presto/var/log +272373839 12 -rw-r--r-- 1 root root 11230 Mar 10 10:42 /root/presto/var/log/http-request.log + +Create the pravega.properties file as previously described. + +## Schema Definitions + +Currently, you must manually create schema definitions using a JSON file. In future releases, the 'CREATE TABLE' Presto command will be available. + From 4ea6809fda9b6c0483c6bb1bcf35e8437cbb8bc2 Mon Sep 17 00:00:00 2001 From: Chip Maurer Date: Wed, 10 Mar 2021 11:24:11 -0500 Subject: [PATCH 8/8] Cleanup some code samples Signed-off-by: Chip Maurer --- README.md | 34 +++++++++++++++++----------------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/README.md b/README.md index ff0abe6..2900ecb 100644 --- a/README.md +++ b/README.md @@ -24,7 +24,7 @@ Pravega Presto connector is a standard Gradle project. Simply run the following ./gradlew clean build -On the first build, Gradle will download all the dependencies from various locations of the internet and cache them in the local repository (`~/.gradle/caches`), which can take a considerable amount of time. Subsequent builds will be faster. +On the first build, Gradle will download all the dependencies from various locations of the internet and cache them in the local repository (`~/.gradle / caches `), which can take a considerable amount of time. Subsequent builds will be faster. Pravega Presto connector has a set of unit tests that can take several minutes to run. You can run the tests using this command: @@ -34,10 +34,10 @@ Pravega Presto connector has a set of unit tests that can take several minutes t The plugin file that gets created during the build process is: ./build/distributions/pravega-.tar.gz. This file can be untar'd in the /usr/lib/presto/lib/plugins directory of a running Presto installation. Like all Presto connectors, the Pravega Presto connector uses a properties files to point to the storage provider (e.g. Pravega controller). Create a properties file similar to below, but replace the # characters with the appropriate IP address of the Pravega Controller and the Pravega Schema Registry server of your configuration. -[root@lrmk226 ~]# cat /etc/presto/catalog/pravega.properties -connector.name=pravega -pravega.controller=tcp://##.###.###.###:9090 -pravega.schema-registry=http://##.###.###.###:9092 + [root@lrmk226 ~]# cat /etc/presto/catalog/pravega.properties + connector.name=pravega + pravega.controller=tcp://##.###.###.###:9090 + pravega.schema-registry=http://##.###.###.###:9092 ## Running Presto in your IDE @@ -58,18 +58,18 @@ Use the following options to create a run configuration that runs the Presto ser The working directory should be manually created and set to where the configuration properties files are located on your host. -[root@lrmk226 ~]# find /root/presto -ls -537429121 0 drwxr-xr-x 4 root root 28 Mar 10 10:40 /root/presto -537429123 0 drwxr-xr-x 4 root root 97 Mar 2 17:12 /root/presto/etc -805328026 0 drwxr-xr-x 2 root root 32 Mar 3 15:58 /root/presto/etc/catalog -816318053 4 -rw-r--r-- 1 root root 119 Mar 3 15:58 /root/presto/etc/catalog/pravega.properties -272367596 0 drwxr-xr-x 2 root root 6 Mar 2 17:04 /root/presto/etc/ecs -537429402 4 -rw-r--r-- 1 root root 854 Mar 2 17:10 /root/presto/etc/config.properties -537429389 4 -rw-r--r-- 1 root root 351 Mar 2 17:11 /root/presto/etc/jvm.config -537435775 4 -rw-r--r-- 1 root root 378 Mar 2 17:12 /root/presto/etc/log.properties -575879 0 drwxr-xr-x 3 root root 17 Mar 2 17:17 /root/presto/var -268833640 0 drwxr-xr-x 2 root root 30 Mar 10 10:42 /root/presto/var/log -272373839 12 -rw-r--r-- 1 root root 11230 Mar 10 10:42 /root/presto/var/log/http-request.log + [root@lrmk226 ~]# find /root/presto -ls + 537429121 0 drwxr-xr-x 4 root root 28 Mar 10 10:40 /root/presto + 537429123 0 drwxr-xr-x 4 root root 97 Mar 2 17:12 /root/presto/etc + 805328026 0 drwxr-xr-x 2 root root 32 Mar 3 15:58 /root/presto/etc/catalog + 816318053 4 -rw-r--r-- 1 root root 119 Mar 3 15:58 /root/presto/etc/catalog/pravega.properties + 272367596 0 drwxr-xr-x 2 root root 6 Mar 2 17:04 /root/presto/etc/ecs + 537429402 4 -rw-r--r-- 1 root root 854 Mar 2 17:10 /root/presto/etc/config.properties + 537429389 4 -rw-r--r-- 1 root root 351 Mar 2 17:11 /root/presto/etc/jvm.config + 537435775 4 -rw-r--r-- 1 root root 378 Mar 2 17:12 /root/presto/etc/log.properties + 575879 0 drwxr-xr-x 3 root root 17 Mar 2 17:17 /root/presto/var + 268833640 0 drwxr-xr-x 2 root root 30 Mar 10 10:42 /root/presto/var/log + 272373839 12 -rw-r--r-- 1 root root 11230 Mar 10 10:42 /root/presto/var/log/http-request.log Create the pravega.properties file as previously described.