From 6cc362774c93a63809476208961af51687ec43a2 Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Sun, 1 Mar 2020 17:28:56 +0800 Subject: [PATCH] [Flink-Connector]Get PulsarClient from cache should always return an open instance (#6436) --- .../pulsar/client/impl/PulsarClientImpl.java | 6 ++++- .../connectors/pulsar/CachedPulsarClient.java | 8 +++++- .../pulsar/CachedPulsarClientTest.java | 25 +++++++++++++++++++ 3 files changed, 37 insertions(+), 2 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index f51fb6b65c3f3..db66c90ca7150 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -94,7 +94,7 @@ public class PulsarClientImpl implements PulsarClient { private final Timer timer; private final ExecutorProvider externalExecutorProvider; - enum State { + public enum State { Open, Closing, Closed } @@ -167,6 +167,10 @@ public Clock getClientClock() { return clientClock; } + public AtomicReference getState() { + return state; + } + @Override public ProducerBuilder newProducer() { return new ProducerBuilderImpl<>(this, Schema.BYTES); diff --git a/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/CachedPulsarClient.java b/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/CachedPulsarClient.java index 5d5715e846d9e..4de514520e19f 100644 --- a/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/CachedPulsarClient.java +++ b/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/CachedPulsarClient.java @@ -77,7 +77,13 @@ private static PulsarClientImpl createPulsarClient( } public static PulsarClientImpl getOrCreate(ClientConfigurationData config) throws ExecutionException { - return guavaCache.get(config); + PulsarClientImpl instance = guavaCache.get(config); + if (instance.getState().get() == PulsarClientImpl.State.Open) { + return instance; + } else { + guavaCache.invalidate(config); + return guavaCache.get(config); + } } private static void close(ClientConfigurationData clientConfig, PulsarClientImpl client) { diff --git a/pulsar-flink/src/test/java/org/apache/flink/streaming/connectors/pulsar/CachedPulsarClientTest.java b/pulsar-flink/src/test/java/org/apache/flink/streaming/connectors/pulsar/CachedPulsarClientTest.java index a41609f4a2ba2..39cdca19a37cc 100644 --- a/pulsar-flink/src/test/java/org/apache/flink/streaming/connectors/pulsar/CachedPulsarClientTest.java +++ b/pulsar-flink/src/test/java/org/apache/flink/streaming/connectors/pulsar/CachedPulsarClientTest.java @@ -100,4 +100,29 @@ public void testShouldCloseTheCorrectClient() throws Exception { assertEquals(map2.values().iterator().next(), client1); } + + @Test + public void getClientFromCacheShouldAlwaysReturnAnOpenedInstance() throws Exception { + PulsarClientImpl impl1 = Mockito.mock(PulsarClientImpl.class); + + ClientConfigurationData conf1 = new ClientConfigurationData(); + conf1.setServiceUrl(SERVICE_URL); + + PowerMockito.whenNew(PulsarClientImpl.class) + .withArguments(conf1).thenReturn(impl1); + + PulsarClientImpl client1 = CachedPulsarClient.getOrCreate(conf1); + + ConcurrentMap map1 = CachedPulsarClient.getAsMap(); + assertEquals(map1.size(), 1); + + client1.getState().set(PulsarClientImpl.State.Closed); + + PulsarClientImpl client2 = CachedPulsarClient.getOrCreate(conf1); + + assertNotEquals(client1, client2); + + ConcurrentMap map2 = CachedPulsarClient.getAsMap(); + assertEquals(map2.size(), 1); + } }