You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2021/12/01 23:47:03 UTC

[GitHub] [hbase] ndimiduk commented on a change in pull request #3906: HBASE-26472 Adhere to semantic conventions regarding table data operations

ndimiduk commented on a change in pull request #3906:
URL: https://github.com/apache/hbase/pull/3906#discussion_r760655901



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
##########
@@ -548,14 +591,17 @@ public void run(MultiResponse resp) {
     validatePutsInRowMutations(mutations, conn.connConf.getMaxKeyValueSize());
     long nonceGroup = conn.getNonceGenerator().getNonceGroup();
     long nonce = conn.getNonceGenerator().newNonce();
+    final Supplier<Span> supplier = new TableOperationSpanBuilder()
+      .setTableName(tableName)
+      .setOperation(HBaseSemanticAttributes.Operation.BATCH);

Review comment:
       Yes, this one could use the `mutations` instance.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/trace/TableOperationSpanBuilder.java
##########
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client.trace;
+
+import static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.DB_NAME;
+import static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.DB_OPERATION;
+import static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.NAMESPACE_KEY;
+import static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.TABLE_KEY;
+import io.opentelemetry.api.common.AttributeKey;
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.api.trace.SpanBuilder;
+import io.opentelemetry.api.trace.SpanKind;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.CheckAndMutate;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionCoprocessorServiceExec;
+import org.apache.hadoop.hbase.client.Row;
+import org.apache.hadoop.hbase.client.RowMutations;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.Operation;
+import org.apache.hadoop.hbase.trace.TraceUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Construct {@link io.opentelemetry.api.trace.Span} instances originating from
+ * "table operations" -- the verbs in our public API that interact with data in tables.
+ */
+@InterfaceAudience.Private
+public class TableOperationSpanBuilder implements Supplier<Span> {
+
+  // n.b. The results of this class are tested implicitly by way of the likes of
+  // `TestAsyncTableTracing` and friends.
+
+  private static final String unknown = "UNKNOWN";
+
+  private TableName tableName;
+  private final Map<AttributeKey<?>, Object> attributes = new HashMap<>();
+
+  @Override public Span get() {
+    return build();
+  }
+
+  public TableOperationSpanBuilder setOperation(final Scan scan) {
+    return setOperation(valueFrom(scan));
+  }
+
+  public TableOperationSpanBuilder setOperation(final Row row) {
+    return setOperation(valueFrom(row));
+  }
+
+  public TableOperationSpanBuilder setOperation(final Operation operation) {
+    attributes.put(DB_OPERATION, operation.name());
+    return this;
+  }
+
+  public TableOperationSpanBuilder setTableName(final TableName tableName) {
+    this.tableName = tableName;
+    attributes.put(NAMESPACE_KEY, tableName.getNamespaceAsString());
+    attributes.put(DB_NAME, tableName.getNamespaceAsString());
+    attributes.put(TABLE_KEY, tableName.getNameAsString());
+    return this;
+  }
+
+  @SuppressWarnings("unchecked")
+  public Span build() {
+    final String name = attributes.getOrDefault(DB_OPERATION, unknown)
+        + " "
+        + (tableName != null ? tableName.getNameWithNamespaceInclAsString() : unknown);
+    final SpanBuilder builder = TraceUtil.getGlobalTracer()
+      .spanBuilder(name)
+      // TODO: what about clients embedded in Master/RegionServer/Gateways/&c?
+      .setSpanKind(SpanKind.CLIENT);
+    attributes.forEach((k, v) -> builder.setAttribute((AttributeKey<? super Object>) k, v));
+    return builder.startSpan();
+  }
+
+  private static Operation valueFrom(final Scan scan) {
+    if (scan == null) { return null; }
+    return Operation.SCAN;
+  }
+
+  private static Operation valueFrom(final Row row) {
+    if (row == null) { return null; }
+    if (row instanceof Append) { return Operation.APPEND; }
+    if (row instanceof CheckAndMutate) { return Operation.CHECK_AND_MUTATE; }
+    if (row instanceof Delete) { return Operation.DELETE; }
+    if (row instanceof Get) { return Operation.GET; }
+    if (row instanceof Increment) { return Operation.INCREMENT; }
+    if (row instanceof Put) { return Operation.PUT; }
+    if (row instanceof RegionCoprocessorServiceExec) {
+      return Operation.COPROC_EXEC;
+    }

Review comment:
       No, I don't have coverage for these yet. I haven't decided on how these should be traced yet... I want to assess after I've studied the [RPC Span guidance](https://github.com/open-telemetry/opentelemetry-specification/blob/3e380e2/specification/trace/semantic_conventions/rpc.md#span-name) / implemented HBASE-26521. Let me add a subtask to track this item too. Filed HBASE-26531.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
##########
@@ -329,43 +348,52 @@ private void preCheck() {
     public CompletableFuture<Boolean> thenPut(Put put) {
       validatePut(put, conn.connConf.getMaxKeyValueSize());
       preCheck();
+      final Supplier<Span> supplier = new TableOperationSpanBuilder()
+        .setTableName(tableName)
+        .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE);
       return tracedFuture(
         () -> RawAsyncTableImpl.this.<Boolean> newCaller(row, put.getPriority(), rpcTimeoutNs)
           .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, put,
             (rn, p) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier, op, value,
               null, timeRange, p, HConstants.NO_NONCE, HConstants.NO_NONCE),
             (c, r) -> r.getProcessed()))
           .call(),
-        "AsyncTable.CheckAndMutateBuilder.thenPut", tableName);
+        supplier);
     }
 
     @Override
     public CompletableFuture<Boolean> thenDelete(Delete delete) {
       preCheck();
+      final Supplier<Span> supplier = new TableOperationSpanBuilder()
+        .setTableName(tableName)
+        .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE);
       return tracedFuture(
         () -> RawAsyncTableImpl.this.<Boolean> newCaller(row, delete.getPriority(), rpcTimeoutNs)
           .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, delete,
             (rn, d) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier, op, value,
               null, timeRange, d, HConstants.NO_NONCE, HConstants.NO_NONCE),
             (c, r) -> r.getProcessed()))
           .call(),
-        "AsyncTable.CheckAndMutateBuilder.thenDelete", tableName);
+        supplier);
     }
 
     @Override
-    public CompletableFuture<Boolean> thenMutate(RowMutations mutation) {
+    public CompletableFuture<Boolean> thenMutate(RowMutations mutations) {
       preCheck();
-      validatePutsInRowMutations(mutation, conn.connConf.getMaxKeyValueSize());
+      validatePutsInRowMutations(mutations, conn.connConf.getMaxKeyValueSize());
+      final Supplier<Span> supplier = new TableOperationSpanBuilder()
+        .setTableName(tableName)
+        .setOperation(HBaseSemanticAttributes.Operation.BATCH);

Review comment:
       No -- this is confusing -- we're both wrong. The operation of the `CheckAndMutateBuilder` instance is a `CHECK_AND_PUT`. That operation is a "container" that ships multiple underlying operations. `thenMutate` allows the `CHECH_AND_PUT` to apply the contents of `mutations` ; `thenDelete` sends a `DELETE`, &c.
   
   My patch on HBASE-26473 introduces the span attribute `db.hbase.container_operations` as a general mechanism for handling these operations that wrap other operations. That commit is on the feature branch PR that I posted earlier, https://github.com/apache/hbase/pull/3863/commits/ac1cfbbec75e2a9a8ca1282cce307db0ab72c3eb

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
##########
@@ -405,34 +436,40 @@ public CheckAndMutateWithFilterBuilder timeRange(TimeRange timeRange) {
             filter, timeRange, p, HConstants.NO_NONCE, HConstants.NO_NONCE),
           (c, r) -> r.getProcessed()))
         .call(),
-        "AsyncTable.CheckAndMutateWithFilterBuilder.thenPut", tableName);
+        supplier);
     }
 
     @Override
     public CompletableFuture<Boolean> thenDelete(Delete delete) {
+      final Supplier<Span> supplier = new TableOperationSpanBuilder()
+        .setTableName(tableName)
+        .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE);
       return tracedFuture(
         () -> RawAsyncTableImpl.this.<Boolean> newCaller(row, delete.getPriority(), rpcTimeoutNs)
           .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, delete,
             (rn, d) -> RequestConverter.buildMutateRequest(rn, row, null, null, null, null, filter,
               timeRange, d, HConstants.NO_NONCE, HConstants.NO_NONCE),
             (c, r) -> r.getProcessed()))
           .call(),
-        "AsyncTable.CheckAndMutateWithFilterBuilder.thenDelete", tableName);
+        supplier);
     }
 
     @Override
-    public CompletableFuture<Boolean> thenMutate(RowMutations mutation) {
-      validatePutsInRowMutations(mutation, conn.connConf.getMaxKeyValueSize());
+    public CompletableFuture<Boolean> thenMutate(RowMutations mutations) {
+      validatePutsInRowMutations(mutations, conn.connConf.getMaxKeyValueSize());
+      final Supplier<Span> supplier = new TableOperationSpanBuilder()
+        .setTableName(tableName)
+        .setOperation(HBaseSemanticAttributes.Operation.BATCH);

Review comment:
       Nope, I don't think so. Instead, this should be a `CHECK_AND_PUT`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org