From 619ad64212e0f8eb02455e7c038fdfea42b11b56 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 1 Mar 2017 14:09:26 -0800 Subject: [PATCH] Replaced use of System.out.println() in unit tests (#260) --- .../mledger/impl/ManagedLedgerTest.java | 3 +-- .../SimpleLoadManagerImplTest.java | 10 +++++----- .../namespace/NamespaceServiceTest.java | 6 +++++- .../PersistentTopicConcurrentTest.java | 19 ++++++++----------- .../broker/service/PersistentTopicTest.java | 4 ++-- .../pulsar/broker/web/WebServiceTest.java | 7 +++++-- .../api/PartitionedProducerConsumerTest.java | 4 ++-- .../api/SimpleProducerConsumerTest.java | 4 ++-- .../impl/BrokerClientIntegrationTest.java | 8 ++++---- .../common/naming/NamespaceBundlesTest.java | 7 +++++-- .../client/PulsarBrokerStatsClientTest.java | 10 ++++++++-- .../yahoo/pulsar/client/impl/BackoffTest.java | 1 - 12 files changed, 47 insertions(+), 36 deletions(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java index 7396c169dc19b..f5e27b90be237 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java @@ -1898,8 +1898,7 @@ public void testActiveDeactiveCursorWithDiscardEntriesFromCache() throws Excepti cursor2.markDelete((entries2.get(entries2.size() - 1)).getPosition()); // (3) Validate: cache should remove all entries read by both active cursors - System.out.println( - "expected, found :" + (5 * (totalInsertedEntries - readEntries)) + ", " + entryCache.getSize()); + log.info("expected, found : {}, {}", (5 * (totalInsertedEntries - readEntries)), entryCache.getSize()); assertEquals((5 * (totalInsertedEntries - readEntries)), entryCache.getSize()); final int remainingEntries = totalInsertedEntries - readEntries; diff --git a/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java b/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java index 6c361694c0475..7d80959408323 100644 --- a/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java @@ -280,18 +280,18 @@ public void testPrimarySecondary() throws Exception { LocalZooKeeperCache originalLZK1 = (LocalZooKeeperCache) zkCacheField.get(pulsar1); LocalZooKeeperCache originalLZK2 = (LocalZooKeeperCache) zkCacheField.get(pulsar2); - System.out.println("lzk are " + originalLZK1.getChildren(SimpleLoadManagerImpl.LOADBALANCE_BROKERS_ROOT) - + " 2: " + originalLZK2.getChildren(SimpleLoadManagerImpl.LOADBALANCE_BROKERS_ROOT)); + log.info("lzk are {} 2: {}", originalLZK1.getChildren(SimpleLoadManagerImpl.LOADBALANCE_BROKERS_ROOT), + originalLZK2.getChildren(SimpleLoadManagerImpl.LOADBALANCE_BROKERS_ROOT)); zkCacheField.set(pulsar1, mockCache); LocalZooKeeperCache newZk = (LocalZooKeeperCache) pulsar1.getLocalZkCache(); - System.out.println("lzk mocked are " + newZk.getChildren(SimpleLoadManagerImpl.LOADBALANCE_BROKERS_ROOT)); + log.info("lzk mocked are {}", newZk.getChildren(SimpleLoadManagerImpl.LOADBALANCE_BROKERS_ROOT)); ZooKeeperChildrenCache availableActiveBrokers = new ZooKeeperChildrenCache(pulsar1.getLocalZkCache(), SimpleLoadManagerImpl.LOADBALANCE_BROKERS_ROOT); - System.out.println("lzk mocked active brokers are " - + availableActiveBrokers.get(SimpleLoadManagerImpl.LOADBALANCE_BROKERS_ROOT)); + log.info("lzk mocked active brokers are {}", + availableActiveBrokers.get(SimpleLoadManagerImpl.LOADBALANCE_BROKERS_ROOT)); LoadManager loadManager = new SimpleLoadManagerImpl(pulsar1); diff --git a/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/namespace/NamespaceServiceTest.java b/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/namespace/NamespaceServiceTest.java index ea4647659506f..0ea82907a9f5b 100644 --- a/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/namespace/NamespaceServiceTest.java +++ b/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/namespace/NamespaceServiceTest.java @@ -42,6 +42,8 @@ import org.apache.zookeeper.data.Stat; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -126,7 +128,7 @@ public void testSplitAndOwnBundles() throws Exception { Policies policies = ObjectMapperFactory.getThreadLocal().readValue(content, Policies.class); NamespaceBundles localZkBundles = bundleFactory.getBundles(nsname, policies.bundles); assertTrue(updatedNsBundles.equals(localZkBundles)); - System.out.println(policies); + log.info("Policies: {}", policies); // (3) validate ownership of new split bundles by local owner bundleList.stream().forEach(b -> { @@ -287,4 +289,6 @@ private Pair> splitBundles(NamespaceBund CompletableFuture.completedFuture(bundles)); return utilityFactory.splitBundles(targetBundle, 2); } + + private static final Logger log = LoggerFactory.getLogger(NamespaceServiceTest.class); } diff --git a/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/service/PersistentTopicConcurrentTest.java b/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/service/PersistentTopicConcurrentTest.java index f038011ddb976..53d4ff2bc7de6 100644 --- a/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/service/PersistentTopicConcurrentTest.java +++ b/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/service/PersistentTopicConcurrentTest.java @@ -128,7 +128,7 @@ public void run() { barrier.await(); // assertTrue(topic.unsubscribe(successSubName).isDone()); Thread.sleep(5, 0); - System.out.println("deleter outcome is " + topic.delete().get()); + log.info("deleter outcome is {}", topic.delete().get()); } catch (Exception e) { e.printStackTrace(); gotException.set(true); @@ -146,7 +146,7 @@ public void run() { ConcurrentOpenHashMap subscriptions = topic.getSubscriptions(); PersistentSubscription ps = subscriptions.get(successSubName); // Thread.sleep(2,0); - System.out.println("unsubscriber outcome is " + ps.doUnsubscribe(ps.getConsumers().get(0)).get()); + log.info("unsubscriber outcome is {}", ps.doUnsubscribe(ps.getConsumers().get(0)).get()); // assertFalse(ps.delete().isCompletedExceptionally()); } catch (Exception e) { e.printStackTrace(); @@ -186,11 +186,11 @@ public void run() { barrier.await(); // assertTrue(topic.unsubscribe(successSubName).isDone()); // Thread.sleep(5,0); - System.out.println(Thread.currentThread() + " forcing topic GC "); + log.info("{} forcing topic GC ", Thread.currentThread()); for (int i = 0; i < 2000; i++) { topic.checkGC(0); } - System.out.println("GC done.."); + log.info("GC done.."); } catch (Exception e) { e.printStackTrace(); gotException.set(true); @@ -208,7 +208,7 @@ public void run() { ConcurrentOpenHashMap subscriptions = topic.getSubscriptions(); PersistentSubscription ps = subscriptions.get(successSubName); // Thread.sleep(2,0); - System.out.println("unsubscriber outcome is " + ps.doUnsubscribe(ps.getConsumers().get(0)).get()); + log.info("unsubscriber outcome is {}", ps.doUnsubscribe(ps.getConsumers().get(0)).get()); // assertFalse(ps.delete().isCompletedExceptionally()); } catch (Exception e) { e.printStackTrace(); @@ -247,7 +247,7 @@ public void run() { try { barrier.await(); Thread.sleep(4, 700); - System.out.println("deleter outcome is " + topic.delete().get()); + log.info("deleter outcome is {}", topic.delete().get()); } catch (Exception e) { e.printStackTrace(); gotException.set(true); @@ -265,8 +265,8 @@ public void run() { // assertTrue(topic.unsubscribe(successSubName).isDone()); ConcurrentOpenHashMap subscriptions = topic.getSubscriptions(); PersistentSubscription ps = subscriptions.get(successSubName); - System.out.println("unsubscribe result : " + topic.unsubscribe(successSubName).get()); - System.out.println("closing consumer.."); + log.info("unsubscribe result : {}", topic.unsubscribe(successSubName).get()); + log.info("closing consumer.."); ps.getConsumers().get(0).close(); } catch (Exception e) { e.printStackTrace(); @@ -325,10 +325,7 @@ public void run() { // assertTrue(topic.unsubscribe(successSubName).isDone()); ConcurrentOpenHashMap subscriptions = topic.getSubscriptions(); PersistentSubscription ps = subscriptions.get(successSubName); - // System.out.println("unsubscribe result : " + topic.unsubscribe(successSubName).get()); log.info("unsubscribe result : " + ps.doUnsubscribe(ps.getConsumers().get(0)).get()); - // System.out.println("closing consumer.."); - // ps.getConsumers().get(0).close(); } catch (Exception e) { e.printStackTrace(); gotException.set(true); diff --git a/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/service/PersistentTopicTest.java b/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/service/PersistentTopicTest.java index 0d731587a3791..1d85a7b91cd3f 100644 --- a/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/service/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/service/PersistentTopicTest.java @@ -535,7 +535,7 @@ public void run() { barrier.await(); // assertTrue(topic.unsubscribe(successSubName).isDone()); Thread.sleep(5, 0); - System.out.println("deleter outcome is " + topic.delete().get()); + log.info("deleter outcome is {}", topic.delete().get()); } catch (Exception e) { e.printStackTrace(); gotException.set(true); @@ -554,7 +554,7 @@ public void run() { ConcurrentOpenHashMap subscriptions = topic.getSubscriptions(); PersistentSubscription ps = subscriptions.get(successSubName); // Thread.sleep(5,0); - System.out.println("unsubscriber outcome is " + ps.doUnsubscribe(ps.getConsumers().get(0)).get()); + log.info("unsubscriber outcome is {}", ps.doUnsubscribe(ps.getConsumers().get(0)).get()); // assertFalse(ps.delete().isCompletedExceptionally()); } catch (Exception e) { e.printStackTrace(); diff --git a/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/web/WebServiceTest.java b/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/web/WebServiceTest.java index 948fa14734ca8..29c6e97cc61e4 100644 --- a/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/web/WebServiceTest.java +++ b/pulsar-broker/src/test/java/com/yahoo/pulsar/broker/web/WebServiceTest.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; -import java.net.InetAddress; import java.net.URL; import java.security.KeyStore; import java.security.PrivateKey; @@ -40,6 +39,8 @@ import org.apache.bookkeeper.test.PortManager; import org.apache.zookeeper.CreateMode; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.Test; @@ -259,7 +260,7 @@ private String makeHttpRequest(boolean useTls, boolean useAuth) throws Exception response = new URL(BROKER_LOOKUP_URL).openStream(); } String resp = CharStreams.toString(new InputStreamReader(response)); - System.out.println(resp); + log.info("Response: {}", resp); return resp; } finally { Closeables.close(response, false); @@ -337,4 +338,6 @@ void teardown() throws Exception { Assert.fail("Got exception while closing the pulsar instance ", e); } } + + private static final Logger log = LoggerFactory.getLogger(WebServiceTest.class); } diff --git a/pulsar-broker/src/test/java/com/yahoo/pulsar/client/api/PartitionedProducerConsumerTest.java b/pulsar-broker/src/test/java/com/yahoo/pulsar/client/api/PartitionedProducerConsumerTest.java index 970e9fc37e61c..fb9d429656f2a 100644 --- a/pulsar-broker/src/test/java/com/yahoo/pulsar/client/api/PartitionedProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/com/yahoo/pulsar/client/api/PartitionedProducerConsumerTest.java @@ -377,7 +377,7 @@ public void testAsyncPartitionedProducerConsumer() throws Exception { producer.send(message.getBytes()); } - System.out.println(" start receiving messages :"); + log.info(" start receiving messages :"); // receive messages CountDownLatch latch = new CountDownLatch(totalMsg); @@ -428,7 +428,7 @@ public void testAsyncPartitionedProducerConsumerQueueSizeOne() throws Exception producer.send(message.getBytes()); } - System.out.println(" start receiving messages :"); + log.info(" start receiving messages :"); // receive messages CountDownLatch latch = new CountDownLatch(totalMsg); diff --git a/pulsar-broker/src/test/java/com/yahoo/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/com/yahoo/pulsar/client/api/SimpleProducerConsumerTest.java index d5549eb8231d8..90ecba9a67628 100644 --- a/pulsar-broker/src/test/java/com/yahoo/pulsar/client/api/SimpleProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/com/yahoo/pulsar/client/api/SimpleProducerConsumerTest.java @@ -781,7 +781,7 @@ public void testAsyncProducerAndConsumer() throws Exception { produceMsgs.add(message); } - System.out.println(" start receiving messages :"); + log.info(" start receiving messages :"); CountDownLatch latch = new CountDownLatch(totalMsg); // receive messages ExecutorService executor = Executors.newFixedThreadPool(1); @@ -822,7 +822,7 @@ public void testAsyncProducerAndConsumerWithZeroQueueSize() throws Exception { produceMsgs.add(message); } - System.out.println(" start receiving messages :"); + log.info(" start receiving messages :"); CountDownLatch latch = new CountDownLatch(totalMsg); // receive messages ExecutorService executor = Executors.newFixedThreadPool(1); diff --git a/pulsar-broker/src/test/java/com/yahoo/pulsar/client/impl/BrokerClientIntegrationTest.java b/pulsar-broker/src/test/java/com/yahoo/pulsar/client/impl/BrokerClientIntegrationTest.java index bcde1603f49fd..dae05f424f47e 100644 --- a/pulsar-broker/src/test/java/com/yahoo/pulsar/client/impl/BrokerClientIntegrationTest.java +++ b/pulsar-broker/src/test/java/com/yahoo/pulsar/client/impl/BrokerClientIntegrationTest.java @@ -368,12 +368,12 @@ public void testResetCursor(SubscriptionType subType) throws Exception { } consumer.acknowledge(msg); long publishTime = ((MessageImpl) msg).getPublishTime(); - System.out.println(" publish time is " + publishTime + "," + msg.getMessageId()); + log.info(" publish time is " + publishTime + "," + msg.getMessageId()); TimestampEntryCount timestampEntryCount = publishTimeIdMap.computeIfAbsent(publishTime, (k) -> new TimestampEntryCount(publishTime)); timestampEntryCount.incrementAndGet(); } catch (final PulsarClientException e) { - System.out.println("Failed to ack!"); + log.warn("Failed to ack!"); } }); @@ -400,7 +400,7 @@ public void testResetCursor(SubscriptionType subType) throws Exception { received.clear(); // publish testSize num of msgs - System.out.println("Sending more messages."); + log.info("Sending more messages."); for (Integer n = 0; n < testSize; n++) { producer.send(msgBytes); Thread.sleep(1); @@ -421,7 +421,7 @@ public void testResetCursor(SubscriptionType subType) throws Exception { received.clear(); log.info("reset cursor to " + timestamp + " for topic " + destName.toString() + " for subs " + subsId); - System.out.println("issuing admin operation on " + admin.getServiceUrl().toString()); + log.info("issuing admin operation on " + admin.getServiceUrl().toString()); List subList = admin.persistentTopics().getSubscriptions(destName.toString()); for (String subs : subList) { log.info("got sub " + subs); diff --git a/pulsar-broker/src/test/java/com/yahoo/pulsar/common/naming/NamespaceBundlesTest.java b/pulsar-broker/src/test/java/com/yahoo/pulsar/common/naming/NamespaceBundlesTest.java index df4503695bc61..12e2ded9e10f2 100644 --- a/pulsar-broker/src/test/java/com/yahoo/pulsar/common/naming/NamespaceBundlesTest.java +++ b/pulsar-broker/src/test/java/com/yahoo/pulsar/common/naming/NamespaceBundlesTest.java @@ -28,6 +28,8 @@ import java.util.concurrent.CompletableFuture; import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.testng.annotations.Test; import com.github.benmanes.caffeine.cache.AsyncLoadingCache; @@ -269,7 +271,8 @@ private void assertBundleDivideInTwo(NamespaceBundle bundle, List [%s,%s]", range[0], range[1], lRange, uRange)); - + log.info("[{},{}] => [{},{}]", range[0], range[1], lRange, uRange); } + + private static final Logger log = LoggerFactory.getLogger(NamespaceBundlesTest.class); } diff --git a/pulsar-broker/src/test/java/com/yahoo/pulsar/stats/client/PulsarBrokerStatsClientTest.java b/pulsar-broker/src/test/java/com/yahoo/pulsar/stats/client/PulsarBrokerStatsClientTest.java index ef0591f44a794..78ea69d4376ed 100644 --- a/pulsar-broker/src/test/java/com/yahoo/pulsar/stats/client/PulsarBrokerStatsClientTest.java +++ b/pulsar-broker/src/test/java/com/yahoo/pulsar/stats/client/PulsarBrokerStatsClientTest.java @@ -23,6 +23,8 @@ import javax.ws.rs.ClientErrorException; import javax.ws.rs.ServerErrorException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.testng.annotations.Test; import com.yahoo.pulsar.client.admin.PulsarAdmin; @@ -40,7 +42,8 @@ public class PulsarBrokerStatsClientTest { @Test public void testServiceException() throws Exception { URL url = new URL("http://localhost:15000"); - BrokerStatsImpl client = (BrokerStatsImpl) spy(new PulsarAdmin(url, (Authentication) null).brokerStats()); + PulsarAdmin admin = new PulsarAdmin(url, (Authentication) null); + BrokerStatsImpl client = (BrokerStatsImpl) spy(admin.brokerStats()); try { client.getLoadReport(); } catch (PulsarAdminException e) { @@ -64,7 +67,10 @@ public void testServiceException() throws Exception { assertTrue(client.getApiException(new ServerErrorException(500)) instanceof ServerSideErrorException); assertTrue(client.getApiException(new ServerErrorException(503)) instanceof PulsarAdminException); - System.out.println(client); + log.info("Client: ", client); + + admin.close(); } + private static final Logger log = LoggerFactory.getLogger(PulsarBrokerStatsClientTest.class); } diff --git a/pulsar-client/src/test/java/com/yahoo/pulsar/client/impl/BackoffTest.java b/pulsar-client/src/test/java/com/yahoo/pulsar/client/impl/BackoffTest.java index 60fc7dd49542b..1fdbf1c71d018 100644 --- a/pulsar-client/src/test/java/com/yahoo/pulsar/client/impl/BackoffTest.java +++ b/pulsar-client/src/test/java/com/yahoo/pulsar/client/impl/BackoffTest.java @@ -26,7 +26,6 @@ public class BackoffTest { @Test public void shouldBackoffTest() { long currentTimestamp = System.nanoTime(); - System.out.println(currentTimestamp); Backoff testBackoff = new Backoff(currentTimestamp, TimeUnit.NANOSECONDS, 100, TimeUnit.MICROSECONDS); // gives false Assert.assertTrue(!testBackoff.shouldBackoff(0L, TimeUnit.NANOSECONDS, 0));