1   /**
2    * Copyright 2009 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase;
21  
22  import static org.junit.Assert.assertFalse;
23  import static org.junit.Assert.assertEquals;
24  import static org.junit.Assert.assertNotNull;
25  import static org.junit.Assert.assertNull;
26  import static org.junit.Assert.fail;
27  
28  import java.io.IOException;
29  import java.lang.reflect.InvocationTargetException;
30  import java.lang.reflect.Method;
31  import java.util.concurrent.ExecutorService;
32  import java.util.concurrent.SynchronousQueue;
33  import java.util.concurrent.ThreadPoolExecutor;
34  import java.util.concurrent.TimeUnit;
35  
36  import org.apache.commons.logging.Log;
37  import org.apache.commons.logging.LogFactory;
38  import org.apache.hadoop.conf.Configuration;
39  import org.apache.hadoop.hbase.client.Get;
40  import org.apache.hadoop.hbase.client.HBaseAdmin;
41  import org.apache.hadoop.hbase.client.HConnection;
42  import org.apache.hadoop.hbase.client.HConnectionManager;
43  import org.apache.hadoop.hbase.client.HTable;
44  import org.apache.hadoop.hbase.client.Put;
45  import org.apache.hadoop.hbase.master.HMaster;
46  import org.apache.hadoop.hbase.util.Bytes;
47  import org.apache.hadoop.hbase.util.Threads;
48  import org.apache.hadoop.hbase.zookeeper.ZKConfig;
49  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
50  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
51  import org.apache.zookeeper.CreateMode;
52  import org.apache.zookeeper.KeeperException;
53  import org.apache.zookeeper.ZooDefs;
54  import org.apache.zookeeper.ZooKeeper;
55  import org.apache.zookeeper.ZooKeeper.States;
56  import org.junit.AfterClass;
57  import org.junit.Assert;
58  import org.junit.Before;
59  import org.junit.BeforeClass;
60  import org.junit.Test;
61  import org.junit.experimental.categories.Category;
62  
63  @Category(LargeTests.class)
64  public class TestZooKeeper {
65    private final Log LOG = LogFactory.getLog(this.getClass());
66  
67    private final static HBaseTestingUtility
68        TEST_UTIL = new HBaseTestingUtility();
69    private static HConnection persistentConnection;
70  
71    /**
72     * @throws java.lang.Exception
73     */
74    @BeforeClass
75    public static void setUpBeforeClass() throws Exception {
76      // create a connection *before* the cluster is started, to validate that the
77      // connection's ZK trackers are initialized on demand
78      persistentConnection = HConnectionManager.createConnection(TEST_UTIL.getConfiguration());
79  
80      // Test we can first start the ZK cluster by itself
81      TEST_UTIL.startMiniZKCluster();
82      TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
83      TEST_UTIL.startMiniCluster(2);
84    }
85  
86    /**
87     * @throws java.lang.Exception
88     */
89    @AfterClass
90    public static void tearDownAfterClass() throws Exception {
91      persistentConnection.close();
92      TEST_UTIL.shutdownMiniCluster();
93    }
94  
95    /**
96     * @throws java.lang.Exception
97     */
98    @Before
99    public void setUp() throws Exception {
100     TEST_UTIL.ensureSomeRegionServersAvailable(2);
101   }
102 
103   private ZooKeeperWatcher getZooKeeperWatcher(HConnection c) throws
104     NoSuchMethodException, InvocationTargetException, IllegalAccessException {
105 
106     Method getterZK = c.getClass().getMethod("getKeepAliveZooKeeperWatcher");
107     getterZK.setAccessible(true);
108 
109     return (ZooKeeperWatcher) getterZK.invoke(c);
110   }
111 
112   /**
113    * See HBASE-1232 and http://wiki.apache.org/hadoop/ZooKeeper/FAQ#4.
114    * @throws IOException
115    * @throws InterruptedException
116    */
117   // fails frequently, disabled for now, see HBASE-6406
118   // @Test
119   public void testClientSessionExpired()
120   throws Exception {
121     LOG.info("testClientSessionExpired");
122     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
123     new HTable(c, HConstants.META_TABLE_NAME).close();
124     HConnection connection = HConnectionManager.getConnection(c);
125     ZooKeeperWatcher connectionZK = connection.getZooKeeperWatcher();
126     TEST_UTIL.expireSession(connectionZK, false);
127 
128     // provoke session expiration by doing something with ZK
129     ZKUtil.dump(connectionZK);
130 
131     // Check that the old ZK connection is closed, means we did expire
132     System.err.println("ZooKeeper should have timed out");
133     LOG.info("state=" + connectionZK.getRecoverableZooKeeper().getState());
134     Assert.assertTrue(connectionZK.getRecoverableZooKeeper().getState().
135       equals(States.CLOSED));
136 
137     // Check that the client recovered
138     ZooKeeperWatcher newConnectionZK = connection.getZooKeeperWatcher();
139     States state = newConnectionZK.getRecoverableZooKeeper().getState();
140     LOG.info("state=" + state);
141     Assert.assertTrue(state.equals(States.CONNECTED) || state.equals(States.CONNECTING));
142   }
143   
144   @Test
145   public void testRegionServerSessionExpired() throws Exception {
146     LOG.info("Starting testRegionServerSessionExpired");
147     int metaIndex = TEST_UTIL.getMiniHBaseCluster().getServerWithMeta();
148     TEST_UTIL.expireRegionServerSession(metaIndex);
149     testSanity();
150   }
151 
152   // @Test Disabled because seems to make no sense expiring master session
153   // and then trying to create table (down in testSanity); on master side
154   // it will fail because the master's session has expired -- St.Ack 07/24/2012
155   public void testMasterSessionExpired() throws Exception {
156     LOG.info("Starting testMasterSessionExpired");
157     TEST_UTIL.expireMasterSession();
158     testSanity();
159   }
160 
161   /**
162    * Make sure we can use the cluster
163    * @throws Exception
164    */
165   private void testSanity() throws Exception {
166     String tableName = "test"+System.currentTimeMillis();
167     HBaseAdmin admin = new HBaseAdmin(new Configuration(TEST_UTIL.getConfiguration()));
168     testAdminSanity(admin, tableName);
169     HTable table = new HTable(new Configuration(TEST_UTIL.getConfiguration()), tableName);
170     testTableSanity(table, tableName);
171   }
172 
173   private void testSanity(HConnection conn, ExecutorService pool) throws Exception {
174     String tableName = "test"+System.currentTimeMillis();
175     HBaseAdmin admin = new HBaseAdmin(persistentConnection);
176     testAdminSanity(admin, tableName);
177     HTable table = new HTable(Bytes.toBytes(tableName), persistentConnection, pool);
178     testTableSanity(table, tableName);
179 
180   }
181   private void testAdminSanity(HBaseAdmin admin, String tableName) throws Exception {
182     HTableDescriptor desc = new HTableDescriptor(tableName);
183     HColumnDescriptor family = new HColumnDescriptor("fam");
184     desc.addFamily(family);
185     LOG.info("Creating table " + tableName);
186     admin.createTable(desc);
187   }
188 
189   private void testTableSanity(HTable table, String tableName) throws Exception {
190     Put put = new Put(Bytes.toBytes("testrow"));
191     put.add(Bytes.toBytes("fam"), Bytes.toBytes("col"), Bytes.toBytes("testdata"));
192     LOG.info("Putting table " + tableName);
193     table.put(put);
194     table.close();
195   }
196 
197   @Test
198   public void testMultipleZK() {
199     try {
200       HTable localMeta =
201         new HTable(new Configuration(TEST_UTIL.getConfiguration()), HConstants.META_TABLE_NAME);
202       Configuration otherConf = new Configuration(TEST_UTIL.getConfiguration());
203       otherConf.set(HConstants.ZOOKEEPER_QUORUM, "127.0.0.1");
204       HTable ipMeta = new HTable(otherConf, HConstants.META_TABLE_NAME);
205 
206       // dummy, just to open the connection
207       localMeta.exists(new Get(HConstants.LAST_ROW));
208       ipMeta.exists(new Get(HConstants.LAST_ROW));
209 
210       // make sure they aren't the same
211       assertFalse(HConnectionManager.getConnection(localMeta.getConfiguration()).getZooKeeperWatcher()
212           == HConnectionManager.getConnection(otherConf).getZooKeeperWatcher());
213       assertFalse(HConnectionManager.getConnection(localMeta.getConfiguration())
214           .getZooKeeperWatcher().getQuorum().equals(HConnectionManager
215               .getConnection(otherConf).getZooKeeperWatcher().getQuorum()));
216       localMeta.close();
217       ipMeta.close();
218     } catch (Exception e) {
219       e.printStackTrace();
220       fail();
221     }
222   }
223 
224   /**
225    * Create a znode with data
226    * @throws Exception
227    */
228   @Test
229   public void testCreateWithParents() throws Exception {
230     ZooKeeperWatcher zkw =
231         new ZooKeeperWatcher(new Configuration(TEST_UTIL.getConfiguration()),
232           TestZooKeeper.class.getName(), null);
233     byte[] expectedData = new byte[] { 1, 2, 3 };
234     ZKUtil.createWithParents(zkw, "/l1/l2/l3/l4/testCreateWithParents", expectedData);
235     byte[] data = ZKUtil.getData(zkw, "/l1/l2/l3/l4/testCreateWithParents");
236     assertEquals(Bytes.equals(expectedData, data), true);
237     ZKUtil.deleteNodeRecursively(zkw, "/l1");
238 
239     ZKUtil.createWithParents(zkw, "/testCreateWithParents", expectedData);
240     data = ZKUtil.getData(zkw, "/testCreateWithParents");
241     assertEquals(Bytes.equals(expectedData, data), true);
242     ZKUtil.deleteNodeRecursively(zkw, "/testCreateWithParents");
243   }
244 
245   /**
246    * Create a bunch of znodes in a hierarchy, try deleting one that has childs
247    * (it will fail), then delete it recursively, then delete the last znode
248    * @throws Exception
249    */
250   @Test
251   public void testZNodeDeletes() throws Exception {
252     ZooKeeperWatcher zkw = new ZooKeeperWatcher(
253       new Configuration(TEST_UTIL.getConfiguration()), 
254       TestZooKeeper.class.getName(), null);
255     ZKUtil.createWithParents(zkw, "/l1/l2/l3/l4");
256     try {
257       ZKUtil.deleteNode(zkw, "/l1/l2");
258       fail("We should not be able to delete if znode has childs");
259     } catch (KeeperException ex) {
260       assertNotNull(ZKUtil.getDataNoWatch(zkw, "/l1/l2/l3/l4", null));
261     }
262     ZKUtil.deleteNodeRecursively(zkw, "/l1/l2");
263     // make sure it really is deleted
264     assertNull(ZKUtil.getDataNoWatch(zkw, "/l1/l2/l3/l4", null));
265 
266     // do the same delete again and make sure it doesn't crash
267     ZKUtil.deleteNodeRecursively(zkw, "/l1/l2");
268 
269     ZKUtil.deleteNode(zkw, "/l1");
270     assertNull(ZKUtil.getDataNoWatch(zkw, "/l1/l2", null));
271   }
272 
273   @Test
274   public void testClusterKey() throws Exception {
275     testKey("server", "2181", "hbase");
276     testKey("server1,server2,server3", "2181", "hbase");
277     try {
278       ZKUtil.transformClusterKey("2181:hbase");
279     } catch (IOException ex) {
280       // OK
281     }
282   }
283 
284   /**
285    * Test with a connection that existed before the cluster was started
286    */
287   @Test
288   public void testPersistentConnection() throws Exception {
289     ExecutorService pool = new ThreadPoolExecutor(1, 10, 10, TimeUnit.SECONDS,
290         new SynchronousQueue<Runnable>());
291     testSanity(persistentConnection, pool);
292   }
293 
294   private void testKey(String ensemble, String port, String znode)
295       throws IOException {
296     Configuration conf = new Configuration();
297     String key = ensemble+":"+port+":"+znode;
298     String[] parts = ZKUtil.transformClusterKey(key);
299     assertEquals(ensemble, parts[0]);
300     assertEquals(port, parts[1]);
301     assertEquals(znode, parts[2]);
302     ZKUtil.applyClusterKeyToConf(conf, key);
303     assertEquals(parts[0], conf.get(HConstants.ZOOKEEPER_QUORUM));
304     assertEquals(parts[1], conf.get(HConstants.ZOOKEEPER_CLIENT_PORT));
305     assertEquals(parts[2], conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT));
306     String reconstructedKey = ZKUtil.getZooKeeperClusterKey(conf);
307     assertEquals(key, reconstructedKey);
308   }
309 
310   /**
311    * A test for HBASE-3238
312    * @throws IOException A connection attempt to zk failed
313    * @throws InterruptedException One of the non ZKUtil actions was interrupted
314    * @throws KeeperException Any of the zookeeper connections had a
315    * KeeperException
316    */
317   @Test
318   public void testCreateSilentIsReallySilent() throws InterruptedException,
319       KeeperException, IOException {
320     Configuration c = TEST_UTIL.getConfiguration();
321 
322     String aclZnode = "/aclRoot";
323     String quorumServers = ZKConfig.getZKQuorumServersString(c);
324     int sessionTimeout = 5 * 1000; // 5 seconds
325     ZooKeeper zk = new ZooKeeper(quorumServers, sessionTimeout, EmptyWatcher.instance);
326     zk.addAuthInfo("digest", "hbase:rox".getBytes());
327 
328     // Assumes the  root of the ZooKeeper space is writable as it creates a node
329     // wherever the cluster home is defined.
330     ZooKeeperWatcher zk2 = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
331       "testMasterAddressManagerFromZK", null);
332 
333     // I set this acl after the attempted creation of the cluster home node.
334     // Add retries in case of retryable zk exceptions.
335     while (true) {
336       try {
337         zk.setACL("/", ZooDefs.Ids.CREATOR_ALL_ACL, -1);
338         break;
339       } catch (KeeperException e) {
340         switch (e.code()) {
341           case CONNECTIONLOSS:
342           case SESSIONEXPIRED:
343           case OPERATIONTIMEOUT:
344             LOG.warn("Possibly transient ZooKeeper exception: " + e);
345             Threads.sleep(100);
346             break;
347          default:
348             throw e;
349         }
350       }
351     }
352 
353     while (true) {
354       try {
355         zk.create(aclZnode, null, ZooDefs.Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT);
356         break;
357       } catch (KeeperException e) {
358         switch (e.code()) {
359           case CONNECTIONLOSS:
360           case SESSIONEXPIRED:
361           case OPERATIONTIMEOUT:
362             LOG.warn("Possibly transient ZooKeeper exception: " + e);
363             Threads.sleep(100);
364             break;
365          default:
366             throw e;
367         }
368       }
369     }
370     ZKUtil.createAndFailSilent(zk2, aclZnode);
371 
372     // reset /'s ACL for tests that follow
373     zk = new ZooKeeper(quorumServers, sessionTimeout, EmptyWatcher.instance);
374     zk.addAuthInfo("digest", "hbase:rox".getBytes());
375     zk.setACL("/", ZooDefs.Ids.OPEN_ACL_UNSAFE, -1);
376     zk.close();
377  }
378   
379   /**
380    * Test should not fail with NPE when getChildDataAndWatchForNewChildren
381    * invoked with wrongNode
382    */
383   @Test
384   public void testGetChildDataAndWatchForNewChildrenShouldNotThrowNPE()
385       throws Exception {
386     ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
387         "testGetChildDataAndWatchForNewChildrenShouldNotThrowNPE", null);
388     ZKUtil.getChildDataAndWatchForNewChildren(zkw, "/wrongNode");
389   }
390 
391   /**
392    * Master recovery when the znode already exists. Internally, this
393    *  test differs from {@link #testMasterSessionExpired} because here
394    *  the master znode will exist in ZK.
395    */
396   @Test(timeout=60000)
397   public void testMasterZKSessionRecoveryFailure() throws Exception {
398     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
399     HMaster m = cluster.getMaster();
400     m.abort("Test recovery from zk session expired",
401       new KeeperException.SessionExpiredException());
402     assertFalse(m.isStopped());
403     testSanity();
404   }
405 
406   @org.junit.Rule
407   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
408     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
409 }
410