[GitHub] [hbase] bbeaudreault commented on a diff in pull request #4937: HBASE-27536: improve slowlog payload
bbeaudreault commented on code in PR #4937: URL: https://github.com/apache/hbase/pull/4937#discussion_r1085671686 ## hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java: ## @@ -36,13 +43,31 @@ @InterfaceStability.Evolving final public class OnlineLogRecord extends LogEntry { - // used to convert object to pretty printed format - // used by toJsonPrettyPrint() + private static final Logger LOG = LoggerFactory.getLogger(OnlineLogRecord.class.getName()); + private static final JsonElement EXCLUDED_NODE = JsonParser.parseString(HConstants.EMPTY_STRING); + + private static JsonElement serializeCatchAll(Operation operation) { +try { + return JsonParser.parseString(operation.toJSON()); +} catch (Exception e) { + LOG.warn("Suppressing exception during OnlineLogRecord serialization with operation {}", +operation, e); + return EXCLUDED_NODE; +} + } + + private static final Gson INNER_GSON = GsonUtil.createGson().setPrettyPrinting() +.registerTypeAdapter(Operation.class, + (JsonSerializer< +Operation>) (operation, type, jsonSerializationContext) -> serializeCatchAll(operation)) +.registerTypeAdapter(Optional.class, + (JsonSerializer>) (optional, type, jsonSerializationContext) -> optional +.map(jsonSerializationContext::serialize).orElse(EXCLUDED_NODE)) +.create(); private static final Gson GSON = GsonUtil.createGson().setPrettyPrinting().registerTypeAdapter(OnlineLogRecord.class, (JsonSerializer) (slowLogPayload, type, jsonSerializationContext) -> { -Gson gson = new Gson(); -JsonObject jsonObj = (JsonObject) gson.toJsonTree(slowLogPayload); +JsonObject jsonObj = (JsonObject) INNER_GSON.toJsonTree(slowLogPayload); Review Comment: did you try doing `jsonSerliazationContext.serialize(slowPayLoad)`? I feel like that's the appropriate call which should hopefully work with chaining, but not sure. -- 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-unsubscr...@hbase.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] bbeaudreault commented on a diff in pull request #4937: HBASE-27536: improve slowlog payload
bbeaudreault commented on code in PR #4937: URL: https://github.com/apache/hbase/pull/4937#discussion_r1082947181 ## hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java: ## @@ -128,11 +193,52 @@ public int getMultiServiceCalls() { return multiServiceCalls; } + /** + * If {@value org.apache.hadoop.hbase.HConstants#SLOW_LOG_OPERATION_MESSAGE_PAYLOAD_ENABLED} is + * enabled then this value may be present and should represent the Scan that produced the given + * {@link OnlineLogRecord}. This value should only be present if {@link #getMulti()}, + * {@link #getGet()}, and {@link #getMutate()} are empty + */ + public Optional getScan() { +return scan; + } + + /** + * If {@value org.apache.hadoop.hbase.HConstants#SLOW_LOG_OPERATION_MESSAGE_PAYLOAD_ENABLED} is + * enabled then this value may be present and should represent the MultiRequest that produced the + * given {@link OnlineLogRecord}. This value should only be present if {@link #getScan}, + * {@link #getGet()}, and {@link #getMutate()} are empty + */ + public Optional> getMulti() { +return multi; + } + + /** + * If {@value org.apache.hadoop.hbase.HConstants#SLOW_LOG_OPERATION_MESSAGE_PAYLOAD_ENABLED} is + * enabled then this value may be present and should represent the Get that produced the given + * {@link OnlineLogRecord}. This value should only be present if {@link #getScan()}, + * {@link #getMulti()} ()}, and {@link #getMutate()} are empty + */ + public Optional getGet() { +return get; + } + + /** + * If {@value org.apache.hadoop.hbase.HConstants#SLOW_LOG_OPERATION_MESSAGE_PAYLOAD_ENABLED} is + * enabled then this value may be present and should represent the Mutation that produced the + * given {@link OnlineLogRecord}. This value should only be present if {@link #getScan}, + * {@link #getMulti()} ()}, and {@link #getGet()} ()} are empty + */ + public Optional getMutate() { +return mutate; + } Review Comment: We should think a little about whether this is actually the API we want to expose. Since this class is InterfaceAudience.Public, any changes we make here we're stuck with for a long time. We can only remove/rename methods (i.e. breaking changes) at major version releases and we haven't had a major version release in years. There's potentially a little more flexibility since it's InterfaceStability.Evolving, but I don't think there's really consensus across all committers/PMC as to whether this grants us anything. It's not mentioned in our guides except: > Public packages marked as evolving may be changed, but it is discouraged. Anyway, not saying what we have here is wrong per-se. But we should take a minute to think about how we might evolve usage of these slow log stuff over time and make sure what we have here will support that. -- 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-unsubscr...@hbase.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] bbeaudreault commented on a diff in pull request #4937: HBASE-27536: improve slowlog payload
bbeaudreault commented on code in PR #4937: URL: https://github.com/apache/hbase/pull/4937#discussion_r1082940414 ## hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/impl/SlowLogQueueService.java: ## @@ -151,15 +156,30 @@ public void consumeEventFromDisruptor(NamedQueuePayload namedQueuePayload) { final String userName = rpcCall.getRequestUserName().orElse(StringUtils.EMPTY); final String methodDescriptorName = methodDescriptor != null ? methodDescriptor.getName() : StringUtils.EMPTY; -TooSlowLog.SlowLogPayload slowLogPayload = TooSlowLog.SlowLogPayload.newBuilder() +TooSlowLog.SlowLogPayload.Builder slowLogPayloadBuilder = TooSlowLog.SlowLogPayload.newBuilder() .setCallDetails(methodDescriptorName + "(" + param.getClass().getName() + ")") .setClientAddress(clientAddress).setMethodName(methodDescriptorName).setMultiGets(numGets) .setMultiMutations(numMutations).setMultiServiceCalls(numServiceCalls) .setParam(slowLogParams != null ? slowLogParams.getParams() : StringUtils.EMPTY) .setProcessingTime(processingTime).setQueueTime(qTime) .setRegionName(slowLogParams != null ? slowLogParams.getRegionName() : StringUtils.EMPTY) .setResponseSize(responseSize).setServerClass(className).setStartTime(startTime).setType(type) - .setUserName(userName).build(); + .setUserName(userName); +if (slowLogParams != null) { + if (slowLogParams.getScan() != null) { +slowLogPayloadBuilder.setScan(slowLogParams.getScan()); + } + if (slowLogParams.getMulti() != null) { +slowLogPayloadBuilder.setMulti(slowLogParams.getMulti()); + } + if (slowLogParams.getGet() != null) { +slowLogPayloadBuilder.setGet(slowLogParams.getGet()); + } + if (slowLogParams.getMutate() != null) { +slowLogPayloadBuilder.setMutate(slowLogParams.getMutate()); Review Comment: i do think we should do that, for one from a privacy/memory perspective, and for another to be consistent with the case where cell block encoding is enabled. in that case, data won't be in the proto anyway. I don't think the literal bytes of the request are super useful, and we already have the total size of the request. -- 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-unsubscr...@hbase.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] bbeaudreault commented on a diff in pull request #4937: HBASE-27536: improve slowlog payload
bbeaudreault commented on code in PR #4937: URL: https://github.com/apache/hbase/pull/4937#discussion_r1082935124 ## hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java: ## @@ -36,10 +46,48 @@ @InterfaceStability.Evolving final public class OnlineLogRecord extends LogEntry { + private static final Logger LOG = LoggerFactory.getLogger(OnlineLogRecord.class.getName()); + + private static Optional serializeCatchAll(Operation operation) { +try { + return Optional.of(JsonParser.parseString(operation.toJSON())); +} catch (Exception e) { + LOG.warn("Suppressing exception during OnlineLogRecord serialization with operation {}", +operation, e); + return Optional.empty(); +} + } + // used to convert object to pretty printed format // used by toJsonPrettyPrint() - private static final Gson GSON = - GsonUtil.createGson().setPrettyPrinting().registerTypeAdapter(OnlineLogRecord.class, + private static final Type OPERATION_LIST = +TypeToken.getParameterized(List.class, Operation.class).getType(); + private static final Type OPERATION_MAYBE = +TypeToken.getParameterized(Optional.class, Operation.class).getType(); + private static final Type OPERATION_LIST_MAYBE = +TypeToken.getParameterized(Optional.class, OPERATION_LIST.getClass()).getType(); + private static final JsonElement EMPTY_NODE = JsonParser.parseString(HConstants.EMPTY_STRING); + private static final Gson GSON = GsonUtil.createGson().setPrettyPrinting() +.registerTypeAdapter(OPERATION_MAYBE.getClass(), + (JsonSerializer< +Optional>) (operationMaybe, type, jsonSerializationContext) -> operationMaybe + .map(operation -> serializeCatchAll(operation).orElse(EMPTY_NODE)).orElse(EMPTY_NODE)) +.registerTypeAdapter(OPERATION_LIST_MAYBE.getClass(), (JsonSerializer< + Optional>>) (operationsMaybe, type, jsonSerializationContext) -> { +if (!operationsMaybe.isPresent()) { + return EMPTY_NODE; +} +JsonObject jsonObj = new JsonObject(); +final AtomicInteger i = new AtomicInteger(0); +for (Operation operation : operationsMaybe.get()) { + serializeCatchAll(operation).ifPresent(json -> { +jsonObj.add(String.valueOf(i), json); +i.incrementAndGet(); + }); +} +return jsonObj; + }) Review Comment: this seems like a lot. maybe it's the only way, but maybe there's a different way. Below we are registering a type adapter for OnlineLogRecord.class. Currently we just call gson.toJsonTree then make a few cleanups. One question is whether we even need to fully serialize all the operation stuff to json here. I'm not sure what this is used for exactly, but maybe we don't have to include them? Otherwise, i wonder if there's a slightly simpler way to represent these in the below type adapter. I was imaginging you could even do something like this: ``` if (slowLogPayload.getScan().isPresent()) { jsonObj.set("scan", slowLogPayload.getScan().toJSON()); } etc ``` You're already checking for presence of these fields below in order to remove the empties, so it might just be a matter of adding an `else` to each. Not sure, not saying that's definitely better but something to think about. I'm also surprised that there's not a built in type adapter for Optional and List, in which case we would just need a type adapter for Operation which does the toJSON() call. -- 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-unsubscr...@hbase.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] bbeaudreault commented on a diff in pull request #4937: HBASE-27536: improve slowlog payload
bbeaudreault commented on code in PR #4937: URL: https://github.com/apache/hbase/pull/4937#discussion_r1064648281 ## hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java: ## @@ -1072,6 +1077,22 @@ public static ClientProtos.Scan toScan(final Scan scan) throws IOException { return scanBuilder.build(); } + public static List toMulti(MultiRequest multiRequest) { Review Comment: Otherwise, from a completeness perspective it's missing `hasServiceCall()` handling. There is no toServiceCall method yet, but you can just reverse the logic from RequestConverter.buildNoDataRegionActions which converts a RegionCoprocessorServiceExec to a CoprocessorServiceCall. -- 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-unsubscr...@hbase.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] bbeaudreault commented on a diff in pull request #4937: HBASE-27536: improve slowlog payload
bbeaudreault commented on code in PR #4937: URL: https://github.com/apache/hbase/pull/4937#discussion_r1064644139 ## hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java: ## @@ -1072,6 +1077,22 @@ public static ClientProtos.Scan toScan(final Scan scan) throws IOException { return scanBuilder.build(); } + public static List toMulti(MultiRequest multiRequest) { Review Comment: this probably should throw an exception, since that seems more standard than catching it. Seems like you handle catching exceptions in your catchAll method anyway -- 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-unsubscr...@hbase.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] bbeaudreault commented on a diff in pull request #4937: HBASE-27536: improve slowlog payload
bbeaudreault commented on code in PR #4937: URL: https://github.com/apache/hbase/pull/4937#discussion_r1064639625 ## hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java: ## @@ -261,15 +311,17 @@ public boolean equals(Object o) { .append(multiServiceCalls, that.multiServiceCalls).append(clientAddress, that.clientAddress) .append(serverClass, that.serverClass).append(methodName, that.methodName) .append(callDetails, that.callDetails).append(param, that.param) - .append(regionName, that.regionName).append(userName, that.userName).isEquals(); + .append(regionName, that.regionName).append(userName, that.userName).append(scan, that.scan) + .append(multi, that.multi).append(get, that.get).append(mutate, that.mutate).isEquals(); } @Override public int hashCode() { return new HashCodeBuilder(17, 37).append(startTime).append(processingTime).append(queueTime) .append(responseSize).append(clientAddress).append(serverClass).append(methodName) .append(callDetails).append(param).append(regionName).append(userName).append(multiGetsCount) - .append(multiMutationsCount).append(multiServiceCalls).toHashCode(); + .append(multiMutationsCount).append(multiServiceCalls).append(scan).append(multi).append(get) + .append(mutate).toHashCode(); } @Override Review Comment: do we need to handle these new fields in `toJsonPrettyPrint()` at all? Not sure how this method is used, but I see special handling at the top of this class -- 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-unsubscr...@hbase.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [hbase] bbeaudreault commented on a diff in pull request #4937: HBASE-27536: improve slowlog payload
bbeaudreault commented on code in PR #4937: URL: https://github.com/apache/hbase/pull/4937#discussion_r1063706837 ## hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java: ## @@ -3376,7 +3401,9 @@ private static LogEntry getSlowLogRecord(final TooSlowLog.SlowLogPayload slowLog .setQueueTime(slowLogPayload.getQueueTime()).setRegionName(slowLogPayload.getRegionName()) .setResponseSize(slowLogPayload.getResponseSize()) .setServerClass(slowLogPayload.getServerClass()).setStartTime(slowLogPayload.getStartTime()) -.setUserName(slowLogPayload.getUserName()).build(); + .setUserName(slowLogPayload.getUserName()).setScan(slowLogPayload.getScan()) +.setMulti(slowLogPayload.getMulti()).setGet(slowLogPayload.getGet()) Review Comment: We can't expose the actual protos to downstream users. Instead we should use the correct ProtobufUtil or RequestConverter methods to convert these into the actual client models here -- 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-unsubscr...@hbase.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org