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.TableName.META_TABLE_NAME;
021import static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.REGION_NAMES_KEY;
022import static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.SERVER_NAME_KEY;
023import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
024
025import io.opentelemetry.api.trace.Span;
026import io.opentelemetry.api.trace.StatusCode;
027import io.opentelemetry.context.Scope;
028import java.util.Arrays;
029import java.util.Collections;
030import java.util.List;
031import java.util.Objects;
032import java.util.Optional;
033import java.util.concurrent.CompletableFuture;
034import java.util.concurrent.TimeUnit;
035import java.util.function.Function;
036import java.util.function.Supplier;
037import java.util.stream.Collectors;
038import org.apache.hadoop.hbase.HRegionLocation;
039import org.apache.hadoop.hbase.RegionLocations;
040import org.apache.hadoop.hbase.ServerName;
041import org.apache.hadoop.hbase.TableName;
042import org.apache.hadoop.hbase.client.trace.ConnectionSpanBuilder;
043import org.apache.hadoop.hbase.client.trace.TableSpanBuilder;
044import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
045import org.apache.hadoop.hbase.trace.TraceUtil;
046import org.apache.hadoop.hbase.util.Bytes;
047import org.apache.hadoop.hbase.util.FutureUtils;
048import org.apache.yetus.audience.InterfaceAudience;
049import org.slf4j.Logger;
050import org.slf4j.LoggerFactory;
051
052import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
053import org.apache.hbase.thirdparty.io.netty.util.Timeout;
054
055/**
056 * The asynchronous region locator.
057 */
058@InterfaceAudience.Private
059class AsyncRegionLocator {
060
061  private static final Logger LOG = LoggerFactory.getLogger(AsyncRegionLocator.class);
062
063  private final HashedWheelTimer retryTimer;
064
065  private final AsyncConnectionImpl conn;
066
067  private final AsyncMetaRegionLocator metaRegionLocator;
068
069  private final AsyncNonMetaRegionLocator nonMetaRegionLocator;
070
071  AsyncRegionLocator(AsyncConnectionImpl conn, HashedWheelTimer retryTimer) {
072    this.conn = conn;
073    this.metaRegionLocator = new AsyncMetaRegionLocator(conn.registry);
074    this.nonMetaRegionLocator = new AsyncNonMetaRegionLocator(conn);
075    this.retryTimer = retryTimer;
076  }
077
078  private <T> CompletableFuture<T> withTimeout(CompletableFuture<T> future, long timeoutNs,
079    Supplier<String> timeoutMsg) {
080    if (future.isDone() || timeoutNs <= 0) {
081      return future;
082    }
083    Timeout timeoutTask = retryTimer.newTimeout(t -> {
084      if (future.isDone()) {
085        return;
086      }
087      future.completeExceptionally(new TimeoutIOException(timeoutMsg.get()));
088    }, timeoutNs, TimeUnit.NANOSECONDS);
089    FutureUtils.addListener(future, (loc, error) -> {
090      if (error != null && error.getClass() != TimeoutIOException.class) {
091        // cancel timeout task if we are not completed by it.
092        timeoutTask.cancel();
093      }
094    });
095    return future;
096  }
097
098  private boolean isMeta(TableName tableName) {
099    return TableName.isMetaTableName(tableName);
100  }
101
102  private <T> CompletableFuture<T> tracedLocationFuture(
103    Supplier<CompletableFuture<T>> action,
104    Function<T, List<String>> getRegionNames,
105    Supplier<Span> spanSupplier
106  ) {
107    final Span span = spanSupplier.get();
108    try (Scope scope = span.makeCurrent()) {
109      CompletableFuture<T> future = action.get();
110      FutureUtils.addListener(future, (resp, error) -> {
111        if (error != null) {
112          TraceUtil.setError(span, error);
113        } else {
114          List<String> regionNames = getRegionNames.apply(resp);
115          if (!regionNames.isEmpty()) {
116            span.setAttribute(REGION_NAMES_KEY, regionNames);
117          }
118          span.setStatus(StatusCode.OK);
119        }
120        span.end();
121      });
122      return future;
123    }
124  }
125
126  private static List<String> getRegionNames(RegionLocations locs) {
127    if (locs == null || locs.getRegionLocations() == null) {
128      return Collections.emptyList();
129    }
130    return Arrays.stream(locs.getRegionLocations())
131      .filter(Objects::nonNull)
132      .map(HRegionLocation::getRegion)
133      .map(RegionInfo::getRegionNameAsString)
134      .collect(Collectors.toList());
135  }
136
137  private static List<String> getRegionNames(HRegionLocation location) {
138    return Optional.ofNullable(location)
139      .map(HRegionLocation::getRegion)
140      .map(RegionInfo::getRegionNameAsString)
141      .map(Collections::singletonList)
142      .orElseGet(Collections::emptyList);
143  }
144
145  CompletableFuture<RegionLocations> getRegionLocations(TableName tableName, byte[] row,
146    RegionLocateType type, boolean reload, long timeoutNs) {
147    final Supplier<Span> supplier = new TableSpanBuilder(conn)
148      .setName("AsyncRegionLocator.getRegionLocations")
149      .setTableName(tableName);
150    return tracedLocationFuture(() -> {
151      CompletableFuture<RegionLocations> future = isMeta(tableName) ?
152        metaRegionLocator.getRegionLocations(RegionReplicaUtil.DEFAULT_REPLICA_ID, reload) :
153        nonMetaRegionLocator.getRegionLocations(tableName, row,
154          RegionReplicaUtil.DEFAULT_REPLICA_ID, type, reload);
155      return withTimeout(future, timeoutNs,
156        () -> "Timeout(" + TimeUnit.NANOSECONDS.toMillis(timeoutNs) +
157          "ms) waiting for region locations for " + tableName + ", row='" +
158          Bytes.toStringBinary(row) + "'");
159    }, AsyncRegionLocator::getRegionNames, supplier);
160  }
161
162  CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
163    int replicaId, RegionLocateType type, boolean reload, long timeoutNs) {
164    final Supplier<Span> supplier = new TableSpanBuilder(conn)
165      .setName("AsyncRegionLocator.getRegionLocation")
166      .setTableName(tableName);
167    return tracedLocationFuture(() -> {
168      // meta region can not be split right now so we always call the same method.
169      // Change it later if the meta table can have more than one regions.
170      CompletableFuture<HRegionLocation> future = new CompletableFuture<>();
171      CompletableFuture<RegionLocations> locsFuture =
172        isMeta(tableName) ? metaRegionLocator.getRegionLocations(replicaId, reload) :
173          nonMetaRegionLocator.getRegionLocations(tableName, row, replicaId, type, reload);
174      addListener(locsFuture, (locs, error) -> {
175        if (error != null) {
176          future.completeExceptionally(error);
177          return;
178        }
179        HRegionLocation loc = locs.getRegionLocation(replicaId);
180        if (loc == null) {
181          future.completeExceptionally(
182            new RegionOfflineException("No location for " + tableName + ", row='" +
183              Bytes.toStringBinary(row) + "', locateType=" + type + ", replicaId=" + replicaId));
184        } else if (loc.getServerName() == null) {
185          future.completeExceptionally(
186            new RegionOfflineException("No server address listed for region '" +
187              loc.getRegion().getRegionNameAsString() + ", row='" + Bytes.toStringBinary(row) +
188              "', locateType=" + type + ", replicaId=" + replicaId));
189        } else {
190          future.complete(loc);
191        }
192      });
193      return withTimeout(future, timeoutNs,
194        () -> "Timeout(" + TimeUnit.NANOSECONDS.toMillis(timeoutNs) +
195          "ms) waiting for region location for " + tableName + ", row='" +
196          Bytes.toStringBinary(row) + "', replicaId=" + replicaId);
197    }, AsyncRegionLocator::getRegionNames, supplier);
198  }
199
200  CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
201    int replicaId, RegionLocateType type, long timeoutNs) {
202    return getRegionLocation(tableName, row, replicaId, type, false, timeoutNs);
203  }
204
205  CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
206    RegionLocateType type, boolean reload, long timeoutNs) {
207    return getRegionLocation(tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID, type, reload,
208      timeoutNs);
209  }
210
211  CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
212    RegionLocateType type, long timeoutNs) {
213    return getRegionLocation(tableName, row, type, false, timeoutNs);
214  }
215
216  void updateCachedLocationOnError(HRegionLocation loc, Throwable exception) {
217    if (loc.getRegion().isMetaRegion()) {
218      metaRegionLocator.updateCachedLocationOnError(loc, exception);
219    } else {
220      nonMetaRegionLocator.updateCachedLocationOnError(loc, exception);
221    }
222  }
223
224  void clearCache(TableName tableName) {
225    Supplier<Span> supplier = new TableSpanBuilder(conn)
226      .setName("AsyncRegionLocator.clearCache")
227      .setTableName(tableName);
228    TraceUtil.trace(() -> {
229      LOG.debug("Clear meta cache for {}", tableName);
230      if (tableName.equals(META_TABLE_NAME)) {
231        metaRegionLocator.clearCache();
232      } else {
233        nonMetaRegionLocator.clearCache(tableName);
234      }
235    }, supplier);
236  }
237
238  void clearCache(ServerName serverName) {
239    Supplier<Span> supplier = new ConnectionSpanBuilder(conn)
240      .setName("AsyncRegionLocator.clearCache")
241      .addAttribute(SERVER_NAME_KEY, serverName.getServerName());
242    TraceUtil.trace(() -> {
243      LOG.debug("Clear meta cache for {}", serverName);
244      metaRegionLocator.clearCache(serverName);
245      nonMetaRegionLocator.clearCache(serverName);
246      conn.getConnectionMetrics().ifPresent(MetricsConnection::incrMetaCacheNumClearServer);
247    }, supplier);
248  }
249
250  void clearCache() {
251    Supplier<Span> supplier = new ConnectionSpanBuilder(conn)
252      .setName("AsyncRegionLocator.clearCache");
253    TraceUtil.trace(() -> {
254      metaRegionLocator.clearCache();
255      nonMetaRegionLocator.clearCache();
256    }, supplier);
257  }
258
259  AsyncNonMetaRegionLocator getNonMetaRegionLocator() {
260    return nonMetaRegionLocator;
261  }
262
263  // only used for testing whether we have cached the location for a region.
264  RegionLocations getRegionLocationInCache(TableName tableName, byte[] row) {
265    if (TableName.isMetaTableName(tableName)) {
266      return metaRegionLocator.getRegionLocationInCache();
267    } else {
268      return nonMetaRegionLocator.getRegionLocationInCache(tableName, row);
269    }
270  }
271
272  // only used for testing whether we have cached the location for a table.
273  int getNumberOfCachedRegionLocations(TableName tableName) {
274    if (TableName.isMetaTableName(tableName)) {
275      return metaRegionLocator.getNumberOfCachedRegionLocations();
276    } else {
277      return nonMetaRegionLocator.getNumberOfCachedRegionLocations(tableName);
278    }
279  }
280}