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.client;
019
020import static org.apache.hadoop.hbase.HConstants.STATUS_PUBLISHED;
021import static org.apache.hadoop.hbase.HConstants.STATUS_PUBLISHED_DEFAULT;
022import static org.apache.hadoop.hbase.client.ClusterStatusListener.DEFAULT_STATUS_LISTENER_CLASS;
023import static org.apache.hadoop.hbase.client.ClusterStatusListener.STATUS_LISTENER_CLASS;
024import static org.apache.hadoop.hbase.client.ConnectionUtils.NO_NONCE_GENERATOR;
025import static org.apache.hadoop.hbase.client.ConnectionUtils.getStubKey;
026import static org.apache.hadoop.hbase.client.MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY;
027import static org.apache.hadoop.hbase.client.NonceGenerator.CLIENT_NONCES_ENABLED_KEY;
028import static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.SERVER_NAME_KEY;
029import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
030
031import io.opentelemetry.api.trace.Span;
032import java.io.IOException;
033import java.net.SocketAddress;
034import java.util.Optional;
035import java.util.concurrent.CompletableFuture;
036import java.util.concurrent.ConcurrentHashMap;
037import java.util.concurrent.ConcurrentMap;
038import java.util.concurrent.ExecutionException;
039import java.util.concurrent.ExecutorService;
040import java.util.concurrent.TimeUnit;
041import java.util.concurrent.atomic.AtomicBoolean;
042import java.util.concurrent.atomic.AtomicReference;
043import org.apache.commons.io.IOUtils;
044import org.apache.hadoop.conf.Configuration;
045import org.apache.hadoop.hbase.AuthUtil;
046import org.apache.hadoop.hbase.ChoreService;
047import org.apache.hadoop.hbase.MasterNotRunningException;
048import org.apache.hadoop.hbase.ServerName;
049import org.apache.hadoop.hbase.TableName;
050import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
051import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicyFactory;
052import org.apache.hadoop.hbase.ipc.RpcClient;
053import org.apache.hadoop.hbase.ipc.RpcClientFactory;
054import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
055import org.apache.hadoop.hbase.security.User;
056import org.apache.hadoop.hbase.trace.TraceUtil;
057import org.apache.hadoop.hbase.util.ConcurrentMapUtils;
058import org.apache.hadoop.hbase.util.Threads;
059import org.apache.hadoop.security.UserGroupInformation;
060import org.apache.yetus.audience.InterfaceAudience;
061import org.slf4j.Logger;
062import org.slf4j.LoggerFactory;
063
064import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
065import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
066
067import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
068import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
069import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
070import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
071
072/**
073 * The implementation of AsyncConnection.
074 */
075@InterfaceAudience.Private
076public class AsyncConnectionImpl implements AsyncConnection {
077
078  private static final Logger LOG = LoggerFactory.getLogger(AsyncConnectionImpl.class);
079
080  static final HashedWheelTimer RETRY_TIMER = new HashedWheelTimer(
081    new ThreadFactoryBuilder().setNameFormat("Async-Client-Retry-Timer-pool-%d").setDaemon(true)
082      .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build(),
083    10, TimeUnit.MILLISECONDS);
084
085  private final Configuration conf;
086
087  final AsyncConnectionConfiguration connConf;
088
089  protected final User user;
090
091  final ConnectionRegistry registry;
092
093  protected final int rpcTimeout;
094
095  protected final RpcClient rpcClient;
096
097  final RpcControllerFactory rpcControllerFactory;
098
099  private final AsyncRegionLocator locator;
100
101  final AsyncRpcRetryingCallerFactory callerFactory;
102
103  private final NonceGenerator nonceGenerator;
104
105  private final ConcurrentMap<String, ClientService.Interface> rsStubs = new ConcurrentHashMap<>();
106  private final ConcurrentMap<String, AdminService.Interface> adminStubs =
107    new ConcurrentHashMap<>();
108
109  private final AtomicReference<MasterService.Interface> masterStub = new AtomicReference<>();
110
111  private final AtomicReference<CompletableFuture<MasterService.Interface>> masterStubMakeFuture =
112    new AtomicReference<>();
113
114  private final Optional<ServerStatisticTracker> stats;
115  private final ClientBackoffPolicy backoffPolicy;
116
117  private ChoreService choreService;
118
119  private final AtomicBoolean closed = new AtomicBoolean(false);
120
121  private final Optional<MetricsConnection> metrics;
122
123  private final ClusterStatusListener clusterStatusListener;
124
125  private volatile ConnectionOverAsyncConnection conn;
126
127  public AsyncConnectionImpl(Configuration conf, ConnectionRegistry registry, String clusterId,
128    SocketAddress localAddress, User user) {
129    this.conf = conf;
130    this.user = user;
131
132    if (user.isLoginFromKeytab()) {
133      spawnRenewalChore(user.getUGI());
134    }
135    this.connConf = new AsyncConnectionConfiguration(conf);
136    this.registry = registry;
137    if (conf.getBoolean(CLIENT_SIDE_METRICS_ENABLED_KEY, false)) {
138      String scope = MetricsConnection.getScope(conf, clusterId, this);
139      this.metrics = Optional.of(new MetricsConnection(scope, () -> null, () -> null));
140    } else {
141      this.metrics = Optional.empty();
142    }
143    this.rpcClient =
144      RpcClientFactory.createClient(conf, clusterId, localAddress, metrics.orElse(null));
145    this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
146    this.rpcTimeout =
147      (int) Math.min(Integer.MAX_VALUE, TimeUnit.NANOSECONDS.toMillis(connConf.getRpcTimeoutNs()));
148    this.locator = new AsyncRegionLocator(this, RETRY_TIMER);
149    this.callerFactory = new AsyncRpcRetryingCallerFactory(this, RETRY_TIMER);
150    if (conf.getBoolean(CLIENT_NONCES_ENABLED_KEY, true)) {
151      nonceGenerator = PerClientRandomNonceGenerator.get();
152    } else {
153      nonceGenerator = NO_NONCE_GENERATOR;
154    }
155    this.stats = Optional.ofNullable(ServerStatisticTracker.create(conf));
156    this.backoffPolicy = ClientBackoffPolicyFactory.create(conf);
157    ClusterStatusListener listener = null;
158    if (conf.getBoolean(STATUS_PUBLISHED, STATUS_PUBLISHED_DEFAULT)) {
159      // TODO: this maybe a blocking operation, better to create it outside the constructor and pass
160      // it in, just like clusterId. Not a big problem for now as the default value is false.
161      Class<? extends ClusterStatusListener.Listener> listenerClass = conf.getClass(
162        STATUS_LISTENER_CLASS, DEFAULT_STATUS_LISTENER_CLASS, ClusterStatusListener.Listener.class);
163      if (listenerClass == null) {
164        LOG.warn("{} is true, but {} is not set", STATUS_PUBLISHED, STATUS_LISTENER_CLASS);
165      } else {
166        try {
167          listener = new ClusterStatusListener(new ClusterStatusListener.DeadServerHandler() {
168            @Override
169            public void newDead(ServerName sn) {
170              locator.clearCache(sn);
171              rpcClient.cancelConnections(sn);
172            }
173          }, conf, listenerClass);
174        } catch (IOException e) {
175          LOG.warn("Failed create of ClusterStatusListener, not a critical, ignoring...", e);
176        }
177      }
178    }
179    this.clusterStatusListener = listener;
180  }
181
182  private void spawnRenewalChore(final UserGroupInformation user) {
183    ChoreService service = getChoreService();
184    service.scheduleChore(AuthUtil.getAuthRenewalChore(user, conf));
185  }
186
187  /**
188   * If choreService has not been created yet, create the ChoreService. n
189   */
190  synchronized ChoreService getChoreService() {
191    if (isClosed()) {
192      throw new IllegalStateException("connection is already closed");
193    }
194    if (choreService == null) {
195      choreService = new ChoreService("AsyncConn Chore Service");
196    }
197    return choreService;
198  }
199
200  public User getUser() {
201    return user;
202  }
203
204  public ConnectionRegistry getConnectionRegistry() {
205    return registry;
206  }
207
208  @Override
209  public Configuration getConfiguration() {
210    return conf;
211  }
212
213  @Override
214  public boolean isClosed() {
215    return closed.get();
216  }
217
218  @Override
219  public void close() {
220    TraceUtil.trace(() -> {
221      if (!closed.compareAndSet(false, true)) {
222        return;
223      }
224      LOG.info("Connection has been closed by {}.", Thread.currentThread().getName());
225      if (LOG.isDebugEnabled()) {
226        logCallStack(Thread.currentThread().getStackTrace());
227      }
228      IOUtils.closeQuietly(clusterStatusListener,
229        e -> LOG.warn("failed to close clusterStatusListener", e));
230      IOUtils.closeQuietly(rpcClient, e -> LOG.warn("failed to close rpcClient", e));
231      IOUtils.closeQuietly(registry, e -> LOG.warn("failed to close registry", e));
232      synchronized (this) {
233        if (choreService != null) {
234          choreService.shutdown();
235          choreService = null;
236        }
237      }
238      metrics.ifPresent(MetricsConnection::shutdown);
239      ConnectionOverAsyncConnection c = this.conn;
240      if (c != null) {
241        c.closePool();
242      }
243    }, "AsyncConnection.close");
244  }
245
246  private void logCallStack(StackTraceElement[] stackTraceElements) {
247    StringBuilder stackBuilder = new StringBuilder("Call stack:");
248    for (StackTraceElement element : stackTraceElements) {
249      stackBuilder.append("\n    at ");
250      stackBuilder.append(element);
251    }
252    stackBuilder.append("\n");
253    LOG.debug(stackBuilder.toString());
254  }
255
256  @Override
257  public AsyncTableRegionLocator getRegionLocator(TableName tableName) {
258    return new AsyncTableRegionLocatorImpl(tableName, this);
259  }
260
261  @Override
262  public void clearRegionLocationCache() {
263    locator.clearCache();
264  }
265
266  // we will override this method for testing retry caller, so do not remove this method.
267  AsyncRegionLocator getLocator() {
268    return locator;
269  }
270
271  // ditto
272  NonceGenerator getNonceGenerator() {
273    return nonceGenerator;
274  }
275
276  private ClientService.Interface createRegionServerStub(ServerName serverName) throws IOException {
277    return ClientService.newStub(rpcClient.createRpcChannel(serverName, user, rpcTimeout));
278  }
279
280  ClientService.Interface getRegionServerStub(ServerName serverName) throws IOException {
281    return ConcurrentMapUtils.computeIfAbsentEx(rsStubs,
282      getStubKey(ClientService.getDescriptor().getName(), serverName),
283      () -> createRegionServerStub(serverName));
284  }
285
286  private MasterService.Interface createMasterStub(ServerName serverName) throws IOException {
287    return MasterService.newStub(rpcClient.createRpcChannel(serverName, user, rpcTimeout));
288  }
289
290  private AdminService.Interface createAdminServerStub(ServerName serverName) throws IOException {
291    return AdminService.newStub(rpcClient.createRpcChannel(serverName, user, rpcTimeout));
292  }
293
294  AdminService.Interface getAdminStub(ServerName serverName) throws IOException {
295    return ConcurrentMapUtils.computeIfAbsentEx(adminStubs,
296      getStubKey(AdminService.getDescriptor().getName(), serverName),
297      () -> createAdminServerStub(serverName));
298  }
299
300  CompletableFuture<MasterService.Interface> getMasterStub() {
301    return ConnectionUtils.getOrFetch(masterStub, masterStubMakeFuture, false, () -> {
302      CompletableFuture<MasterService.Interface> future = new CompletableFuture<>();
303      addListener(registry.getActiveMaster(), (addr, error) -> {
304        if (error != null) {
305          future.completeExceptionally(error);
306        } else if (addr == null) {
307          future.completeExceptionally(new MasterNotRunningException(
308            "ZooKeeper available but no active master location found"));
309        } else {
310          LOG.debug("The fetched master address is {}", addr);
311          try {
312            future.complete(createMasterStub(addr));
313          } catch (IOException e) {
314            future.completeExceptionally(e);
315          }
316        }
317
318      });
319      return future;
320    }, stub -> true, "master stub");
321  }
322
323  String getClusterId() {
324    try {
325      return registry.getClusterId().get();
326    } catch (InterruptedException | ExecutionException e) {
327      LOG.error("Error fetching cluster ID: ", e);
328    }
329    return null;
330  }
331
332  void clearMasterStubCache(MasterService.Interface stub) {
333    masterStub.compareAndSet(stub, null);
334  }
335
336  Optional<ServerStatisticTracker> getStatisticsTracker() {
337    return stats;
338  }
339
340  ClientBackoffPolicy getBackoffPolicy() {
341    return backoffPolicy;
342  }
343
344  @Override
345  public AsyncTableBuilder<AdvancedScanResultConsumer> getTableBuilder(TableName tableName) {
346    return new AsyncTableBuilderBase<AdvancedScanResultConsumer>(tableName, connConf) {
347
348      @Override
349      public AsyncTable<AdvancedScanResultConsumer> build() {
350        return new RawAsyncTableImpl(AsyncConnectionImpl.this, RETRY_TIMER, this);
351      }
352    };
353  }
354
355  @Override
356  public AsyncTableBuilder<ScanResultConsumer> getTableBuilder(TableName tableName,
357    ExecutorService pool) {
358    return new AsyncTableBuilderBase<ScanResultConsumer>(tableName, connConf) {
359
360      @Override
361      public AsyncTable<ScanResultConsumer> build() {
362        RawAsyncTableImpl rawTable =
363          new RawAsyncTableImpl(AsyncConnectionImpl.this, RETRY_TIMER, this);
364        return new AsyncTableImpl(rawTable, pool);
365      }
366    };
367  }
368
369  @Override
370  public AsyncAdminBuilder getAdminBuilder() {
371    return new AsyncAdminBuilderBase(connConf) {
372      @Override
373      public AsyncAdmin build() {
374        return new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, RETRY_TIMER, this);
375      }
376    };
377  }
378
379  @Override
380  public AsyncAdminBuilder getAdminBuilder(ExecutorService pool) {
381    return new AsyncAdminBuilderBase(connConf) {
382      @Override
383      public AsyncAdmin build() {
384        RawAsyncHBaseAdmin rawAdmin =
385          new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, RETRY_TIMER, this);
386        return new AsyncHBaseAdmin(rawAdmin, pool);
387      }
388    };
389  }
390
391  @Override
392  public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName) {
393    return new AsyncBufferedMutatorBuilderImpl(connConf, getTableBuilder(tableName), RETRY_TIMER);
394  }
395
396  @Override
397  public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName,
398    ExecutorService pool) {
399    return new AsyncBufferedMutatorBuilderImpl(connConf, getTableBuilder(tableName, pool),
400      RETRY_TIMER);
401  }
402
403  @Override
404  public Connection toConnection() {
405    ConnectionOverAsyncConnection c = this.conn;
406    if (c != null) {
407      return c;
408    }
409    synchronized (this) {
410      c = this.conn;
411      if (c != null) {
412        return c;
413      }
414      c = new ConnectionOverAsyncConnection(this);
415      this.conn = c;
416    }
417    return c;
418  }
419
420  private Hbck getHbckInternal(ServerName masterServer) {
421    Span.current().setAttribute(SERVER_NAME_KEY, masterServer.getServerName());
422    // we will not create a new connection when creating a new protobuf stub, and for hbck there
423    // will be no performance consideration, so for simplification we will create a new stub every
424    // time instead of caching the stub here.
425    return new HBaseHbck(MasterProtos.HbckService.newBlockingStub(
426      rpcClient.createBlockingRpcChannel(masterServer, user, rpcTimeout)), rpcControllerFactory);
427  }
428
429  @Override
430  public CompletableFuture<Hbck> getHbck() {
431    return TraceUtil.tracedFuture(() -> {
432      CompletableFuture<Hbck> future = new CompletableFuture<>();
433      addListener(registry.getActiveMaster(), (sn, error) -> {
434        if (error != null) {
435          future.completeExceptionally(error);
436        } else {
437          future.complete(getHbckInternal(sn));
438        }
439      });
440      return future;
441    }, "AsyncConnection.getHbck");
442  }
443
444  @Override
445  public Hbck getHbck(ServerName masterServer) {
446    return TraceUtil.trace(() -> getHbckInternal(masterServer), "AsyncConnection.getHbck");
447  }
448
449  Optional<MetricsConnection> getConnectionMetrics() {
450    return metrics;
451  }
452}