1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.zookeeper;
19
20 import static org.junit.Assert.assertEquals;
21 import static org.junit.Assert.assertTrue;
22
23 import com.google.common.base.Throwables;
24
25 import java.io.IOException;
26 import java.util.concurrent.Callable;
27 import java.util.concurrent.ExecutionException;
28 import java.util.concurrent.ExecutorService;
29 import java.util.concurrent.Executors;
30 import java.util.concurrent.Future;
31 import java.util.concurrent.TimeUnit;
32 import java.util.concurrent.TimeoutException;
33
34 import org.apache.hadoop.conf.Configuration;
35 import org.apache.hadoop.hbase.HBaseTestingUtility;
36 import org.apache.hadoop.hbase.TableName;
37 import org.apache.hadoop.hbase.client.Connection;
38 import org.apache.hadoop.hbase.client.ConnectionFactory;
39 import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
40 import org.apache.hadoop.hbase.client.Table;
41 import org.apache.hadoop.hbase.testclassification.MediumTests;
42 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZooKeeperFactory;
43 import org.apache.zookeeper.Watcher;
44 import org.apache.zookeeper.ZooKeeper;
45 import org.junit.AfterClass;
46 import org.junit.Assert;
47 import org.junit.BeforeClass;
48 import org.junit.Test;
49 import org.junit.experimental.categories.Category;
50 import org.slf4j.Logger;
51 import org.slf4j.LoggerFactory;
52
53 @Category(MediumTests.class)
54 public class TestZKAuthFailedRecovery {
55 final Logger LOG = LoggerFactory.getLogger(getClass());
56 protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
57
58 public static class AuthFailingZooKeeperFactory implements ZooKeeperFactory {
59 @Override
60 public RecoverableZooKeeper create(String quorumServers, int sessionTimeout, Watcher watcher,
61 int maxRetries, int retryIntervalMillis, int maxSleepTime, String identifier,
62 int authFailedRetries, int authFailedPause, int multiMaxSize) throws IOException {
63 return new AuthFailingRecoverableZooKeeper(quorumServers, sessionTimeout, watcher, maxRetries,
64 retryIntervalMillis, maxSleepTime, identifier, authFailedRetries, authFailedPause,
65 multiMaxSize);
66 }
67 }
68
69 private static final int FAILURES_BEFORE_SUCCESS = 3;
70
71 public static class SelfHealingZooKeeperFactory implements ZooKeeperFactory {
72 @Override
73 public RecoverableZooKeeper create(String quorumServers, int sessionTimeout, Watcher watcher,
74 int maxRetries, int retryIntervalMillis, int maxSleepTime, String identifier,
75 int authFailedRetries, int authFailedPause, int multiMaxSize) throws IOException {
76 return new SelfHealingRecoverableZooKeeper(quorumServers, sessionTimeout, watcher, maxRetries,
77 retryIntervalMillis, maxSleepTime, identifier, authFailedRetries, authFailedPause,
78 FAILURES_BEFORE_SUCCESS, multiMaxSize);
79 }
80 }
81
82 @BeforeClass
83 public static void setUpBeforeClass() throws Exception {
84 Configuration conf = TEST_UTIL.getConfiguration();
85 conf.setBoolean("hbase.table.sanity.checks", true);
86 TEST_UTIL.startMiniCluster(1);
87 }
88
89 @AfterClass
90 public static void tearDownAfterClass() throws Exception {
91 TEST_UTIL.shutdownMiniCluster();
92 }
93
94 @Test
95 public void testFaultyClientZK() throws Exception {
96 Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
97 conf.setClass("zookeeper.factory.class", AuthFailingZooKeeperFactory.class,
98 ZooKeeperFactory.class);
99 LOG.debug("Reading meta first time");
100 final Connection conn = ConnectionFactory.createConnection(conf);
101 try (Table t = conn.getTable(TableName.valueOf("hbase:meta"))) {
102 LOG.info(TEST_UTIL.countRows(t) + " rows in meta");
103 }
104
105 ZooKeeper zk = HConnectionTestingUtility.unwrapZK(conn).checkZk();
106 assertEquals(AuthFailingZooKeeper.class, zk.getClass());
107
108 ((AuthFailingZooKeeper) zk).triggerAuthFailed();
109
110 HConnectionTestingUtility.clearRegionCache(conn);
111
112
113 ExecutorService svc = Executors.newSingleThreadExecutor();
114 Future<Boolean> res = svc.submit(new Callable<Boolean>() {
115 public Boolean call() {
116 LOG.debug("Reading meta after clearing the Region caches");
117 try (Table t = conn.getTable(TableName.valueOf("hbase:meta"))) {
118 LOG.info(TEST_UTIL.countRows(t) + " rows in meta");
119 return true;
120 } catch (Exception e) {
121 LOG.error("Failed to read hbase:meta", e);
122 return false;
123 }
124 }
125 });
126
127
128
129 try {
130 res.get(30, TimeUnit.SECONDS);
131 } catch (ExecutionException e) {
132 LOG.error("Failed to execute task", e);
133 Assert.fail("Failed to recover from AUTH_FAILED state in zookeeper client");
134 } catch (TimeoutException e) {
135 LOG.error("Task timed out instead of recovering", e);
136 Assert.fail("Failed to recover from AUTH_FAILED state in zookeeper client");
137 }
138 }
139
140 @Test
141 public void eventuallyRecoveringZKClient() throws Exception {
142 Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
143 conf.setClass("zookeeper.factory.class", SelfHealingZooKeeperFactory.class,
144 ZooKeeperFactory.class);
145
146 conf.setInt(ZKUtil.AUTH_FAILED_RETRIES_KEY, FAILURES_BEFORE_SUCCESS + 1);
147
148 conf.setInt(ZKUtil.AUTH_FAILED_PAUSE_KEY, 0);
149
150 final Connection conn = ConnectionFactory.createConnection(conf);
151
152
153 RecoverableZooKeeper recoverableZk = HConnectionTestingUtility.unwrapZK(conn);
154 assertEquals(SelfHealingRecoverableZooKeeper.class, recoverableZk.getClass());
155 ZooKeeper zk = recoverableZk.checkZk();
156 assertEquals(AuthFailingZooKeeper.class, zk.getClass());
157
158 try (Table t = conn.getTable(TableName.valueOf("hbase:meta"))) {
159 LOG.info(TEST_UTIL.countRows(t) + " rows in meta");
160 }
161 }
162
163 @Test
164 public void retriesExceededOnAuthFailed() throws Exception {
165 Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
166 conf.setClass("zookeeper.factory.class", SelfHealingZooKeeperFactory.class,
167 ZooKeeperFactory.class);
168
169 conf.setInt(ZKUtil.AUTH_FAILED_RETRIES_KEY, FAILURES_BEFORE_SUCCESS - 1);
170
171 conf.setInt(ZKUtil.AUTH_FAILED_PAUSE_KEY, 0);
172
173 Connection conn = null;
174 try {
175 conn = ConnectionFactory.createConnection(conf);
176 } catch (Exception e) {
177
178 LOG.info("Caught exception, validating it", e);
179 Throwable rootCause = Throwables.getRootCause(e);
180 assertEquals(RuntimeException.class, rootCause.getClass());
181 assertTrue("Expected the exception to contain the text 'AUTH_FAILED'",
182 rootCause.getMessage().contains("AUTH_FAILED"));
183 } finally {
184 if (conn != null) {
185 conn.close();
186 }
187 }
188 }
189 }