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