001/** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.zookeeper; 019 020import static org.hamcrest.CoreMatchers.instanceOf; 021import static org.junit.Assert.assertArrayEquals; 022import static org.junit.Assert.assertEquals; 023import static org.junit.Assert.assertNotEquals; 024import static org.junit.Assert.assertNotNull; 025import static org.junit.Assert.assertNotSame; 026import static org.junit.Assert.assertNull; 027import static org.junit.Assert.assertSame; 028import static org.junit.Assert.assertThat; 029import static org.junit.Assert.fail; 030import static org.mockito.ArgumentMatchers.any; 031import static org.mockito.ArgumentMatchers.anyBoolean; 032import static org.mockito.ArgumentMatchers.anyString; 033import static org.mockito.Mockito.doAnswer; 034import static org.mockito.Mockito.mock; 035import static org.mockito.Mockito.never; 036import static org.mockito.Mockito.times; 037import static org.mockito.Mockito.verify; 038import static org.mockito.Mockito.when; 039 040import java.io.IOException; 041import java.util.concurrent.CompletableFuture; 042import java.util.concurrent.Exchanger; 043import java.util.concurrent.ExecutionException; 044import java.util.concurrent.ThreadLocalRandom; 045import org.apache.hadoop.conf.Configuration; 046import org.apache.hadoop.hbase.HBaseClassTestRule; 047import org.apache.hadoop.hbase.HBaseZKTestingUtility; 048import org.apache.hadoop.hbase.HConstants; 049import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; 050import org.apache.hadoop.hbase.testclassification.MediumTests; 051import org.apache.hadoop.hbase.testclassification.ZKTests; 052import org.apache.zookeeper.AsyncCallback; 053import org.apache.zookeeper.CreateMode; 054import org.apache.zookeeper.KeeperException; 055import org.apache.zookeeper.KeeperException.Code; 056import org.apache.zookeeper.ZooDefs; 057import org.apache.zookeeper.ZooKeeper; 058import org.junit.AfterClass; 059import org.junit.BeforeClass; 060import org.junit.ClassRule; 061import org.junit.Test; 062import org.junit.experimental.categories.Category; 063 064@Category({ ZKTests.class, MediumTests.class }) 065public class TestReadOnlyZKClient { 066 067 @ClassRule 068 public static final HBaseClassTestRule CLASS_RULE = 069 HBaseClassTestRule.forClass(TestReadOnlyZKClient.class); 070 071 private static HBaseZKTestingUtility UTIL = new HBaseZKTestingUtility(); 072 073 private static String PATH = "/test"; 074 075 private static byte[] DATA; 076 077 private static int CHILDREN = 5; 078 079 private static ReadOnlyZKClient RO_ZK; 080 081 @BeforeClass 082 public static void setUp() throws Exception { 083 final int port = UTIL.startMiniZKCluster().getClientPort(); 084 085 ZooKeeper zk = ZooKeeperHelper.getConnectedZooKeeper("localhost:" + port, 10000); 086 DATA = new byte[10]; 087 ThreadLocalRandom.current().nextBytes(DATA); 088 zk.create(PATH, DATA, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); 089 for (int i = 0; i < CHILDREN; i++) { 090 zk.create(PATH + "/c" + i, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); 091 } 092 zk.close(); 093 Configuration conf = UTIL.getConfiguration(); 094 conf.set(HConstants.ZOOKEEPER_QUORUM, "localhost:" + port); 095 conf.setInt(ReadOnlyZKClient.RECOVERY_RETRY, 3); 096 conf.setInt(ReadOnlyZKClient.RECOVERY_RETRY_INTERVAL_MILLIS, 100); 097 conf.setInt(ReadOnlyZKClient.KEEPALIVE_MILLIS, 3000); 098 RO_ZK = new ReadOnlyZKClient(conf); 099 // only connect when necessary 100 assertNull(RO_ZK.zookeeper); 101 } 102 103 @AfterClass 104 public static void tearDown() throws IOException { 105 RO_ZK.close(); 106 UTIL.shutdownMiniZKCluster(); 107 UTIL.cleanupTestDir(); 108 } 109 110 private void waitForIdleConnectionClosed() throws Exception { 111 // The zookeeper client should be closed finally after the keep alive time elapsed 112 UTIL.waitFor(10000, new ExplainingPredicate<Exception>() { 113 114 @Override 115 public boolean evaluate() { 116 return RO_ZK.zookeeper == null; 117 } 118 119 @Override 120 public String explainFailure() { 121 return "Connection to zookeeper is still alive"; 122 } 123 }); 124 } 125 126 @Test 127 public void testGetAndExists() throws Exception { 128 assertArrayEquals(DATA, RO_ZK.get(PATH).get()); 129 assertEquals(CHILDREN, RO_ZK.exists(PATH).get().getNumChildren()); 130 assertNotNull(RO_ZK.zookeeper); 131 waitForIdleConnectionClosed(); 132 } 133 134 @Test 135 public void testNoNode() throws InterruptedException, ExecutionException { 136 String pathNotExists = PATH + "_whatever"; 137 try { 138 RO_ZK.get(pathNotExists).get(); 139 fail("should fail because of " + pathNotExists + " does not exist"); 140 } catch (ExecutionException e) { 141 assertThat(e.getCause(), instanceOf(KeeperException.class)); 142 KeeperException ke = (KeeperException) e.getCause(); 143 assertEquals(Code.NONODE, ke.code()); 144 assertEquals(pathNotExists, ke.getPath()); 145 } 146 // exists will not throw exception. 147 assertNull(RO_ZK.exists(pathNotExists).get()); 148 } 149 150 @Test 151 public void testSessionExpire() throws Exception { 152 assertArrayEquals(DATA, RO_ZK.get(PATH).get()); 153 ZooKeeper zk = RO_ZK.zookeeper; 154 long sessionId = zk.getSessionId(); 155 UTIL.getZkCluster().getZooKeeperServers().get(0).closeSession(sessionId); 156 // should not reach keep alive so still the same instance 157 assertSame(zk, RO_ZK.zookeeper); 158 byte[] got = RO_ZK.get(PATH).get(); 159 assertArrayEquals(DATA, got); 160 assertNotNull(RO_ZK.zookeeper); 161 assertNotSame(zk, RO_ZK.zookeeper); 162 assertNotEquals(sessionId, RO_ZK.zookeeper.getSessionId()); 163 } 164 165 @Test 166 public void testNotCloseZkWhenPending() throws Exception { 167 ZooKeeper mockedZK = mock(ZooKeeper.class); 168 Exchanger<AsyncCallback.DataCallback> exchanger = new Exchanger<>(); 169 doAnswer(i -> { 170 exchanger.exchange(i.getArgument(2)); 171 return null; 172 }).when(mockedZK).getData(anyString(), anyBoolean(), 173 any(AsyncCallback.DataCallback.class), any()); 174 doAnswer(i -> null).when(mockedZK).close(); 175 when(mockedZK.getState()).thenReturn(ZooKeeper.States.CONNECTED); 176 RO_ZK.zookeeper = mockedZK; 177 CompletableFuture<byte[]> future = RO_ZK.get(PATH); 178 AsyncCallback.DataCallback callback = exchanger.exchange(null); 179 // 2 * keep alive time to ensure that we will not close the zk when there are pending requests 180 Thread.sleep(6000); 181 assertNotNull(RO_ZK.zookeeper); 182 verify(mockedZK, never()).close(); 183 callback.processResult(Code.OK.intValue(), PATH, null, DATA, null); 184 assertArrayEquals(DATA, future.get()); 185 // now we will close the idle connection. 186 waitForIdleConnectionClosed(); 187 verify(mockedZK, times(1)).close(); 188 } 189}