This is an automated email from the ASF dual-hosted git repository.

hsaputra pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new cde0502  state store: getStream: log stream info
cde0502 is described below

commit cde05029ffcc8492aab821f0350a43793e237f3e
Author: mauricebarnum <[email protected]>
AuthorDate: Wed Aug 18 09:40:03 2021 -0700

    state store: getStream: log stream info
    
    log the the tuple (namespace id, stream id, stream name) in 
RootStorageService getRange request.
    
    ### Motivation
    
    Server request metrics are labeled with the stream id, extracted from the 
routing header.  The stream name
    (aka "table name") is not available but more useful.  Rather than making a 
(cacheable) RPC request to fetch
    the id -> name mapping in the metrics, logging the information allows one 
to find the name without requiring
    admin access to the state store service.
    
    
    
    Reviewers: Ivan Kelly <[email protected]>, Enrico Olivelli 
<[email protected]>, Henry Saputra <[email protected]>
    
    This closes #2758 from mauricebarnum/log-stream-name and squashes the 
following commits:
    
    4ef7ac070 [Maurice Barnum] cleanup: remove extraneous "final" declarations
    284b6430b [Maurice Barnum] state store: create and delete stream: log 
stream info
---
 .../storage/impl/metadata/RootRangeStoreImpl.java    | 20 +++++++++++++-------
 1 file changed, 13 insertions(+), 7 deletions(-)

diff --git 
a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java
 
b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java
index 3a4d78a..507cb56 100644
--- 
a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java
+++ 
b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java
@@ -80,7 +80,7 @@ public class RootRangeStoreImpl
     /**
      * ns name key: [NS_NAME_TAG][ns_name].
      */
-    static final byte[] getNamespaceNameKey(String nsName) {
+    static byte[] getNamespaceNameKey(String nsName) {
         byte[] nsNameBytes = nsName.getBytes(UTF_8);
         byte[] nsNameKey = new byte[nsNameBytes.length + 1];
         System.arraycopy(nsNameBytes, 0, nsNameKey, 1, nsNameBytes.length);
@@ -91,7 +91,7 @@ public class RootRangeStoreImpl
     /**
      * ns id key: [NS_ID_TAG][ns_id].
      */
-    static final byte[] getNamespaceIdKey(long nsId) {
+    static byte[] getNamespaceIdKey(long nsId) {
         byte[] nsIdBytes = new byte[Long.BYTES + 1];
         nsIdBytes[0] = NS_ID_TAG;
         Bytes.toBytes(nsId, nsIdBytes, 1);
@@ -101,7 +101,7 @@ public class RootRangeStoreImpl
     /**
      * ns id end key: [NS_ID_TAG][ns_id][NS_END_SEP].
      */
-    static final byte[] getNamespaceIdEndKey(long nsId) {
+    static byte[] getNamespaceIdEndKey(long nsId) {
         byte[] nsIdBytes = new byte[Long.BYTES + 2];
         nsIdBytes[0] = NS_ID_TAG;
         Bytes.toBytes(nsId, nsIdBytes, 1);
@@ -112,7 +112,7 @@ public class RootRangeStoreImpl
     /**
      * stream name key: [NS_ID_TAG][ns_id][STREAM_NAME_SEP][stream_name].
      */
-    static final byte[] getStreamNameKey(long nsId, String streamName) {
+    static byte[] getStreamNameKey(long nsId, String streamName) {
         byte[] streamNameBytes = streamName.getBytes(UTF_8);
         byte[] streamNameKey = new byte[streamNameBytes.length + Long.BYTES + 
2];
         streamNameKey[0] = NS_ID_TAG;
@@ -125,7 +125,7 @@ public class RootRangeStoreImpl
     /**
      * stream name id: [NS_ID_TAG][ns_id][STREAM_ID_SEP][stream_id].
      */
-    static final byte[] getStreamIdKey(long nsId, long streamId) {
+    static byte[] getStreamIdKey(long nsId, long streamId) {
         byte[] streamIdBytes = new byte[2 * Long.BYTES + 2];
         streamIdBytes[0] = NS_ID_TAG;
         Bytes.toBytes(nsId, streamIdBytes, 1);
@@ -137,7 +137,7 @@ public class RootRangeStoreImpl
     /**
      * stream id: [STREAM_ID_TAG][stream_id].
      */
-    static final byte[] getStreamIdKey(long streamId) {
+    static byte[] getStreamIdKey(long streamId) {
         byte[] streamIdBytes = new byte[Long.BYTES + 1];
         streamIdBytes[0] = STREAM_ID_TAG;
         Bytes.toBytes(streamId, streamIdBytes, 1);
@@ -449,6 +449,11 @@ public class RootRangeStoreImpl
 
     }
 
+    private static void logStreamOp(String op, long nsId, long streamId, 
String streamName, StatusCode result) {
+        log.info("stream {} namespace_id={} stream_id={} stream_name={} 
result={}",
+                op, nsId, streamId, streamName, result);
+    }
+
     private CompletableFuture<CreateStreamResponse> 
executeCreateStreamTxn(long nsId,
                                                                            
String streamName,
                                                                            
StreamConfiguration streamConf,
@@ -463,7 +468,6 @@ public class RootRangeStoreImpl
 
         long scId = placementPolicy.placeStreamRange(streamId, 0L);
 
-
         StreamProperties streamProps = StreamProperties.newBuilder()
             .setStreamId(streamId)
             .setStreamName(streamName)
@@ -506,6 +510,7 @@ public class RootRangeStoreImpl
                         // TODO: differentiate the error codes
                         respBuilder.setCode(StatusCode.INTERNAL_SERVER_ERROR);
                     }
+                    logStreamOp("create", nsId, streamId, streamName, 
respBuilder.getCode());
                     return respBuilder.build();
                 } finally {
                     txnResult.close();
@@ -593,6 +598,7 @@ public class RootRangeStoreImpl
                 } else {
                     respBuilder.setCode(StatusCode.INTERNAL_SERVER_ERROR);
                 }
+                logStreamOp("delete", nsId, streamId, streamName, 
respBuilder.getCode());
                 return respBuilder.build();
             } finally {
                 txnResult.close();

Reply via email to