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.apache.hadoop.hbase.HConstants.DEFAULT_ZK_SESSION_TIMEOUT; 021import static org.apache.hadoop.hbase.HConstants.ZK_SESSION_TIMEOUT; 022 023import java.io.Closeable; 024import java.io.IOException; 025import java.util.Arrays; 026import java.util.EnumSet; 027import java.util.concurrent.CompletableFuture; 028import java.util.concurrent.DelayQueue; 029import java.util.concurrent.Delayed; 030import java.util.concurrent.TimeUnit; 031import java.util.concurrent.atomic.AtomicBoolean; 032import org.apache.hadoop.conf.Configuration; 033import org.apache.hadoop.hbase.util.Threads; 034import org.apache.yetus.audience.InterfaceAudience; 035import org.apache.zookeeper.KeeperException; 036import org.apache.zookeeper.KeeperException.Code; 037import org.apache.zookeeper.ZooKeeper; 038import org.apache.zookeeper.data.Stat; 039import org.slf4j.Logger; 040import org.slf4j.LoggerFactory; 041 042import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; 043 044/** 045 * A very simple read only zookeeper implementation without watcher support. 046 */ 047@InterfaceAudience.Private 048public final class ReadOnlyZKClient implements Closeable { 049 050 private static final Logger LOG = LoggerFactory.getLogger(ReadOnlyZKClient.class); 051 052 public static final String RECOVERY_RETRY = "zookeeper.recovery.retry"; 053 054 private static final int DEFAULT_RECOVERY_RETRY = 30; 055 056 public static final String RECOVERY_RETRY_INTERVAL_MILLIS = 057 "zookeeper.recovery.retry.intervalmill"; 058 059 private static final int DEFAULT_RECOVERY_RETRY_INTERVAL_MILLIS = 1000; 060 061 public static final String KEEPALIVE_MILLIS = "zookeeper.keep-alive.time"; 062 063 private static final int DEFAULT_KEEPALIVE_MILLIS = 60000; 064 065 private static final EnumSet<Code> FAIL_FAST_CODES = EnumSet.of(Code.NOAUTH, Code.AUTHFAILED); 066 067 private final String connectString; 068 069 private final int sessionTimeoutMs; 070 071 private final int maxRetries; 072 073 private final int retryIntervalMs; 074 075 private final int keepAliveTimeMs; 076 077 private static abstract class Task implements Delayed { 078 079 protected long time = System.nanoTime(); 080 081 public boolean needZk() { 082 return false; 083 } 084 085 public void exec(ZooKeeper zk) { 086 } 087 088 public void connectFailed(IOException e) { 089 } 090 091 public void closed(IOException e) { 092 } 093 094 @Override 095 public int compareTo(Delayed o) { 096 Task that = (Task) o; 097 int c = Long.compare(time, that.time); 098 if (c != 0) { 099 return c; 100 } 101 return Integer.compare(System.identityHashCode(this), System.identityHashCode(that)); 102 } 103 104 @Override 105 public long getDelay(TimeUnit unit) { 106 return unit.convert(time - System.nanoTime(), TimeUnit.NANOSECONDS); 107 } 108 } 109 110 private static final Task CLOSE = new Task() { 111 }; 112 113 private final DelayQueue<Task> tasks = new DelayQueue<>(); 114 115 private final AtomicBoolean closed = new AtomicBoolean(false); 116 117 @VisibleForTesting 118 ZooKeeper zookeeper; 119 120 private int pendingRequests = 0; 121 122 private String getId() { 123 return String.format("0x%08x", System.identityHashCode(this)); 124 } 125 126 public ReadOnlyZKClient(Configuration conf) { 127 // We might use a different ZK for client access 128 String clientZkQuorumServers = ZKConfig.getClientZKQuorumServersString(conf); 129 if (clientZkQuorumServers != null) { 130 this.connectString = clientZkQuorumServers; 131 } else { 132 this.connectString = ZKConfig.getZKQuorumServersString(conf); 133 } 134 this.sessionTimeoutMs = conf.getInt(ZK_SESSION_TIMEOUT, DEFAULT_ZK_SESSION_TIMEOUT); 135 this.maxRetries = conf.getInt(RECOVERY_RETRY, DEFAULT_RECOVERY_RETRY); 136 this.retryIntervalMs = 137 conf.getInt(RECOVERY_RETRY_INTERVAL_MILLIS, DEFAULT_RECOVERY_RETRY_INTERVAL_MILLIS); 138 this.keepAliveTimeMs = conf.getInt(KEEPALIVE_MILLIS, DEFAULT_KEEPALIVE_MILLIS); 139 LOG.debug( 140 "Connect {} to {} with session timeout={}ms, retries {}, " + 141 "retry interval {}ms, keepAlive={}ms", 142 getId(), connectString, sessionTimeoutMs, maxRetries, retryIntervalMs, keepAliveTimeMs); 143 Threads.setDaemonThreadRunning(new Thread(this::run), 144 "ReadOnlyZKClient-" + connectString + "@" + getId()); 145 } 146 147 private abstract class ZKTask<T> extends Task { 148 149 protected final String path; 150 151 private final CompletableFuture<T> future; 152 153 private final String operationType; 154 155 private int retries; 156 157 protected ZKTask(String path, CompletableFuture<T> future, String operationType) { 158 this.path = path; 159 this.future = future; 160 this.operationType = operationType; 161 } 162 163 protected final void onComplete(ZooKeeper zk, int rc, T ret, boolean errorIfNoNode) { 164 tasks.add(new Task() { 165 166 @Override 167 public void exec(ZooKeeper alwaysNull) { 168 pendingRequests--; 169 Code code = Code.get(rc); 170 if (code == Code.OK) { 171 future.complete(ret); 172 } else if (code == Code.NONODE) { 173 if (errorIfNoNode) { 174 future.completeExceptionally(KeeperException.create(code, path)); 175 } else { 176 future.complete(ret); 177 } 178 } else if (FAIL_FAST_CODES.contains(code)) { 179 future.completeExceptionally(KeeperException.create(code, path)); 180 } else { 181 if (code == Code.SESSIONEXPIRED) { 182 LOG.warn("{} to {} session expired, close and reconnect", getId(), connectString); 183 try { 184 zk.close(); 185 } catch (InterruptedException e) { 186 } 187 } 188 if (ZKTask.this.delay(retryIntervalMs, maxRetries)) { 189 LOG.warn("{} to {} failed for {} of {}, code = {}, retries = {}", getId(), 190 connectString, operationType, path, code, ZKTask.this.retries); 191 tasks.add(ZKTask.this); 192 } else { 193 LOG.warn("{} to {} failed for {} of {}, code = {}, retries = {}, give up", getId(), 194 connectString, operationType, path, code, ZKTask.this.retries); 195 future.completeExceptionally(KeeperException.create(code, path)); 196 } 197 } 198 } 199 200 @Override 201 public void closed(IOException e) { 202 // It may happen that a request is succeeded and the onComplete has been called and pushed 203 // us into the task queue, but before we get called a close is called and here we will 204 // fail the request, although it is succeeded actually. 205 // This is not a perfect solution but anyway, it is better than hang the requests for 206 // ever, and also acceptable as if you close the zk client before actually getting the 207 // response then a failure is always possible. 208 future.completeExceptionally(e); 209 } 210 }); 211 } 212 213 @Override 214 public boolean needZk() { 215 return true; 216 } 217 218 protected abstract void doExec(ZooKeeper zk); 219 220 @Override 221 public final void exec(ZooKeeper zk) { 222 pendingRequests++; 223 doExec(zk); 224 } 225 226 public boolean delay(long intervalMs, int maxRetries) { 227 if (retries >= maxRetries) { 228 return false; 229 } 230 retries++; 231 time = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(intervalMs); 232 return true; 233 } 234 235 @Override 236 public void connectFailed(IOException e) { 237 if (delay(retryIntervalMs, maxRetries)) { 238 LOG.warn("{} to {} failed to connect to zk fo {} of {}, retries = {}", getId(), 239 connectString, operationType, path, retries, e); 240 tasks.add(this); 241 } else { 242 LOG.warn("{} to {} failed to connect to zk fo {} of {}, retries = {}, give up", getId(), 243 connectString, operationType, path, retries, e); 244 future.completeExceptionally(e); 245 } 246 } 247 248 @Override 249 public void closed(IOException e) { 250 future.completeExceptionally(e); 251 } 252 } 253 254 private static <T> CompletableFuture<T> failed(Throwable e) { 255 CompletableFuture<T> future = new CompletableFuture<>(); 256 future.completeExceptionally(e); 257 return future; 258 } 259 260 public CompletableFuture<byte[]> get(String path) { 261 if (closed.get()) { 262 return failed(new IOException("Client already closed")); 263 } 264 CompletableFuture<byte[]> future = new CompletableFuture<>(); 265 tasks.add(new ZKTask<byte[]>(path, future, "get") { 266 267 @Override 268 protected void doExec(ZooKeeper zk) { 269 zk.getData(path, false, 270 (rc, path, ctx, data, stat) -> onComplete(zk, rc, data, true), null); 271 } 272 }); 273 return future; 274 } 275 276 public CompletableFuture<Stat> exists(String path) { 277 if (closed.get()) { 278 return failed(new IOException("Client already closed")); 279 } 280 CompletableFuture<Stat> future = new CompletableFuture<>(); 281 tasks.add(new ZKTask<Stat>(path, future, "exists") { 282 283 @Override 284 protected void doExec(ZooKeeper zk) { 285 zk.exists(path, false, (rc, path, ctx, stat) -> onComplete(zk, rc, stat, false), null); 286 } 287 }); 288 return future; 289 } 290 291 private void closeZk() { 292 if (zookeeper != null) { 293 try { 294 zookeeper.close(); 295 } catch (InterruptedException e) { 296 } 297 zookeeper = null; 298 } 299 } 300 301 private ZooKeeper getZk() throws IOException { 302 // may be closed when session expired 303 if (zookeeper == null || !zookeeper.getState().isAlive()) { 304 zookeeper = new ZooKeeper(connectString, sessionTimeoutMs, e -> {}); 305 } 306 return zookeeper; 307 } 308 309 private void run() { 310 for (;;) { 311 Task task; 312 try { 313 task = tasks.poll(keepAliveTimeMs, TimeUnit.MILLISECONDS); 314 } catch (InterruptedException e) { 315 continue; 316 } 317 if (task == CLOSE) { 318 break; 319 } 320 if (task == null) { 321 if (pendingRequests == 0) { 322 LOG.trace("{} to {} inactive for {}ms; closing (Will reconnect when new requests)", 323 getId(), connectString, keepAliveTimeMs); 324 closeZk(); 325 } 326 continue; 327 } 328 if (!task.needZk()) { 329 task.exec(null); 330 } else { 331 ZooKeeper zk; 332 try { 333 zk = getZk(); 334 } catch (IOException e) { 335 task.connectFailed(e); 336 continue; 337 } 338 task.exec(zk); 339 } 340 } 341 closeZk(); 342 IOException error = new IOException("Client already closed"); 343 Arrays.stream(tasks.toArray(new Task[0])).forEach(t -> t.closed(error)); 344 tasks.clear(); 345 } 346 347 @Override 348 public void close() { 349 if (closed.compareAndSet(false, true)) { 350 LOG.debug("Close zookeeper connection {} to {}", getId(), connectString); 351 tasks.add(CLOSE); 352 } 353 } 354 355 @VisibleForTesting 356 public String getConnectString() { 357 return connectString; 358 } 359}