From ca8367cc5be48b615c1c0a7b6d42c29b507e6d11 Mon Sep 17 00:00:00 2001 From: Michael Marshall Date: Fri, 3 Feb 2023 03:52:22 +0800 Subject: [PATCH 1/2] [security] Proxy to broker and broker to broker hostname verification --- .../TransactionMarkerChannelInitializer.java | 24 ++++++++++++--- .../TransactionMarkerChannelManager.java | 11 +++++-- .../handlers/kop/utils/ssl/SSLUtils.java | 5 ++-- .../kop/security/KafkaSSLChannelTest.java | 25 ++++++++++------ .../KafkaSSLChannelWithClientAuthTest.java | 28 +++++++----------- .../ssl/certificate/broker.keystore.jks | Bin 3991 -> 2255 bytes .../ssl/certificate/broker.truststore.jks | Bin 944 -> 969 bytes .../ssl/certificate/client.keystore.jks | Bin 3993 -> 2255 bytes .../ssl/certificate/client.truststore.jks | Bin 944 -> 970 bytes 9 files changed, 59 insertions(+), 34 deletions(-) diff --git a/kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/coordinator/transaction/TransactionMarkerChannelInitializer.java b/kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/coordinator/transaction/TransactionMarkerChannelInitializer.java index 8a0d4de326..142c2e9521 100644 --- a/kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/coordinator/transaction/TransactionMarkerChannelInitializer.java +++ b/kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/coordinator/transaction/TransactionMarkerChannelInitializer.java @@ -16,6 +16,7 @@ import static io.streamnative.pulsar.handlers.kop.KafkaChannelInitializer.MAX_FRAME_LENGTH; import static io.streamnative.pulsar.handlers.kop.KafkaProtocolHandler.TLS_HANDLER; +import io.netty.channel.Channel; import io.netty.channel.ChannelInitializer; import io.netty.channel.socket.SocketChannel; import io.netty.handler.codec.LengthFieldBasedFrameDecoder; @@ -23,6 +24,7 @@ import io.netty.handler.ssl.SslHandler; import io.streamnative.pulsar.handlers.kop.KafkaServiceConfiguration; import io.streamnative.pulsar.handlers.kop.utils.ssl.SSLUtils; +import java.util.concurrent.CompletableFuture; import org.eclipse.jetty.util.ssl.SslContextFactory; /** @@ -50,13 +52,27 @@ public TransactionMarkerChannelInitializer(KafkaServiceConfiguration kafkaConfig @Override protected void initChannel(SocketChannel ch) throws Exception { - if (this.enableTls) { - ch.pipeline().addLast(TLS_HANDLER, - new SslHandler(SSLUtils.createClientSslEngine(sslContextFactory))); - } ch.pipeline().addLast(lengthFieldPrepender); ch.pipeline().addLast("frameDecoder", new LengthFieldBasedFrameDecoder(MAX_FRAME_LENGTH, 0, 4, 0, 4)); ch.pipeline().addLast("txnHandler", new TransactionMarkerChannelHandler(transactionMarkerChannelManager)); } + + protected CompletableFuture initTls(Channel ch, String host, int port) { + if (this.enableTls) { + CompletableFuture initTlsFuture = new CompletableFuture<>(); + ch.eventLoop().execute(() -> { + try { + ch.pipeline().addFirst(TLS_HANDLER, + new SslHandler(SSLUtils.createClientSslEngine(sslContextFactory, host, port))); + initTlsFuture.complete(ch); + } catch (Throwable t) { + initTlsFuture.completeExceptionally(t); + } + }); + return initTlsFuture; + } else { + return CompletableFuture.completedFuture(ch); + } + } } diff --git a/kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/coordinator/transaction/TransactionMarkerChannelManager.java b/kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/coordinator/transaction/TransactionMarkerChannelManager.java index 310a79460b..4f29a3524a 100644 --- a/kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/coordinator/transaction/TransactionMarkerChannelManager.java +++ b/kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/coordinator/transaction/TransactionMarkerChannelManager.java @@ -77,6 +77,8 @@ public class TransactionMarkerChannelManager { private final Bootstrap bootstrap; + private final TransactionMarkerChannelInitializer transactionMarkerChannelInitializer; + private final Map> handlerMap = new ConcurrentHashMap<>(); @@ -182,7 +184,9 @@ public TransactionMarkerChannelManager(String tenant, bootstrap = new Bootstrap(); bootstrap.group(eventLoopGroup); bootstrap.channel(EventLoopUtil.getClientSocketChannelClass(eventLoopGroup)); - bootstrap.handler(new TransactionMarkerChannelInitializer(kafkaConfig, enableTls, this)); + transactionMarkerChannelInitializer = + new TransactionMarkerChannelInitializer(kafkaConfig, enableTls, this); + bootstrap.handler(transactionMarkerChannelInitializer); } public CompletableFuture getChannel(InetSocketAddress socketAddress) { @@ -192,7 +196,10 @@ public CompletableFuture getChannel(InetSocketA ensureDrainQueuedTransactionMarkersActivity(); return handlerMap.computeIfAbsent(socketAddress, address -> { CompletableFuture handlerFuture = new CompletableFuture<>(); - ChannelFutures.toCompletableFuture(bootstrap.connect(socketAddress)) + ChannelFutures.toCompletableFuture(bootstrap.register()) + .thenCompose(ch -> transactionMarkerChannelInitializer + .initTls(ch, socketAddress.getHostString(), socketAddress.getPort())) + .thenCompose(ch -> ChannelFutures.toCompletableFuture(ch.connect(socketAddress))) .thenAccept(channel -> { handlerFuture.complete( (TransactionMarkerChannelHandler) channel.pipeline().get("txnHandler")); diff --git a/kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/utils/ssl/SSLUtils.java b/kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/utils/ssl/SSLUtils.java index 616458391d..c8c2cef6bc 100644 --- a/kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/utils/ssl/SSLUtils.java +++ b/kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/utils/ssl/SSLUtils.java @@ -257,9 +257,10 @@ public static SSLEngine createSslEngine(SslContextFactory.Server sslContextFacto return engine; } - public static SSLEngine createClientSslEngine(SslContextFactory.Client sslContextFactory) throws Exception { + public static SSLEngine createClientSslEngine(SslContextFactory.Client sslContextFactory, + String host, int port) throws Exception { sslContextFactory.start(); - SSLEngine engine = sslContextFactory.newSSLEngine(); + SSLEngine engine = sslContextFactory.newSSLEngine(host, port); engine.setUseClientMode(true); return engine; diff --git a/tests/src/test/java/io/streamnative/pulsar/handlers/kop/security/KafkaSSLChannelTest.java b/tests/src/test/java/io/streamnative/pulsar/handlers/kop/security/KafkaSSLChannelTest.java index 7d7fbd785e..5a1504e17b 100644 --- a/tests/src/test/java/io/streamnative/pulsar/handlers/kop/security/KafkaSSLChannelTest.java +++ b/tests/src/test/java/io/streamnative/pulsar/handlers/kop/security/KafkaSSLChannelTest.java @@ -84,20 +84,22 @@ public KafkaSSLChannelTest(final String entryFormat, boolean withCertHost) { * @param withCertHost the keystore with certHost or not. */ private void setSslConfigurations(boolean withCertHost) { - String path = "./src/test/resources/ssl/certificate" + (withCertHost ? "2" : "") + "/"; - if (!withCertHost) { + String path = "./src/test/resources/ssl/certificate" + (withCertHost ? "" : "2") + "/"; + if (withCertHost) { this.kopSslKeystoreLocation = path + "broker.keystore.jks"; this.kopSslKeystorePassword = "broker"; this.kopSslTruststoreLocation = path + "broker.truststore.jks"; this.kopSslTruststorePassword = "broker"; + this.kopClientTruststoreLocation = path + "broker.truststore.jks"; + this.kopClientTruststorePassword = "broker"; } else { this.kopSslKeystoreLocation = path + "server.keystore.jks"; this.kopSslKeystorePassword = "server"; this.kopSslTruststoreLocation = path + "server.truststore.jks"; this.kopSslTruststorePassword = "server"; + kopClientTruststorePassword = "client"; + kopClientTruststoreLocation = path + "client.truststore.jks"; } - kopClientTruststoreLocation = path + "client.truststore.jks"; - kopClientTruststorePassword = "client"; } @Factory @@ -111,6 +113,10 @@ public static Object[] instances() { } protected void sslSetUpForBroker() throws Exception { + + // require TLS verification when hostname is on certificate + conf.setTlsHostnameVerificationEnabled(withCertHost); + conf.setKafkaTransactionCoordinatorEnabled(true); conf.setKopTlsEnabledWithBroker(true); conf.setKopSslKeystoreType("JKS"); @@ -153,7 +159,7 @@ public void testKafkaProduceSSL() throws Exception { String messageStrPrefix = "Message_Kop_KafkaProduceKafkaConsume_" + partitionNumber + "_"; @Cleanup - SslProducer kProducer = new SslProducer(topicName, getKafkaBrokerPortTls(), + SslProducer kProducer = new SslProducer(topicName, getKafkaBrokerPortTls(), withCertHost, kopClientTruststoreLocation, kopClientTruststorePassword); for (int i = 0; i < totalMsgs; i++) { @@ -188,7 +194,8 @@ public static class SslProducer implements Closeable { private final KafkaProducer producer; private final String topic; - public SslProducer(String topic, int port, String truststoreLocation, String truststorePassword) { + public SslProducer(String topic, int port, boolean withCertHost, String truststoreLocation, + String truststorePassword) { Properties props = new Properties(); props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost" + ":" + port); props.put(ProducerConfig.CLIENT_ID_CONFIG, "DemoKafkaOnPulsarProducerSSL"); @@ -201,7 +208,7 @@ public SslProducer(String topic, int port, String truststoreLocation, String tru props.put("ssl.truststore.password", truststorePassword); // default is https, here need to set empty. - props.put("ssl.endpoint.identification.algorithm", ""); + props.put("ssl.endpoint.identification.algorithm", withCertHost ? "HTTPS" : ""); producer = new KafkaProducer<>(props); this.topic = topic; @@ -233,7 +240,7 @@ public void basicProduceAndConsumeWithTxTest() throws Exception { producerProps.put("ssl.truststore.password", kopClientTruststorePassword); // default is https, here need to set empty. - producerProps.put("ssl.endpoint.identification.algorithm", ""); + producerProps.put("ssl.endpoint.identification.algorithm", withCertHost ? "HTTPS" : ""); @Cleanup KafkaProducer producer = new KafkaProducer<>(producerProps); @@ -292,7 +299,7 @@ private void consumeTxData(String kafkaServer, String topicName, String isolatio consumerProps.put("ssl.truststore.password", kopClientTruststorePassword); // default is https, here need to set empty. - consumerProps.put("ssl.endpoint.identification.algorithm", ""); + consumerProps.put("ssl.endpoint.identification.algorithm", withCertHost ? "HTTPS" : ""); @Cleanup KafkaConsumer consumer = new KafkaConsumer<>(consumerProps); diff --git a/tests/src/test/java/io/streamnative/pulsar/handlers/kop/security/KafkaSSLChannelWithClientAuthTest.java b/tests/src/test/java/io/streamnative/pulsar/handlers/kop/security/KafkaSSLChannelWithClientAuthTest.java index 57fb22f225..fe7da51497 100644 --- a/tests/src/test/java/io/streamnative/pulsar/handlers/kop/security/KafkaSSLChannelWithClientAuthTest.java +++ b/tests/src/test/java/io/streamnative/pulsar/handlers/kop/security/KafkaSSLChannelWithClientAuthTest.java @@ -47,14 +47,11 @@ public class KafkaSSLChannelWithClientAuthTest extends KopProtocolHandlerTestBas static { final HostnameVerifier defaultHostnameVerifier = javax.net.ssl.HttpsURLConnection.getDefaultHostnameVerifier(); - final HostnameVerifier localhostAcceptedHostnameVerifier = new HostnameVerifier() { - - public boolean verify(String hostname, javax.net.ssl.SSLSession sslSession) { - if (hostname.equals("localhost")) { - return true; - } - return defaultHostnameVerifier.verify(hostname, sslSession); + final HostnameVerifier localhostAcceptedHostnameVerifier = (hostname, sslSession) -> { + if (hostname.equals("localhost")) { + return true; } + return defaultHostnameVerifier.verify(hostname, sslSession); }; javax.net.ssl.HttpsURLConnection.setDefaultHostnameVerifier(localhostAcceptedHostnameVerifier); } @@ -71,13 +68,13 @@ public static Object[] instances() { }; } - protected void sslSetUpForBroker() throws Exception { - ((KafkaServiceConfiguration) conf).setKopSslClientAuth("required"); - ((KafkaServiceConfiguration) conf).setKopSslKeystoreType("JKS"); - ((KafkaServiceConfiguration) conf).setKopSslKeystoreLocation(kopSslKeystoreLocation); - ((KafkaServiceConfiguration) conf).setKopSslKeystorePassword(kopSslKeystorePassword); - ((KafkaServiceConfiguration) conf).setKopSslTruststoreLocation(kopSslTruststoreLocation); - ((KafkaServiceConfiguration) conf).setKopSslTruststorePassword(kopSslTruststorePassword); + protected void sslSetUpForBroker() { + conf.setKopSslClientAuth("required"); + conf.setKopSslKeystoreType("JKS"); + conf.setKopSslKeystoreLocation(kopSslKeystoreLocation); + conf.setKopSslKeystorePassword(kopSslKeystorePassword); + conf.setKopSslTruststoreLocation(kopSslTruststoreLocation); + conf.setKopSslTruststorePassword(kopSslTruststorePassword); } @BeforeMethod @@ -161,9 +158,6 @@ public SslProducer(String topic, int port) { props.put("ssl.keystore.location", "./src/test/resources/ssl/certificate/client.keystore.jks"); props.put("ssl.keystore.password", "client"); - // default is https, here need to set empty. - props.put("ssl.endpoint.identification.algorithm", ""); - producer = new KafkaProducer<>(props); this.topic = topic; } diff --git a/tests/src/test/resources/ssl/certificate/broker.keystore.jks b/tests/src/test/resources/ssl/certificate/broker.keystore.jks index 063b4f8b6d9355d60aad63d98c052aa8517f6906..ce1bc84a65d37e5c781fc665be06a564029d3108 100644 GIT binary patch delta 2074 zcmV+#2<7*eAI}kg{_Xzl00002000010000100v@mZ);_80004o642J%000F7FoFdB zFb)O^D+U1s0V)C!0RaU71cC(W4P#T0Lpq;3wuj)$;@qyamt2_t;;u|Mw)Jc|FD$p8 z!-lvIK-@Za_d#ZhKkf=ZepKY@f-YBQW57!ppXxn>1t#-<#+miU4QRPq>9H9wU4TGN zA9vp}&S7B&LKY)umSCmAEoEMgH?DCPtGxZ1Tk{s=QZ$idt=rI_Nu`Ex`}#nK_AkUS zjq1r??K~U1zI29N3hXGWTI#gN#$jx+VxX)?iq+T4@vgSsGh;xUGd{ZV!xa&BQcBc= zd<5y3U4fNvtla7Iv)$*K^`5QieoHW zql>ivYy?cn0@%M=O0Jac^-l}lQPE1#K>a_8ndn3ACJ#AiGj3c3Akea z`z7E<7&58fi4z$I+2b%7u28mWK1?K>l%qqORH#*n`{v#lri|3Q2c}XfUIZ#~=RmW< zn~B563ZB$cVu6fF{sLW?1J4m!og2xpe0LMtyt9$CJi4dzKndQ*@;zWFO*B8WQJ@av z9&^5b-m-$wcCjT}*d^e>ug9in53x|WOVDGoEE5%(L zSL7L0oWzsbx;rQ5#3YfTnD9C|zAG1JFeuKUaWdYE?Gt~&a+kneY5$|C*_@L1SGgnw zb<_h?vvOFuu_=UXl8wmoc20@WQG&?Q+!~30!P;F1m$#iQZ4#PG=k>GIBkVf8m7!9c z-vv!@AL6yPM?&F?1{xs8tzRWQDnsXE@6f~LjZTJqi_^xAiHV=8^E^7AukL$3N}}YV=k42B{%*3U}c(}ceYex=JXvIhCYy2lmpR(J0YnKPZr4Y zBsIgo?$5~K)e1sLQkTrb!P6Fm09=$^>#kqm_dus82@B9Muv)RmTh5U~bv#eNwsCgQ zsLq(O4G<}tLy27UjJ^}N4-!d8B_m&d2KLA!wK$h4XBme$%Di{OJE}A8?8I?xMqx>L zpknzEjiXtrIg2ZD@?;{6!z+g>)J_%#XQV@#^T5JI{C4L# z=pQ~OnM&BvrE}Y@z5WsgoY2Y_y+}UH1xMBB#p;yIDJGvO`(s`!VK_dKZIc{-h)zuh zetKYs;-q(XUt+0H+T(+5%lWu z1aLM&i+;8JJ3Kj&4&*xMu8$w$CW|dB*Q*`7b=h$Hy%;a$5o2r5n!L^v4qV#cmnbJa zPxm%Y)A3b;OPEJek@ZLnGELckNXdW6UL4vj3|2xOQ(ZZB)ICyw-`2XT?UV+CLH{s5 zRFHL!HQccH5;KYp3+e;*ACH|Ha8p$u-@rf*?o3WZO@jDw`D!B{D)iOItq|Oei>>Nr z;v8wfT^%AhRrprivhMrDIytZSVu1IEFT?a*@v7p%hT<6YZe{EBi14&%o}f@=ewIK5 zJZ$(lUsfi(EzKt`$G?gypjUdS!)-cgUeA}jNPvP(H}}Ef6V|yu{@7d)r4F(A+hDe6@4FLfQ1pqK^lU@m1Hxe)q z1_MXm4|LFd!EVGBYqTFfuVUG&nRdS{M&9F)}kSF)%qXH8eOhFXm4|LlMMkOf6%4|QA(=+iWe&s)+mCZ?M8vQF><_Z zC!T=(+J9Daoh~xxz(U~qXgH_N(Vt#mk&OiG2U-y)8v8vX&FmsZJ|dC)%4v&}1$+)9 za;w4CWhcmGx!O%Z8w>Xf4NEj;hIYSy?Qx09i3!w8`-9mx_g$E8JX&gPL_YY#5hZit zeQ44F(A+hDe6@4FLfQ1potr0RaGov(Q}L9b&xjxkkCy_guJg ztPcvo{zZ6odk){+2iily$Kt|Q{JZ^u@jt8J#fVibds8kRS~iQxKFjbrtcQ2R{UU< zWK*Rh6l!F}m~M9T$}}i)w1L53RIb7M*8{g(kXSqH{+R-V%EGW3C$cjN`*U41<{D*i z%qTl%ukk5v#K^Xrj@moP*3YeFyrv{Ld#L7X9$0OtePE-VtH}YV2NMsQO42Zncj2mY E(NMq67ytkO literal 3991 zcmeH}X*d*GAHZj`Gz`Ybl08{sjIoR+SyIWGeMv%=v4pV=62jP*A=*SEV=J_fts*YDNe;kt7KK3{1pvVQXd>7LO$1rxfk8kJh(}qXkL|Dy*SaV%r)|gC zhED!m?0|+b?~{$!i(iKUfox0ww2mShdLJ{`3c`#4XuNeQj0h(Tk~SVinPeoiID zJ%HJ3{{(;4MnvGjKOO$P5d6@?SljiY@^q$uqDVF73J`&y_edhH@282As6V+w9a+xOJ z$zbDWc2p2=OLGfG_LXP1ITwn!WD`tstMa1bB39uCBg8{mIiHAis+c}-`_#H=Ho@NGHAPw~zv+@9#>;E_ zZT_OKHtr5pbwu)F>!^$Ms?I0)9KG^ZyDta~1ON*x&}L}iy*m@;fbc?ih|_e%!@UWW z^%HX@Ut|#(fsz$6Kdw|*0>Y2xjr#8sIDcJ_00MU*Ola_4_s6jBWzJsMfq<(9^J6p1 z7{4YP7`-TWm?PXeq&n!iOh{a(G= z*SSs;epjI>dpKV20;`(P_%h!pl!YTf&_lP*+fmc#O2*)4o$^z_CV0Fgt_}|JoL#js z%nZIURHeY6Pc%AUliHD(CE=FGqIr4aHt0hgtED#>gRS=pQyWf>3I<}wMY_C;cET$y zg)hE3GM-fZ5c_tDTM(vs`P!y3nKE%fDJ;1)aGgVeD_yM2)Os;)wmDoT?VhvZjONn; z&>|?>bB=Fg@tVHz>q%0bQ_C6BcL|7Qr_9<;p7a-+9__47X0l~sgpcB0O#%V?ynS7q zyghsae=Es6egI&vvWSo^GzT+Oju{96%7Wm)y;Ow|As^E&z+;8pxI9mcx zyB*Px6Hsb8cz;GMG~;M~iG2w>?7M|VElL}00ZObF9>337bViEC3?z7P&6N53 z%(-jZ_TIsDwOkU?Z+Wz7F0Q5;YYa_$fU7q8>inhLzxs8{^``f&(+jC%cAaucwR2Ag zQctB?P9C)BY+905IZE-T(-XHFN7kB`Eqd8!lrt0f#3(&ZcFu1&OZhA&xpxzH>kXvz z8vKrBA*Wu~uMwiYc8*HmvB#(v-Yy$FkBj1j3mT{fylpO-5s;MQ2)HfV=5!@E*)4S{ zb6~4*Q_5=LgZ9bmaSJ||pA6xT72nA_`fcjr=KgND(Na zdj^cq;yJbhs*CQMT{T}8<%;qk1rtaqr(R-;1kE9|)=5I>i%MCt9`@j{9qgI?hXsVs z=c&B8_wD%{IZpck>}3y6ed;}R{$zC74Wt=GvHHC0G}HORCKZWs>x+6?W$W{1Rx1(v zj1s6;+Cmo2n+(nmpX|h59(-pd{|U9Twy_bO91$k0)fkHAKJI>Ef=t@Sn(poK%BlP8BaE?)WYKZ6 z_p^VS8|&FYw+j~>alBMb%DE)nF_h(4AzFY}NPi=J(nF?TSmK&Oq5LD>Ovge=@7lus zzUjH!Yg~SIMQpgVF=9iTzp>`*)&$j~lEZ4dW6NZ+bQs7b3lNXCWzQklavq;JfNip; z>Yf;Slm5bR%(Q?+L6+pn1tc76Oi{j}r!{K4t1jO4Dct@Jva6}^>6P#5sF->+T6*$! z{M5|obQ4%tV}O-{tJNu4$4r-_D@@c~jbr21orUI9pd*Vi!JYRM36qK1}MC(oECT;C*aOTUt~3gLF*PfC#q;)&oHFqMEzGT_n-UtnLWMeT zX+{WDFOX%sj^){Q~dH-M=H>sm-Ta8->HuxIgO-Sh56<{m_-) z)kJ$wO*Hn@L=wa!c?)*OAam9lh7+)jPYgh|EW399^tb#~vhW{PqH^>nE7=2RUq!CH zNBW2Uuvp=}@V+7Ve-T#1&xGYq1o|h!_HXi=OMwwk7jI7-o)8p>3)u6N|Lrf`JSSr@ zcT6gTY0adyty~p>P#yU~yZRlkCz#E7!2y&Tr6>S0P>|d-7=jC0+#JHlk8W7QiENtm zf?bsQ1xc6una$_BD3($rtH2e9Qhm4c)_`{-qg1&RdN(CbfB1csuSzzFnt!~10PAxtTBaAwyN;g007HhoiftoxJ`C4nXy|j~6M)CQQs6D)| zF<K3RA}W;CuW)Np-}kIunWWZo zB9&F|VBp`@!`0y_o)84yof{QJZ!!JIMA=$HdLcsG&5$Z%?@~*he)!8z>F`4}LgRm- zA1u-JuWmn5wXo=Lkd}E-gE<)Duj0g)8d#&UkblEc`ipzHQy8K>s^k3}^Tfa*iH0BN?VV=K6cERwxvEltbxiduuf2DU$O)N zjlR!uB=RaD^|8de<>9l9gA=3!Xv(-utB6X9?xxD}xs+$%fa006A2b&ll{FwRn#BF$ zd!r6FH=ZGy{aH-<#Qc*5AMj4k!F>cmPo*n4Bg!wj%Yp1D74bL88LcTib{QcKXVvPO zYm?(JYa6k~o#UMvdItuAV0U$^>o(_SOy$|0KFBtb&yve9+d9GLJuA9|TK-$}|JMA! dHUIyk`HWGDVNl;{#UyGM?iJ(O7%k5;{0}Fcu7&^r diff --git a/tests/src/test/resources/ssl/certificate/broker.truststore.jks b/tests/src/test/resources/ssl/certificate/broker.truststore.jks index dc062946bbb258897fdc177f39c9b4855957d8c9..851da10b1f741cf767f7a34212e439b998a106ab 100644 GIT binary patch literal 969 zcmezO_TO6u1_mY|W(3n*B}JvhCB-HAMX5lcHlYjm92r<6^h^yb85o$`44Rmm44Rm- z7BDk0GBI&7{7SCYocI3bQv+T$POUbNw(q=*jNGgY26=`827GMHp)72|?4fztdHLmV z5e^Iyb_@|N3=wXGh>(E*$Yf?=&Yb+@#GH)$;t~S|ab6>110w??LsJtA6T>J8ej`Jm zfT4jURDkX-YGPDE4irXK2IeM4eg=akMlPl%Mn;AU%h&?FR{if1*A}^^(X{ZbTjNf{ zqCL6l^BVr#st+rkt7r85fYXCt85YaWU7R16FtL~A4SSTJy3}u5<+E>;-0YMl{yCM| zJ(;zJPo-$}!E34NCsKFb@^zHv{?5(oZIaejw!i*uVdtq%&MV%(n{S$bkDZZk6P1vqyCqdWy#8D-DJ^70^w(K$ ze{i3Zus&zE?~-Wj!*bEs;~}+o!wqE~Zk1iT&pUZqx9Tp2%PBMWGBGnUFfLX!kT;M8 z#-=PEix`WDAj`p9Ld7f+`u_cgMK`R7eBrXHA2|?#DGL~gj0~-tFT~uFP1^Her`yi! z-(z+Zt>Nc7_|LVXw3_e!9rjz!2ai8G)aEwdea=qD#Z%9GO%8cho>39+F?nl9@ao%^ zyYB3$dfq!(dd0FWO|0+!#=MI^&3GrlcjeCImH(H8-0CT~bz`sMMux7mwxhC7bEhqv zboQ@HeWlxT|B1W4X1RaA(aBlMTKm)XdBt=mkFE#7_m<0V6rK}kTX2-GWYVmjP{B=% zJN&oqS@vFRRoI_|sVTurl|`~rj?Ty{`*O-eqi9Rx!Gv`Oe_vwp>K+irR0j XTNy9k$hf+M+n{69qaTXw91rXPy`Xo? literal 944 zcmezO_TO6u1_mY|W(3o0$%#ez`6WPZV_JsPE(X>JJyQcq1_tIrgC^!&gC?fX1T?M|&H_blsFDd-p+)vBm40zc%wc0$|zVk9NaJ8a*O6r=L$AxP5o9FMY z%yn87&vI&p;iN>X$IIgm7d}1qbBdR6Q*cntqO`MNuTD0k%DT6mPwsCHDyy|mZe7{8 zyiz*nx#QJkT4Ha^Yxb%8-n=@IW$tB>hEoA=tL{i-+ksP*~-R zrY(7wY8A^5^KVi(VG^(=?Ze@pfwy_iWZ&z3yWrAkaV-7-zD4EMhDoO%L7~YMyQ1yyx~)|Ib>&6N^dAH%&Tl})|8 zMgRS~+xz1zI*;m@zLq_-UVg!X8{q~j$3Cu_`u@`4G_PINu0 Ttp2!6V!iD!i&&WiEwik diff --git a/tests/src/test/resources/ssl/certificate/client.keystore.jks b/tests/src/test/resources/ssl/certificate/client.keystore.jks index 9d61ecef52eccf2c3ffb842277f9cb957ca3e724..7e5a8561736ccac616441843a56895df074e7d88 100644 GIT binary patch delta 2008 zcmV;}2PgQMAI}kg{_Xzl00002000010000100v`hX=QG70004o642VC000F6FoFdA zFb)O^D+U1s0V)C!0RaU71cC(V8f1)h$BO{QPlNdDSy%s9TPte=1a$70cr(0&o_Z91 zQ`+B?_g5wD-z&;hsIBPJ(!pq-VMI%&wm0KA1wE?WJd@CW&QXp&nuza-bn(x0b&#HN4SUIYi&1-3U2b!V;0`e z7{#1uuL}DXb}98@yx{&8!>Gdr{ERtHs1cCaIps_#ip1n_J$mJOCd}=~CF%~I-*CH3 zvq}7SDV|k-i$E&PKOLk}`sEfxyTDz{Juy~lX&K7bU8k|=^%dbMuFs|@E~bwOg~p)Q z$+Bt`!6`1)OyApFzd1ZjR{WT0N=@!3ola7Lfzt``EQXMR^S2nreRdo&%vPxS^34*Q zI3VT%#oD8saKkm3{|1tCi~HjHM1omFzB1rw(zNG)R&MrJb0|+se8HI9q}M3t-5XS4 zzuOnpSs-pkusfs7|QglwkzcTgn3ljLS&RTcRr?YkbQU8iR}wQ7k=Mhh8oPf2--G# z;6)W}U$9BB1v?5yaT>;-p^$4V;yX-nD(VM6$?s8hd!io3+U-IUwS>7%awZ&2v}gc- zhI?ihL91^SCcOMwmuhKx8ri68%+67_$i1JJvZ^Vjbp`cmOEDeOGJr?68K3li1;ik#=4Vz<%*j8P~QNxDagwm55NhhO@q@bV(UTRlDQ6ArXmS+a;07v*pKRP(j=K+}Uep z>+*HfBHeZs@iib^t8(x4zZ~l2fJL()fe#ENRb~fq_s37URvMiE000311z0XMFgXAK z1BWnz1A{Pv0&AcH0s#U72nL13AIWO=XD|&02`Yw2hW8Bt0Sg5HlU@mOARreFGBYqT zFfuVUG&nReS{M&9F)}kSF)%qXH8eOhGLu~ilYhgtDR5jViGv_>Y1uK0M%8P+cDD@| zgq%)@qC26KY@s zfZ!dhq>tvU?U;8Fult=UdKXEx!y5AEa>c(THskAFRKA@lTMy8$w@h`stW=snck=!p zA0WQSK$JE1X5`V!98R>XAHR0v`vyO-0xJ5w0s{d60iz)>A21yT163Uk1QrAo4LEgj zfeL`4j&_WxQZ$8c!!4DHlW`6de^eBm%PxL|6v`^TC7rGU6znT-S^sZmQ6b^nz!luW znU13$Xo@u-E|KBgI7oKy;AVngt1Y8$AOkBrv-{SF@o$g-z4j`@=a<1ayCLzpL&%@@ z^Az=Ca62sz76R+|C#k}!w5#JpVP3ix-U*JeTh#xZ)MGJf0P3cUcMyeY7~Y#jchCkz^i q_y}SaLN5NNR9rDD-x}g87UQ@;%W}D_3UG3yw?d%IZwYe}qFB0APQ)Sr literal 3993 zcmeH}X*3(y9>9}Ggh(3uT1$%|C`rcN+G}fTX_3TIN=oe+1f%w)mbR#^R8R&{r9~0d zSc@vEwo*o`N;_&zV;NiN)4qB0`expFpXQwRWj@?<&;6Zy&pr45|KEFm`^)>w0002= zdt)U!`;$my000>6=KhKw03j%mBj|k$^+7~!SJBDz6{vlFe{Re>oF^` zGuah$1TFpr!r+3YzGPQ_UstjO-s9pW4|0HnB{@Xm4EZ8jLI8oqqE#^}XmzwYM%_*j ziNzcMm4o(A@K_w0{7-%(w#MD%N#rW-apySJjD}Z zbuR2Dw|?A0Pn4^uar^)=HTGOkxDN43c}iJ`i1(b{C{>PsNiD2J#%y9V4V*v18`W#M zeEF4-2ZKY9)k*0{XNp^p!t?6fPL~ofVlLUDpzf#Wp(%LqnV01+SrZ|rG*aROZ(8bd z73D+3^eX0L?snQ0bmyBbc5&lEa$@Ryhr^EqcOV}~JXN-=boaEA&~k=t0gz4}tl?yfM=& z7*u~kZPVm?i#0i(v;gTE*{>-Tl>`f-`J?{(1mu_XJV4-QFa*tV(0v>BgUUGwI}mW= z)ZELNB^AcJ5yL?@(G~GvkT&&l- zbU&=tTGX>gZ9J*|p8A_9ZV`m`<&5p`^NS`gU_-BV1Z*Bs;!KsOu*5A~nQae~O(}E1 z%xFIx0xf`|J>Cm!Eo7LQPrOd1J3lr{-jxJvKb2kIFO*rsKhw*3%9?MT8192PxRXEt zyElpG?Cnkp`0YwAY61Wd5IP3Dk3Pf-lV=5jfpQ=&;6bT^W55f3wkj$hbIZ3rw#28G zHBkI4dp)|~J%tkY66@t0QeX!U$znRtYq5L*p1s3_h^(a=r7>y76} z%FRIQhBh@&C=^y=9B5E=*36bT0QUtGPI1`bjm1#S&BjYB2Ac->pC2>s@6&`UpNRJe z3~h@-KM-$MZc)GtEw#e@8F1b;q4%PSH3~`ahi3( z8iwX{!`j?g_CcD}b?^QX_M(dvgULO8{q=xl zYRg=PX|#D{^Z9$e?>?%0?8NF!SSfXOCH)c8q>|kKBKmx{9d;b233*T& zRqn^5ddy~MmnNfdG@0pLv6*gv@g)tGK?m!&R<0K`!`e<9ujZK&ga+$e8&D-jl-NFsUbP+RF6Siu!BDiM_(rkho8Mhc=1EW-1RDZ(#FGf z76X3X=KgH+62-nBb{7O6y4ZbkLQ{mU*9j_YVUdxWgGOU2bIvFPs&Iff(h#ugRa4)g zNv&)#2;obS&F>7!se2zOC6{@!Rebtz>hY^6NF91=mM1r`PdP%H**ld!Ae%JAD*dr` z7%JT$Y)w33NkUW+~dUK%?GsczJh z-~Xzk7iI)->yBfSEjEe>ezl_(g`>uSUUF#d+}%|klk6Q?C8|1@fxIC8Xp2Q)(&Uh2 z&$33#Cp9qtlK+F(KB3BkW8evVDfDr@`rIoWDA)5lkifN5tGCZAw0qC2eAbNSUVorR z=Y(^1L4gBEIUcvGfp-QfLynV{!}wH@0Ome`y(VC^Zi@{R_+G=&d~VZWt$Us-Y<8^2 zZ3Crg8b-w}o55v(J6vlQuMh?*9-Q?(G%=&xAVW@e(DirOkR7clmA`O-bjBd_ToZ)R z-<38C1idw4Ymw-0>ue}4tw10Ti-CB#Dp1<@tQB_PkYq?Kz zRN6>xyCyM@&RFe7!T`wK!_adT3C_+9=gUHH~ZG%%R|^pYzp@vxh(@itW;C8A)HHiy%tgo7BHQhax`u5dN< zf0lD@Hqd2#EIcebcD1S6CQXY>UaVf1S+DKpLXuw%<`3oYRW^LD zS4?G$cX-uh#bdPgZe^AUJ%rl(RVZ%szU$1a4^Y}6t}kULOh4DudlIpKXEZTEMmgQS z&WgMfNO4!0sENf>tJ2^Q(dmR7#Zca9L2s3l-l#1?L{~23xngp?*d_ne+;YW_cs8n7 zwjA&_u53$F-f@STOBnDN+C^$TN>0=Y-mX5OJ7%~L#sGv@k_;ydGxRr^Mk1mTk&$Zd(Iljc^34-q z!jkt*jP{XOHYzqwtCv$S?uuBvP~QjtpxC#$ce76CEyE)U0~5Ap0v`eP_5K$9zeWFV d(f|L5J|X?&^z5)My|AP;sX#TCyxJ@({|^fwmW}`b diff --git a/tests/src/test/resources/ssl/certificate/client.truststore.jks b/tests/src/test/resources/ssl/certificate/client.truststore.jks index e8b93df69647d8bea7c1f133e523c9a45d13b0bf..5c5b6374fcf0dfe4ff9655e7e72a8ea0b7ce9661 100644 GIT binary patch literal 970 zcmezO_TO6u1_mY|W(3n*B}JvhCB-HAMX5lcHlYi5GZ)}KHLs+!IJL+?L7dmf*ucoZ$k5cp!o(;_g5Ssx zC}3z{2^FBflbRTnkb{Mhm4Ug5k)Oe!iIIz`iII`v@K()&7|qURh2qSchTU#gv-g#4 z=M`_6qQ#dkl;?BcOKKPg(+ z?Tf_a9~Ex}&U|2u(G%Bg{LOLj&5?~MCJM{qH(7@)R^__T`ow7FA_eWuF(($98Hq^q z&bz>q{&TV`=W@wUI~5N2D5WipjOMCuT=*>abI13X>}xj!oDQ`dRaebi=JX`0Cbp&7 z)po+qNxRvm?lWw7AiHKs|FgAkXOs)B|2$m%q?pYHu%c1sc9O(wcu*cWM`MAOh1CFc29TLPX}C)~jm~IiLMdCK*p zeDU=(Cx&B&Rf~=4Za?JO%dmKl=7zmFGXEB<^GJ36;7Ag4()+hOB*swtzSJY_$2)}7 XHtpx=?avU~9&TMM7u&zE=~e~+Gbn7g literal 944 zcmezO_TO6u1_mY|W(3o0$%#ez`6WPZV_Jr@I|FNko~eN)0|RrRK@)SXK@(Hx0%j&g zCMFSM<#XJct^(iXn`R&RmlXbP?x$sO2E1&XT5TR}-+37sxmg(uTn)JmIN6v(S=fY` z977F-4Fo|P4j!)H{M=OC;F83WR6|7rd5|C%kA!DlNorAEYKcO4W=eWyNwGpeNu`2M zNs57jIIoeBftjJHfrWvEp+%GgzmXwOz!Zof0@SjoiBSpJ3yiD`%uS5^3A(C-*l8mDSoOx3274 zUMZdP-0|u%EwMM|HT%?kZ(g0qGWW7b!>NF`Rd*ya-*Zc_#p^2`=}RlqIk#cfk)$PZ z$+JrQ=WSWkc<~-jxzF{L*V#H)y)64$B&G#82ritTR}`L^4IgMe8{eE4a=yAopv+hlKBd}Ne3J1Tyw6E13%FIT+&!^7~_`UVxxf9>-$UfX? zr={8Ek-%XkbK{NpH7<6(cIgcF-8u1gK6Mi>eRAF4!gxTiO*M75AXDb!kKx{v%BEi4 zqW}Kg?fr2Uokw*{U&|g^FTY^Hjc@~%V;@&deShij@+Cd{oUch2TilR4m9z9;&E^o< ztdr(9I=8O1xPMPbn%Ays@=xQr3vVYFRd*dO{?4Z_(66vMAnaA^nmiT*EZ88D*0S Tt%VOgyBHa6X>!3XH0dD#kRo*W From 9dbe38926a35b9affa83bdffb828e7ea71fb9e84 Mon Sep 17 00:00:00 2001 From: Demogorgon314 Date: Fri, 28 Jul 2023 16:40:36 +0800 Subject: [PATCH 2/2] Fix code style --- .../handlers/kop/security/KafkaSSLChannelWithClientAuthTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/src/test/java/io/streamnative/pulsar/handlers/kop/security/KafkaSSLChannelWithClientAuthTest.java b/tests/src/test/java/io/streamnative/pulsar/handlers/kop/security/KafkaSSLChannelWithClientAuthTest.java index fe7da51497..01dd5708a4 100644 --- a/tests/src/test/java/io/streamnative/pulsar/handlers/kop/security/KafkaSSLChannelWithClientAuthTest.java +++ b/tests/src/test/java/io/streamnative/pulsar/handlers/kop/security/KafkaSSLChannelWithClientAuthTest.java @@ -15,7 +15,6 @@ import static java.nio.charset.StandardCharsets.UTF_8; -import io.streamnative.pulsar.handlers.kop.KafkaServiceConfiguration; import io.streamnative.pulsar.handlers.kop.KopProtocolHandlerTestBase; import java.io.Closeable; import java.util.Properties;