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.Collections;
042import java.util.List;
043import java.util.concurrent.CompletableFuture;
044import java.util.concurrent.Exchanger;
045import java.util.concurrent.ExecutionException;
046import java.util.concurrent.ThreadLocalRandom;
047import org.apache.hadoop.conf.Configuration;
048import org.apache.hadoop.hbase.HBaseClassTestRule;
049import org.apache.hadoop.hbase.HBaseZKTestingUtility;
050import org.apache.hadoop.hbase.HConstants;
051import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
052import org.apache.hadoop.hbase.testclassification.MediumTests;
053import org.apache.hadoop.hbase.testclassification.ZKTests;
054import org.apache.zookeeper.AsyncCallback;
055import org.apache.zookeeper.CreateMode;
056import org.apache.zookeeper.KeeperException;
057import org.apache.zookeeper.KeeperException.Code;
058import org.apache.zookeeper.ZooDefs;
059import org.apache.zookeeper.ZooKeeper;
060import org.junit.AfterClass;
061import org.junit.BeforeClass;
062import org.junit.ClassRule;
063import org.junit.Test;
064import org.junit.experimental.categories.Category;
065
066@Category({ ZKTests.class, MediumTests.class })
067public class TestReadOnlyZKClient {
068
069  @ClassRule
070  public static final HBaseClassTestRule CLASS_RULE =
071      HBaseClassTestRule.forClass(TestReadOnlyZKClient.class);
072
073  private static HBaseZKTestingUtility UTIL = new HBaseZKTestingUtility();
074
075  private static String PATH = "/test";
076
077  private static byte[] DATA;
078
079  private static int CHILDREN = 5;
080
081  private static ReadOnlyZKClient RO_ZK;
082
083  @BeforeClass
084  public static void setUp() throws Exception {
085    final int port = UTIL.startMiniZKCluster().getClientPort();
086
087    ZooKeeper zk = ZooKeeperHelper.getConnectedZooKeeper("localhost:" + port, 10000);
088    DATA = new byte[10];
089    ThreadLocalRandom.current().nextBytes(DATA);
090    zk.create(PATH, DATA, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
091    for (int i = 0; i < CHILDREN; i++) {
092      zk.create(PATH + "/c" + i, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
093    }
094    zk.close();
095    Configuration conf = UTIL.getConfiguration();
096    conf.set(HConstants.ZOOKEEPER_QUORUM, "localhost:" + port);
097    conf.setInt(ReadOnlyZKClient.RECOVERY_RETRY, 3);
098    conf.setInt(ReadOnlyZKClient.RECOVERY_RETRY_INTERVAL_MILLIS, 100);
099    conf.setInt(ReadOnlyZKClient.KEEPALIVE_MILLIS, 3000);
100    RO_ZK = new ReadOnlyZKClient(conf);
101    // only connect when necessary
102    assertNull(RO_ZK.zookeeper);
103  }
104
105  @AfterClass
106  public static void tearDown() throws IOException {
107    RO_ZK.close();
108    UTIL.shutdownMiniZKCluster();
109    UTIL.cleanupTestDir();
110  }
111
112  private void waitForIdleConnectionClosed() throws Exception {
113    // The zookeeper client should be closed finally after the keep alive time elapsed
114    UTIL.waitFor(10000, new ExplainingPredicate<Exception>() {
115
116      @Override
117      public boolean evaluate() {
118        return RO_ZK.zookeeper == null;
119      }
120
121      @Override
122      public String explainFailure() {
123        return "Connection to zookeeper is still alive";
124      }
125    });
126  }
127
128  @Test
129  public void testRead() throws Exception {
130    assertArrayEquals(DATA, RO_ZK.get(PATH).get());
131    assertEquals(CHILDREN, RO_ZK.exists(PATH).get().getNumChildren());
132    List<String> children = RO_ZK.list(PATH).get();
133    assertEquals(CHILDREN, children.size());
134    Collections.sort(children);
135    for (int i = 0; i < CHILDREN; i++) {
136      assertEquals("c" + i, children.get(i));
137    }
138    assertNotNull(RO_ZK.zookeeper);
139    waitForIdleConnectionClosed();
140  }
141
142  @Test
143  public void testNoNode() throws InterruptedException, ExecutionException {
144    String pathNotExists = PATH + "_whatever";
145    try {
146      RO_ZK.get(pathNotExists).get();
147      fail("should fail because of " + pathNotExists + " does not exist");
148    } catch (ExecutionException e) {
149      assertThat(e.getCause(), instanceOf(KeeperException.class));
150      KeeperException ke = (KeeperException) e.getCause();
151      assertEquals(Code.NONODE, ke.code());
152      assertEquals(pathNotExists, ke.getPath());
153    }
154    try {
155      RO_ZK.list(pathNotExists).get();
156      fail("should fail because of " + pathNotExists + " does not exist");
157    } catch (ExecutionException e) {
158      assertThat(e.getCause(), instanceOf(KeeperException.class));
159      KeeperException ke = (KeeperException) e.getCause();
160      assertEquals(Code.NONODE, ke.code());
161      assertEquals(pathNotExists, ke.getPath());
162    }
163    // exists will not throw exception.
164    assertNull(RO_ZK.exists(pathNotExists).get());
165  }
166
167  @Test
168  public void testSessionExpire() throws Exception {
169    assertArrayEquals(DATA, RO_ZK.get(PATH).get());
170    ZooKeeper zk = RO_ZK.zookeeper;
171    long sessionId = zk.getSessionId();
172    UTIL.getZkCluster().getZooKeeperServers().get(0).closeSession(sessionId);
173    // should not reach keep alive so still the same instance
174    assertSame(zk, RO_ZK.zookeeper);
175    byte[] got = RO_ZK.get(PATH).get();
176    assertArrayEquals(DATA, got);
177    assertNotNull(RO_ZK.zookeeper);
178    assertNotSame(zk, RO_ZK.zookeeper);
179    assertNotEquals(sessionId, RO_ZK.zookeeper.getSessionId());
180  }
181
182  @Test
183  public void testNotCloseZkWhenPending() throws Exception {
184    ZooKeeper mockedZK = mock(ZooKeeper.class);
185    Exchanger<AsyncCallback.DataCallback> exchanger = new Exchanger<>();
186    doAnswer(i -> {
187      exchanger.exchange(i.getArgument(2));
188      return null;
189    }).when(mockedZK).getData(anyString(), anyBoolean(),
190      any(AsyncCallback.DataCallback.class), any());
191    doAnswer(i -> null).when(mockedZK).close();
192    when(mockedZK.getState()).thenReturn(ZooKeeper.States.CONNECTED);
193    RO_ZK.zookeeper = mockedZK;
194    CompletableFuture<byte[]> future = RO_ZK.get(PATH);
195    AsyncCallback.DataCallback callback = exchanger.exchange(null);
196    // 2 * keep alive time to ensure that we will not close the zk when there are pending requests
197    Thread.sleep(6000);
198    assertNotNull(RO_ZK.zookeeper);
199    verify(mockedZK, never()).close();
200    callback.processResult(Code.OK.intValue(), PATH, null, DATA, null);
201    assertArrayEquals(DATA, future.get());
202    // now we will close the idle connection.
203    waitForIdleConnectionClosed();
204    verify(mockedZK, times(1)).close();
205  }
206}