Skip to content

Commit 9fda5cc

Browse files
nodecenicoloboschi
authored andcommittedFeb 28, 2023
[fix][proxy] Fix refresh client auth (apache#17831)
* [fix][proxy] Fix refresh client auth Signed-off-by: Zixuan Liu <nodeces@gmail.com> * Fix style Signed-off-by: Zixuan Liu <nodeces@gmail.com> (cherry picked from commit c952f3c) Signed-off-by: Zixuan Liu <nodeces@gmail.com> (cherry picked from commit 388938e)
1 parent 8de0609 commit 9fda5cc

File tree

6 files changed

+337
-27
lines changed

6 files changed

+337
-27
lines changed
 

‎pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java

+13-2
Original file line numberDiff line numberDiff line change
@@ -149,7 +149,7 @@ public class ClientCnx extends PulsarHandler {
149149

150150
private final int maxNumberOfRejectedRequestPerConnection;
151151
private final int rejectedRequestResetTimeSec = 60;
152-
private final int protocolVersion;
152+
protected final int protocolVersion;
153153
private final long operationTimeoutMs;
154154

155155
protected String proxyToTargetBrokerAddress = null;
@@ -165,7 +165,10 @@ public class ClientCnx extends PulsarHandler {
165165
protected AuthenticationDataProvider authenticationDataProvider;
166166
private TransactionBufferHandler transactionBufferHandler;
167167

168-
enum State {
168+
@Getter
169+
private long lastDisconnectedTimestamp;
170+
171+
protected enum State {
169172
None, SentConnectFrame, Ready, Failed, Connecting
170173
}
171174

@@ -269,6 +272,7 @@ protected ByteBuf newConnectCommand() throws Exception {
269272
@Override
270273
public void channelInactive(ChannelHandlerContext ctx) throws Exception {
271274
super.channelInactive(ctx);
275+
lastDisconnectedTimestamp = System.currentTimeMillis();
272276
log.info("{} Disconnected", ctx.channel());
273277
if (!connectionFuture.isDone()) {
274278
connectionFuture.completeExceptionally(new PulsarClientException("Connection already closed"));
@@ -1170,6 +1174,13 @@ public void close() {
11701174
}
11711175
}
11721176

1177+
protected void closeWithException(Throwable e) {
1178+
if (ctx != null) {
1179+
connectionFuture.completeExceptionally(e);
1180+
ctx.close();
1181+
}
1182+
}
1183+
11731184
private void checkRequestTimeout() {
11741185
while (!requestTimeoutQueue.isEmpty()) {
11751186
RequestTime request = requestTimeoutQueue.peek();

‎pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java

+8
Original file line numberDiff line numberDiff line change
@@ -33,15 +33,18 @@
3333
import java.net.InetSocketAddress;
3434
import java.net.URI;
3535
import java.net.URISyntaxException;
36+
import java.util.Collections;
3637
import java.util.Iterator;
3738
import java.util.List;
3839
import java.util.Map;
3940
import java.util.Optional;
4041
import java.util.Random;
42+
import java.util.Set;
4143
import java.util.concurrent.CompletableFuture;
4244
import java.util.concurrent.ConcurrentHashMap;
4345
import java.util.concurrent.ConcurrentMap;
4446
import java.util.function.Supplier;
47+
import java.util.stream.Collectors;
4548
import org.apache.commons.lang3.StringUtils;
4649
import org.apache.pulsar.client.api.PulsarClientException;
4750
import org.apache.pulsar.client.api.PulsarClientException.InvalidServiceURL;
@@ -365,6 +368,11 @@ int getPoolSize() {
365368
return pool.values().stream().mapToInt(Map::size).sum();
366369
}
367370

371+
public Set<CompletableFuture<ClientCnx>> getConnections() {
372+
return Collections.unmodifiableSet(
373+
pool.values().stream().flatMap(n -> n.values().stream()).collect(Collectors.toSet()));
374+
}
375+
368376
private static final Logger log = LoggerFactory.getLogger(ConnectionPool.class);
369377

370378
}

‎pulsar-proxy/pom.xml

+5
Original file line numberDiff line numberDiff line change
@@ -185,6 +185,11 @@
185185
<artifactId>ipaddress</artifactId>
186186
<version>${seancfoley.ipaddress.version}</version>
187187
</dependency>
188+
<dependency>
189+
<groupId>org.awaitility</groupId>
190+
<artifactId>awaitility</artifactId>
191+
<scope>test</scope>
192+
</dependency>
188193
</dependencies>
189194
<build>
190195
<plugins>

‎pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyClientCnx.java

+63-14
Original file line numberDiff line numberDiff line change
@@ -18,30 +18,35 @@
1818
*/
1919
package org.apache.pulsar.proxy.server;
2020

21+
import static com.google.common.base.Preconditions.checkArgument;
2122
import io.netty.buffer.ByteBuf;
2223
import io.netty.channel.EventLoopGroup;
24+
import java.util.Arrays;
25+
import lombok.extern.slf4j.Slf4j;
2326
import org.apache.pulsar.PulsarVersion;
2427
import org.apache.pulsar.client.impl.ClientCnx;
2528
import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
2629
import org.apache.pulsar.common.api.AuthData;
30+
import org.apache.pulsar.common.api.proto.CommandAuthChallenge;
2731
import org.apache.pulsar.common.protocol.Commands;
28-
import org.slf4j.Logger;
29-
import org.slf4j.LoggerFactory;
3032

33+
@Slf4j
3134
public class ProxyClientCnx extends ClientCnx {
32-
33-
String clientAuthRole;
34-
AuthData clientAuthData;
35-
String clientAuthMethod;
36-
int protocolVersion;
35+
private final boolean forwardClientAuthData;
36+
private final String clientAuthMethod;
37+
private final String clientAuthRole;
38+
private final AuthData clientAuthData;
39+
private final ProxyConnection proxyConnection;
3740

3841
public ProxyClientCnx(ClientConfigurationData conf, EventLoopGroup eventLoopGroup, String clientAuthRole,
39-
AuthData clientAuthData, String clientAuthMethod, int protocolVersion) {
40-
super(conf, eventLoopGroup);
42+
AuthData clientAuthData, String clientAuthMethod, int protocolVersion,
43+
boolean forwardClientAuthData, ProxyConnection proxyConnection) {
44+
super(conf, eventLoopGroup, protocolVersion);
4145
this.clientAuthRole = clientAuthRole;
4246
this.clientAuthData = clientAuthData;
4347
this.clientAuthMethod = clientAuthMethod;
44-
this.protocolVersion = protocolVersion;
48+
this.forwardClientAuthData = forwardClientAuthData;
49+
this.proxyConnection = proxyConnection;
4550
}
4651

4752
@Override
@@ -54,10 +59,54 @@ protected ByteBuf newConnectCommand() throws Exception {
5459

5560
authenticationDataProvider = authentication.getAuthData(remoteHostName);
5661
AuthData authData = authenticationDataProvider.authenticate(AuthData.INIT_AUTH_DATA);
57-
return Commands.newConnect(authentication.getAuthMethodName(), authData, this.protocolVersion,
58-
PulsarVersion.getVersion(), proxyToTargetBrokerAddress, clientAuthRole, clientAuthData,
59-
clientAuthMethod);
62+
return Commands.newConnect(authentication.getAuthMethodName(), authData, protocolVersion,
63+
PulsarVersion.getVersion(), proxyToTargetBrokerAddress, clientAuthRole, clientAuthData,
64+
clientAuthMethod);
6065
}
6166

62-
private static final Logger log = LoggerFactory.getLogger(ProxyClientCnx.class);
67+
@Override
68+
protected void handleAuthChallenge(CommandAuthChallenge authChallenge) {
69+
checkArgument(authChallenge.hasChallenge());
70+
checkArgument(authChallenge.getChallenge().hasAuthData());
71+
72+
boolean isRefresh = Arrays.equals(AuthData.REFRESH_AUTH_DATA_BYTES, authChallenge.getChallenge().getAuthData());
73+
if (!forwardClientAuthData || !isRefresh) {
74+
super.handleAuthChallenge(authChallenge);
75+
return;
76+
}
77+
78+
try {
79+
if (log.isDebugEnabled()) {
80+
log.debug("Proxy {} request to refresh the original client authentication data for "
81+
+ "the proxy client {}", proxyConnection.ctx().channel(), ctx.channel());
82+
}
83+
84+
proxyConnection.ctx().writeAndFlush(Commands.newAuthChallenge(clientAuthMethod, AuthData.REFRESH_AUTH_DATA,
85+
protocolVersion))
86+
.addListener(writeFuture -> {
87+
if (writeFuture.isSuccess()) {
88+
if (log.isDebugEnabled()) {
89+
log.debug("Proxy {} sent the auth challenge to original client to refresh credentials "
90+
+ "with method {} for the proxy client {}",
91+
proxyConnection.ctx().channel(), clientAuthMethod, ctx.channel());
92+
}
93+
} else {
94+
log.error("Failed to send the auth challenge to original client by the proxy {} "
95+
+ "for the proxy client {}",
96+
proxyConnection.ctx().channel(),
97+
ctx.channel(),
98+
writeFuture.cause());
99+
closeWithException(writeFuture.cause());
100+
}
101+
});
102+
103+
if (state == State.SentConnectFrame) {
104+
state = State.Connecting;
105+
}
106+
} catch (Exception e) {
107+
log.error("Failed to send the auth challenge to origin client by the proxy {} for the proxy client {}",
108+
proxyConnection.ctx().channel(), ctx.channel(), e);
109+
closeWithException(e);
110+
}
111+
}
63112
}

‎pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java

+62-11
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,7 @@
4141
import javax.naming.AuthenticationException;
4242
import javax.net.ssl.SSLSession;
4343
import lombok.Getter;
44+
import org.apache.pulsar.PulsarVersion;
4445
import org.apache.pulsar.broker.PulsarServerException;
4546
import org.apache.pulsar.broker.authentication.AuthenticationDataSource;
4647
import org.apache.pulsar.broker.authentication.AuthenticationProvider;
@@ -266,12 +267,11 @@ private synchronized void completeConnect(AuthData clientData) throws PulsarClie
266267
this.clientAuthData = clientData;
267268
this.clientAuthMethod = authMethod;
268269
}
269-
clientCnxSupplier =
270-
() -> new ProxyClientCnx(clientConf, service.getWorkerGroup(), clientAuthRole, clientAuthData,
271-
clientAuthMethod, protocolVersionToAdvertise);
270+
clientCnxSupplier = () -> new ProxyClientCnx(clientConf, service.getWorkerGroup(), clientAuthRole,
271+
clientAuthData, clientAuthMethod, protocolVersionToAdvertise,
272+
service.getConfiguration().isForwardAuthorizationCredentials(), this);
272273
} else {
273-
clientCnxSupplier =
274-
() -> new ClientCnx(clientConf, service.getWorkerGroup(), protocolVersionToAdvertise);
274+
clientCnxSupplier = () -> new ClientCnx(clientConf, service.getWorkerGroup(), protocolVersionToAdvertise);
275275
}
276276

277277
if (this.connectionPool == null) {
@@ -372,16 +372,22 @@ public void brokerConnected(DirectProxyHandler directProxyHandler, CommandConnec
372372
}
373373

374374
// According to auth result, send newConnected or newAuthChallenge command.
375-
private void doAuthentication(AuthData clientData) throws Exception {
375+
private void doAuthentication(AuthData clientData)
376+
throws Exception {
376377
AuthData brokerData = authState.authenticate(clientData);
377378
// authentication has completed, will send newConnected command.
378379
if (authState.isComplete()) {
379380
clientAuthRole = authState.getAuthRole();
380381
if (LOG.isDebugEnabled()) {
381382
LOG.debug("[{}] Client successfully authenticated with {} role {}",
382-
remoteAddress, authMethod, clientAuthRole);
383+
remoteAddress, authMethod, clientAuthRole);
384+
}
385+
386+
// First connection
387+
if (this.connectionPool == null || state == State.Connecting) {
388+
// authentication has completed, will send newConnected command.
389+
completeConnect(clientData);
383390
}
384-
completeConnect(clientData);
385391
return;
386392
}
387393

@@ -390,7 +396,7 @@ private void doAuthentication(AuthData clientData) throws Exception {
390396
.addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE);
391397
if (LOG.isDebugEnabled()) {
392398
LOG.debug("[{}] Authentication in progress client by method {}.",
393-
remoteAddress, authMethod);
399+
remoteAddress, authMethod);
394400
}
395401
}
396402

@@ -472,18 +478,63 @@ remoteAddress, protocolVersionToAdvertise, getRemoteEndpointProtocolVersion(),
472478

473479
@Override
474480
protected void handleAuthResponse(CommandAuthResponse authResponse) {
475-
checkArgument(state == State.Connecting);
476481
checkArgument(authResponse.hasResponse());
477482
checkArgument(authResponse.getResponse().hasAuthData() && authResponse.getResponse().hasAuthMethodName());
478483

479484
if (LOG.isDebugEnabled()) {
480485
LOG.debug("Received AuthResponse from {}, auth method: {}",
481-
remoteAddress, authResponse.getResponse().getAuthMethodName());
486+
remoteAddress, authResponse.getResponse().getAuthMethodName());
482487
}
483488

484489
try {
485490
AuthData clientData = AuthData.of(authResponse.getResponse().getAuthData());
486491
doAuthentication(clientData);
492+
if (service.getConfiguration().isForwardAuthorizationCredentials()
493+
&& connectionPool != null && state == State.ProxyLookupRequests) {
494+
connectionPool.getConnections().forEach(toBrokerCnxFuture -> {
495+
String clientVersion;
496+
if (authResponse.hasClientVersion()) {
497+
clientVersion = authResponse.getClientVersion();
498+
} else {
499+
clientVersion = PulsarVersion.getVersion();
500+
}
501+
int protocolVersion;
502+
if (authResponse.hasProtocolVersion()) {
503+
protocolVersion = authResponse.getProtocolVersion();
504+
} else {
505+
protocolVersion = Commands.getCurrentProtocolVersion();
506+
}
507+
508+
ByteBuf cmd =
509+
Commands.newAuthResponse(clientAuthMethod, clientData, protocolVersion, clientVersion);
510+
toBrokerCnxFuture.thenAccept(toBrokerCnx -> toBrokerCnx.ctx().writeAndFlush(cmd)
511+
.addListener(writeFuture -> {
512+
if (writeFuture.isSuccess()) {
513+
if (LOG.isDebugEnabled()) {
514+
LOG.debug("{} authentication is refreshed successfully by {}, "
515+
+ "auth method: {} ",
516+
toBrokerCnx.ctx().channel(), ctx.channel(), clientAuthMethod);
517+
}
518+
} else {
519+
LOG.error("Failed to forward the auth response "
520+
+ "from the proxy to the broker through the proxy client, "
521+
+ "proxy: {}, proxy client: {}",
522+
ctx.channel(),
523+
toBrokerCnx.ctx().channel(),
524+
writeFuture.cause());
525+
toBrokerCnx.ctx().channel().pipeline()
526+
.fireExceptionCaught(writeFuture.cause());
527+
}
528+
}))
529+
.whenComplete((__, ex) -> {
530+
if (ex != null) {
531+
LOG.error("Failed to forward the auth response from the proxy to "
532+
+ "the broker through the proxy client, proxy: {}",
533+
ctx().channel(), ex);
534+
}
535+
});
536+
});
537+
}
487538
} catch (Exception e) {
488539
String msg = "Unable to handleAuthResponse";
489540
LOG.warn("[{}] {} ", remoteAddress, msg, e);
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,186 @@
1+
/**
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
package org.apache.pulsar.proxy.server;
20+
21+
import static java.util.concurrent.TimeUnit.SECONDS;
22+
import static org.mockito.Mockito.spy;
23+
import static org.testng.Assert.assertTrue;
24+
import com.google.common.collect.Sets;
25+
import io.jsonwebtoken.SignatureAlgorithm;
26+
import java.util.Calendar;
27+
import java.util.Collections;
28+
import java.util.HashSet;
29+
import java.util.Optional;
30+
import java.util.Properties;
31+
import java.util.Set;
32+
import java.util.concurrent.CompletableFuture;
33+
import javax.crypto.SecretKey;
34+
import lombok.Cleanup;
35+
import lombok.extern.slf4j.Slf4j;
36+
import org.apache.pulsar.broker.authentication.AuthenticationProviderToken;
37+
import org.apache.pulsar.broker.authentication.AuthenticationService;
38+
import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils;
39+
import org.apache.pulsar.client.admin.PulsarAdmin;
40+
import org.apache.pulsar.client.api.Producer;
41+
import org.apache.pulsar.client.api.ProducerConsumerBase;
42+
import org.apache.pulsar.client.api.PulsarClient;
43+
import org.apache.pulsar.client.impl.ClientCnx;
44+
import org.apache.pulsar.client.impl.PulsarClientImpl;
45+
import org.apache.pulsar.client.impl.auth.AuthenticationToken;
46+
import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
47+
import org.apache.pulsar.common.policies.data.ClusterData;
48+
import org.apache.pulsar.common.policies.data.TenantInfoImpl;
49+
import org.awaitility.Awaitility;
50+
import org.mockito.Mockito;
51+
import org.testng.annotations.AfterClass;
52+
import org.testng.annotations.BeforeClass;
53+
import org.testng.annotations.DataProvider;
54+
import org.testng.annotations.Test;
55+
56+
@Slf4j
57+
public class ProxyRefreshAuthTest extends ProducerConsumerBase {
58+
private final SecretKey SECRET_KEY = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256);
59+
60+
private ProxyService proxyService;
61+
private final ProxyConfiguration proxyConfig = new ProxyConfiguration();
62+
63+
@Override
64+
protected void doInitConf() throws Exception {
65+
super.doInitConf();
66+
67+
// enable tls and auth&auth at broker
68+
conf.setAuthenticationEnabled(true);
69+
conf.setAuthorizationEnabled(false);
70+
conf.setTopicLevelPoliciesEnabled(false);
71+
conf.setProxyRoles(Collections.singleton("Proxy"));
72+
conf.setAdvertisedAddress(null);
73+
conf.setAuthenticateOriginalAuthData(true);
74+
conf.setBrokerServicePort(Optional.of(0));
75+
conf.setWebServicePort(Optional.of(0));
76+
77+
Set<String> superUserRoles = new HashSet<>();
78+
superUserRoles.add("superUser");
79+
conf.setSuperUserRoles(superUserRoles);
80+
81+
conf.setAuthenticationProviders(Set.of(AuthenticationProviderToken.class.getName()));
82+
Properties properties = new Properties();
83+
properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY));
84+
conf.setProperties(properties);
85+
86+
conf.setClusterName("proxy-authorization");
87+
conf.setNumExecutorThreadPoolSize(5);
88+
89+
conf.setAuthenticationRefreshCheckSeconds(1);
90+
}
91+
92+
@BeforeClass
93+
@Override
94+
protected void setup() throws Exception {
95+
super.init();
96+
97+
admin = PulsarAdmin.builder().serviceHttpUrl(pulsar.getWebServiceAddress())
98+
.authentication(new AuthenticationToken(
99+
() -> AuthTokenUtils.createToken(SECRET_KEY, "client", Optional.empty()))).build();
100+
String namespaceName = "my-tenant/my-ns";
101+
admin.clusters().createCluster("proxy-authorization",
102+
ClusterData.builder().serviceUrlTls(brokerUrlTls.toString()).build());
103+
admin.tenants().createTenant("my-tenant",
104+
new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("proxy-authorization")));
105+
admin.namespaces().createNamespace(namespaceName);
106+
107+
// start proxy service
108+
proxyConfig.setAuthenticationEnabled(true);
109+
proxyConfig.setAuthorizationEnabled(false);
110+
proxyConfig.setForwardAuthorizationCredentials(true);
111+
proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl());
112+
proxyConfig.setAdvertisedAddress(null);
113+
114+
proxyConfig.setServicePort(Optional.of(0));
115+
proxyConfig.setBrokerProxyAllowedTargetPorts("*");
116+
proxyConfig.setWebServicePort(Optional.of(0));
117+
118+
proxyConfig.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName());
119+
proxyConfig.setBrokerClientAuthenticationParameters(
120+
AuthTokenUtils.createToken(SECRET_KEY, "Proxy", Optional.empty()));
121+
proxyConfig.setAuthenticationProviders(Set.of(AuthenticationProviderToken.class.getName()));
122+
Properties properties = new Properties();
123+
properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY));
124+
proxyConfig.setProperties(properties);
125+
126+
proxyService = Mockito.spy(new ProxyService(proxyConfig,
127+
new AuthenticationService(
128+
PulsarConfigurationLoader.convertFrom(proxyConfig))));
129+
}
130+
131+
@AfterClass(alwaysRun = true)
132+
@Override
133+
protected void cleanup() throws Exception {
134+
super.internalCleanup();
135+
proxyService.close();
136+
}
137+
138+
private void startProxy(boolean forwardAuthData) throws Exception {
139+
pulsar.getConfiguration().setAuthenticateOriginalAuthData(forwardAuthData);
140+
proxyConfig.setForwardAuthorizationCredentials(forwardAuthData);
141+
proxyService.start();
142+
}
143+
144+
@DataProvider
145+
Object[] forwardAuthDataProvider() {
146+
return new Object[]{true, false};
147+
}
148+
149+
@Test(dataProvider = "forwardAuthDataProvider")
150+
public void testAuthDataRefresh(boolean forwardAuthData) throws Exception {
151+
log.info("-- Starting {} test --", methodName);
152+
153+
startProxy(forwardAuthData);
154+
155+
AuthenticationToken authenticationToken = new AuthenticationToken(() -> {
156+
Calendar calendar = Calendar.getInstance();
157+
calendar.add(Calendar.SECOND, 1);
158+
return AuthTokenUtils.createToken(SECRET_KEY, "client", Optional.of(calendar.getTime()));
159+
});
160+
161+
pulsarClient = PulsarClient.builder().serviceUrl(proxyService.getServiceUrl())
162+
.authentication(authenticationToken)
163+
.build();
164+
165+
String topic = "persistent://my-tenant/my-ns/my-topic1";
166+
@Cleanup
167+
Producer<byte[]> ignored = spy(pulsarClient.newProducer()
168+
.topic(topic).create());
169+
170+
PulsarClientImpl pulsarClientImpl = (PulsarClientImpl) pulsarClient;
171+
Set<CompletableFuture<ClientCnx>> connections = pulsarClientImpl.getCnxPool().getConnections();
172+
173+
Awaitility.await().during(4, SECONDS).untilAsserted(() -> {
174+
pulsarClient.getPartitionsForTopic(topic).get();
175+
assertTrue(connections.stream().allMatch(n -> {
176+
try {
177+
ClientCnx clientCnx = n.get();
178+
long timestamp = clientCnx.getLastDisconnectedTimestamp();
179+
return timestamp == 0;
180+
} catch (Exception e) {
181+
throw new RuntimeException(e);
182+
}
183+
}));
184+
});
185+
}
186+
}

0 commit comments

Comments
 (0)
Please sign in to comment.