diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/integration/processor/IrTestOpProcessor.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/integration/processor/IrTestOpProcessor.java index 2a3d35a74e3d..d1eeec433dde 100644 --- a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/integration/processor/IrTestOpProcessor.java +++ b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/integration/processor/IrTestOpProcessor.java @@ -24,6 +24,7 @@ import com.alibaba.graphscope.common.store.IrMeta; import com.alibaba.graphscope.gremlin.integration.result.GraphProperties; import com.alibaba.graphscope.gremlin.integration.result.GremlinTestResultProcessor; +import com.alibaba.graphscope.gremlin.plugin.QueryStatusCallback; import com.alibaba.graphscope.gremlin.plugin.processor.IrStandardOpProcessor; import com.alibaba.graphscope.gremlin.plugin.script.AntlrGremlinScriptEngine; @@ -93,21 +94,21 @@ public ThrowingConsumer select(Context ctx) { Traversal traversal = (Traversal) scriptEngine.eval(script, this.context); applyStrategies(traversal); - long jobId = JOB_ID_COUNTER.incrementAndGet(); IrMeta irMeta = metaQueryCallback.beforeExec(); + QueryStatusCallback statusCallback = + createQueryStatusCallback(script, jobId); processTraversal( traversal, new GremlinTestResultProcessor( ctx, traversal, - createMetricsPrinter(jobId, script), + statusCallback, testGraph, this.configs), - jobId, - script, irMeta, - new QueryTimeoutConfig(ctx.getRequestTimeout())); + new QueryTimeoutConfig(ctx.getRequestTimeout()), + statusCallback.getQueryLogger()); metaQueryCallback.afterExec(irMeta); }); return op; diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/integration/result/GremlinTestResultProcessor.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/integration/result/GremlinTestResultProcessor.java index 7327e35d84fb..c4d5c962f702 100644 --- a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/integration/result/GremlinTestResultProcessor.java +++ b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/integration/result/GremlinTestResultProcessor.java @@ -17,8 +17,8 @@ package com.alibaba.graphscope.gremlin.integration.result; import com.alibaba.graphscope.common.config.Configs; +import com.alibaba.graphscope.gremlin.plugin.QueryStatusCallback; import com.alibaba.graphscope.gremlin.result.processor.GremlinResultProcessor; -import com.alibaba.graphscope.gremlin.service.MetricsPrinter; import com.google.common.collect.ImmutableMap; import org.apache.tinkerpop.gremlin.process.remote.traversal.DefaultRemoteTraverser; @@ -46,10 +46,10 @@ public class GremlinTestResultProcessor extends GremlinResultProcessor { public GremlinTestResultProcessor( Context writeResult, Traversal traversal, - MetricsPrinter metricsPrinter, + QueryStatusCallback statusCallback, GraphProperties testGraph, Configs configs) { - super(writeResult, traversal, metricsPrinter); + super(writeResult, traversal, statusCallback); this.cachedProperties = testGraph.getProperties(configs); } diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/plugin/MetricsCollector.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/plugin/MetricsCollector.java new file mode 100644 index 000000000000..94ee09060fa2 --- /dev/null +++ b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/plugin/MetricsCollector.java @@ -0,0 +1,48 @@ +/* + * Copyright 2020 Alibaba Group Holding Limited. + * + * Licensed 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 com.alibaba.graphscope.gremlin.plugin; + +import com.alibaba.graphscope.gremlin.Utils; +import com.codahale.metrics.Timer; + +import java.util.concurrent.TimeUnit; + +// collect metrics per gremlin query +public class MetricsCollector { + private final Timer.Context timeContext; + private long startMillis; + private long elapsedMillis; + + public MetricsCollector(Timer timer) { + this.timeContext = timer.time(); + } + + public long getStartMillis() { + return this.startMillis; + } + + public long getElapsedMillis() { + return this.elapsedMillis; + } + + public void stop() { + this.startMillis = + TimeUnit.NANOSECONDS.toMillis( + Utils.getFieldValue(Timer.Context.class, timeContext, "startTime")); + this.elapsedMillis = TimeUnit.NANOSECONDS.toMillis(timeContext.stop()); + } +} diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/plugin/QueryLogger.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/plugin/QueryLogger.java new file mode 100644 index 000000000000..e8d4165fbce2 --- /dev/null +++ b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/plugin/QueryLogger.java @@ -0,0 +1,62 @@ +/* + * Copyright 2020 Alibaba Group Holding Limited. + * + * Licensed 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 com.alibaba.graphscope.gremlin.plugin; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class QueryLogger { + private static final Logger defaultLogger = LoggerFactory.getLogger(QueryLogger.class); + private static Logger metricLogger = LoggerFactory.getLogger("MetricLog"); + + private final String query; + private final long queryId; + + public QueryLogger(String query, long queryId) { + this.query = query; + this.queryId = queryId; + } + + public void info(String format, Object... args) { + defaultLogger.info(this + " : " + format, args); + } + + public void warn(String format, Object... args) { + defaultLogger.warn(this + " : " + format, args); + } + + public void error(String format, Object... args) { + defaultLogger.error(this + " : " + format, args); + } + + public void metricsInfo(String format, Object... args) { + metricLogger.info(queryId + " | " + query + " | " + format, args); + } + + @Override + public String toString() { + return "[" + "query='" + query + '\'' + ", queryId=" + queryId + ']'; + } + + public String getQuery() { + return query; + } + + public long getQueryId() { + return queryId; + } +} diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/plugin/QueryStatusCallback.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/plugin/QueryStatusCallback.java new file mode 100644 index 000000000000..00f9b17d565d --- /dev/null +++ b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/plugin/QueryStatusCallback.java @@ -0,0 +1,43 @@ +/* + * Copyright 2020 Alibaba Group Holding Limited. + * + * Licensed 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 com.alibaba.graphscope.gremlin.plugin; + +public class QueryStatusCallback { + private final MetricsCollector metricsCollector; + private final QueryLogger queryLogger; + + public QueryStatusCallback(MetricsCollector metricsCollector, QueryLogger queryLogger) { + this.metricsCollector = metricsCollector; + this.queryLogger = queryLogger; + } + + public void onStart() {} + + public void onEnd(boolean isSucceed) { + this.metricsCollector.stop(); + queryLogger.info("total execution time is {} ms", metricsCollector.getElapsedMillis()); + queryLogger.metricsInfo( + "{} | {} | {}", + isSucceed, + metricsCollector.getElapsedMillis(), + metricsCollector.getStartMillis()); + } + + public QueryLogger getQueryLogger() { + return queryLogger; + } +} diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/plugin/processor/IrStandardOpProcessor.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/plugin/processor/IrStandardOpProcessor.java index 0d1dd3c05e7f..8b7835df83f8 100644 --- a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/plugin/processor/IrStandardOpProcessor.java +++ b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/plugin/processor/IrStandardOpProcessor.java @@ -36,13 +36,15 @@ import com.alibaba.graphscope.common.store.IrMeta; import com.alibaba.graphscope.gremlin.InterOpCollectionBuilder; import com.alibaba.graphscope.gremlin.Utils; +import com.alibaba.graphscope.gremlin.plugin.MetricsCollector; +import com.alibaba.graphscope.gremlin.plugin.QueryLogger; +import com.alibaba.graphscope.gremlin.plugin.QueryStatusCallback; import com.alibaba.graphscope.gremlin.plugin.script.AntlrGremlinScriptEngineFactory; import com.alibaba.graphscope.gremlin.plugin.strategy.ExpandFusionStepStrategy; import com.alibaba.graphscope.gremlin.plugin.strategy.RemoveUselessStepStrategy; import com.alibaba.graphscope.gremlin.plugin.strategy.ScanFusionStepStrategy; import com.alibaba.graphscope.gremlin.result.processor.AbstractResultProcessor; import com.alibaba.graphscope.gremlin.result.processor.GremlinResultProcessor; -import com.alibaba.graphscope.gremlin.service.MetricsPrinter; import com.alibaba.pegasus.RpcClient; import com.alibaba.pegasus.intf.ResultProcessor; import com.alibaba.pegasus.service.protocol.PegasusClient; @@ -66,8 +68,6 @@ import org.apache.tinkerpop.gremlin.server.op.standard.StandardOpProcessor; import org.apache.tinkerpop.gremlin.structure.Graph; import org.codehaus.groovy.control.MultipleCompilationErrorsException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Map; @@ -82,9 +82,6 @@ import javax.script.SimpleBindings; public class IrStandardOpProcessor extends StandardOpProcessor { - private static Logger metricLogger = LoggerFactory.getLogger("MetricLog"); - private static Logger logger = LoggerFactory.getLogger(IrStandardOpProcessor.class); - protected static final AtomicLong JOB_ID_COUNTER = new AtomicLong(0L); protected Graph graph; protected GraphTraversalSource g; @@ -126,16 +123,10 @@ protected void evalOpInternal( long jobId = JOB_ID_COUNTER.incrementAndGet(); IrMeta irMeta = metaQueryCallback.beforeExec(); - MetricsPrinter metricsPrinter = createMetricsPrinter(jobId, script); + QueryStatusCallback statusCallback = createQueryStatusCallback(script, jobId); GremlinExecutor.LifeCycle lifeCycle = createLifeCycle( - ctx, - gremlinExecutorSupplier, - bindingsSupplier, - jobId, - script, - irMeta, - metricsPrinter); + ctx, gremlinExecutorSupplier, bindingsSupplier, irMeta, statusCallback); try { CompletableFuture evalFuture = gremlinExecutor.eval(script, language, new SimpleBindings(), lifeCycle); @@ -143,7 +134,7 @@ protected void evalOpInternal( (v, t) -> { metaQueryCallback.afterExec(irMeta); if (t != null) { - metricsPrinter.stop(false); + statusCallback.onEnd(false); if (v instanceof AbstractResultProcessor) { ((AbstractResultProcessor) v).cancel(); } @@ -174,7 +165,7 @@ protected void evalOpInternal( + " increasing the limit given to" + " TimedInterruptCustomizerProvider", msg); - logger.warn(errorMessage); + statusCallback.getQueryLogger().warn(errorMessage); ctx.writeAndFlush( ResponseMessage.build(msg) .code(ResponseStatusCode.SERVER_ERROR_TIMEOUT) @@ -190,7 +181,7 @@ protected void evalOpInternal( "Script evaluation exceeded the configured" + " threshold for request [%s]", msg); - logger.warn(errorMessage, t); + statusCallback.getQueryLogger().warn(errorMessage, t); ctx.writeAndFlush( ResponseMessage.build(msg) .code(ResponseStatusCode.SERVER_ERROR_TIMEOUT) @@ -210,7 +201,7 @@ protected void evalOpInternal( + " allowed by the JVM, please split it" + " into multiple smaller statements - %s", msg); - logger.warn(errorMessage); + statusCallback.getQueryLogger().warn(errorMessage); ctx.writeAndFlush( ResponseMessage.build(msg) .code( @@ -222,12 +213,14 @@ protected void evalOpInternal( } else { errorMessage = t.getMessage() == null ? t.toString() : t.getMessage(); - logger.warn( - String.format( - "Exception processing a script on request" - + " [%s].", - msg), - t); + statusCallback + .getQueryLogger() + .warn( + String.format( + "Exception processing a script on" + + " request [%s].", + msg), + t); ctx.writeAndFlush( ResponseMessage.build(msg) .code( @@ -250,18 +243,17 @@ protected void evalOpInternal( } } - protected MetricsPrinter createMetricsPrinter(long queryId, String script) { - return new MetricsPrinter(queryId, script, evalOpTimer, metricLogger); + protected QueryStatusCallback createQueryStatusCallback(String query, long queryId) { + return new QueryStatusCallback( + new MetricsCollector(evalOpTimer), new QueryLogger(query, queryId)); } protected GremlinExecutor.LifeCycle createLifeCycle( Context ctx, Supplier gremlinExecutorSupplier, BindingSupplier bindingsSupplier, - long jobId, - String script, IrMeta irMeta, - MetricsPrinter metricsPrinter) { + QueryStatusCallback statusCallback) { QueryTimeoutConfig timeoutConfig = new QueryTimeoutConfig(ctx.getRequestTimeout()); return GremlinExecutor.LifeCycle.build() .evaluationTimeoutOverride(timeoutConfig.getExecutionTimeoutMS()) @@ -290,11 +282,10 @@ protected GremlinExecutor.LifeCycle createLifeCycle( processTraversal( traversal, new GremlinResultProcessor( - ctx, traversal, metricsPrinter), - jobId, - script, + ctx, traversal, statusCallback), irMeta, - timeoutConfig); + timeoutConfig, + statusCallback.getQueryLogger()); } } catch (Exception e) { throw new RuntimeException(e); @@ -307,10 +298,9 @@ protected GremlinExecutor.LifeCycle createLifeCycle( protected void processTraversal( Traversal traversal, ResultProcessor resultProcessor, - long jobId, - String script, IrMeta irMeta, - QueryTimeoutConfig timeoutConfig) + QueryTimeoutConfig timeoutConfig, + QueryLogger queryLogger) throws InvalidProtocolBufferException, IOException, RuntimeException { InterOpCollection opCollection = (new InterOpCollectionBuilder(traversal)).build(); // fuse order with limit to topK @@ -318,14 +308,11 @@ protected void processTraversal( // add sink operator InterOpCollection.process(opCollection); + long jobId = queryLogger.getQueryId(); String jobName = "ir_plan_" + jobId; IrPlan irPlan = new IrPlan(irMeta, opCollection); // print script and jobName with ir plan - logger.info( - "gremlin query \"{}\", job conf name \"{}\", ir plan {}", - script, - jobName, - irPlan.getPlanAsJson()); + queryLogger.info("ir plan {}", irPlan.getPlanAsJson()); byte[] physicalPlanBytes = irPlan.toPhysicalBytes(configs); irPlan.close(); diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/result/processor/AbstractResultProcessor.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/result/processor/AbstractResultProcessor.java index ae3bb77e1a90..d52a318c95b1 100644 --- a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/result/processor/AbstractResultProcessor.java +++ b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/result/processor/AbstractResultProcessor.java @@ -17,7 +17,7 @@ package com.alibaba.graphscope.gremlin.result.processor; import com.alibaba.graphscope.common.result.ResultParser; -import com.alibaba.graphscope.gremlin.service.MetricsPrinter; +import com.alibaba.graphscope.gremlin.plugin.QueryStatusCallback; import com.alibaba.pegasus.intf.ResultProcessor; import com.alibaba.pegasus.service.protocol.PegasusClient; @@ -48,18 +48,19 @@ public abstract class AbstractResultProcessor extends StandardOpProcessor protected final Context writeResult; protected final ResultParser resultParser; - protected final MetricsPrinter metricsPrinter; + protected final QueryStatusCallback statusCallback; protected final List resultCollectors; protected final int resultCollectorsBatchSize; - protected boolean locked; + // can write back to gremlin context session if true + protected boolean isContextWritable; protected AbstractResultProcessor( - Context writeResult, ResultParser resultParser, MetricsPrinter metricsPrinter) { + Context writeResult, ResultParser resultParser, QueryStatusCallback statusCallback) { this.writeResult = writeResult; this.resultParser = resultParser; - this.metricsPrinter = metricsPrinter; + this.statusCallback = statusCallback; RequestMessage msg = writeResult.getRequestMessage(); Settings settings = writeResult.getSettings(); @@ -70,12 +71,13 @@ protected AbstractResultProcessor( msg.optionalArgs(Tokens.ARGS_BATCH_SIZE) .orElse(settings.resultIterationBatchSize); this.resultCollectors = new ArrayList<>(this.resultCollectorsBatchSize); + this.isContextWritable = true; } @Override public synchronized void process(PegasusClient.JobResponse response) { try { - if (!locked) { + if (isContextWritable) { // send back a page of results if batch size is met and then reset the // resultCollectors if (this.resultCollectors.size() >= this.resultCollectorsBatchSize) { @@ -87,10 +89,10 @@ public synchronized void process(PegasusClient.JobResponse response) { resultCollectors.addAll(resultParser.parseFrom(response)); } } catch (Exception e) { - logger.error("process response from grpc fail", e); + statusCallback.getQueryLogger().error("process response from grpc fail", e); // cannot write to this context any more - locked = true; - metricsPrinter.stop(false); + isContextWritable = false; + statusCallback.onEnd(false); writeResultList( writeResult, Collections.singletonList(e.getMessage()), @@ -100,9 +102,9 @@ public synchronized void process(PegasusClient.JobResponse response) { @Override public synchronized void finish() { - if (!locked) { - locked = true; - metricsPrinter.stop(true); + if (isContextWritable) { + isContextWritable = false; + statusCallback.onEnd(true); aggregateResults(); writeResultList(writeResult, resultCollectors, ResponseStatusCode.SUCCESS); } @@ -111,9 +113,9 @@ public synchronized void finish() { @Override public synchronized void error(Status status) { logger.error("error return from grpc, status {}", status); - if (!locked) { - locked = true; - metricsPrinter.stop(false); + if (isContextWritable) { + isContextWritable = false; + statusCallback.onEnd(false); writeResultList( writeResult, Collections.singletonList(status.toString()), @@ -122,7 +124,7 @@ public synchronized void error(Status status) { } public synchronized void cancel() { - this.locked = true; + this.isContextWritable = false; } protected abstract void aggregateResults(); diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/result/processor/GremlinResultProcessor.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/result/processor/GremlinResultProcessor.java index 234ba786d3a5..5d23ffc6ac94 100644 --- a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/result/processor/GremlinResultProcessor.java +++ b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/result/processor/GremlinResultProcessor.java @@ -16,23 +16,24 @@ package com.alibaba.graphscope.gremlin.result.processor; +import com.alibaba.graphscope.gremlin.plugin.QueryStatusCallback; import com.alibaba.graphscope.gremlin.result.GremlinResultAnalyzer; import com.alibaba.graphscope.gremlin.result.GroupResultParser; -import com.alibaba.graphscope.gremlin.service.MetricsPrinter; import org.apache.tinkerpop.gremlin.process.traversal.Traversal; import org.apache.tinkerpop.gremlin.server.Context; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.*; +import java.util.LinkedHashMap; +import java.util.Map; public class GremlinResultProcessor extends AbstractResultProcessor { private static Logger logger = LoggerFactory.getLogger(GremlinResultProcessor.class); public GremlinResultProcessor( - Context writeResult, Traversal traversal, MetricsPrinter metricsPrinter) { - super(writeResult, GremlinResultAnalyzer.analyze(traversal), metricsPrinter); + Context writeResult, Traversal traversal, QueryStatusCallback statusCallback) { + super(writeResult, GremlinResultAnalyzer.analyze(traversal), statusCallback); } // format group result as a single map diff --git a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/service/MetricsPrinter.java b/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/service/MetricsPrinter.java deleted file mode 100644 index a2046d40a0d6..000000000000 --- a/interactive_engine/compiler/src/main/java/com/alibaba/graphscope/gremlin/service/MetricsPrinter.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Copyright 2020 Alibaba Group Holding Limited. - * - * Licensed 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 com.alibaba.graphscope.gremlin.service; - -import com.alibaba.graphscope.gremlin.Utils; -import com.codahale.metrics.Timer; - -import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.concurrent.TimeUnit; - -// collect metrics per gremlin query and print to logs -public class MetricsPrinter { - private static final Logger logger = LoggerFactory.getLogger(MetricsPrinter.class); - private final long queryId; - private final String query; - private final Timer.Context timeContext; - private final @Nullable Logger extraLogger; - - public MetricsPrinter(long queryId, String query, Timer timer, @Nullable Logger extraLogger) { - this.queryId = queryId; - this.query = query; - this.timeContext = timer.time(); - this.extraLogger = extraLogger; - } - - public void stop(boolean isSucceed) { - long startMillis = - TimeUnit.NANOSECONDS.toMillis( - Utils.getFieldValue(Timer.Context.class, timeContext, "startTime")); - long elapsedMillis = TimeUnit.NANOSECONDS.toMillis(timeContext.stop()); - logger.info("query \"{}\" total execution time is {} ms", query, elapsedMillis); - if (extraLogger != null) { - extraLogger.info( - "{} | {} | {} | {} | {}", - queryId, - query, - isSucceed, - elapsedMillis, - startMillis); - } - } -}