-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Report connector provided read time as connector metric #10472
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,6 +29,7 @@ | |
| import io.trino.memory.context.LocalMemoryContext; | ||
| import io.trino.memory.context.MemoryTrackingContext; | ||
| import io.trino.operator.OperationTimer.OperationTiming; | ||
| import io.trino.plugin.base.metrics.DurationTiming; | ||
| import io.trino.plugin.base.metrics.TDigestHistogram; | ||
| import io.trino.spi.Page; | ||
| import io.trino.spi.TrinoException; | ||
|
|
@@ -74,6 +75,7 @@ public class OperatorContext | |
|
|
||
| private final CounterStat physicalInputDataSize = new CounterStat(); | ||
| private final CounterStat physicalInputPositions = new CounterStat(); | ||
| private final AtomicLong physicalInputReadTimeNanos = new AtomicLong(); | ||
|
|
||
| private final CounterStat internalNetworkInputDataSize = new CounterStat(); | ||
| private final CounterStat internalNetworkPositions = new CounterStat(); | ||
|
|
@@ -182,7 +184,7 @@ public void recordPhysicalInputWithTiming(long sizeInBytes, long positions, long | |
| { | ||
| physicalInputDataSize.update(sizeInBytes); | ||
| physicalInputPositions.update(positions); | ||
| addInputTiming.record(readNanos, 0); | ||
| physicalInputReadTimeNanos.getAndAdd(readNanos); | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -532,6 +534,16 @@ public static Metrics getOperatorMetrics(Metrics operatorMetrics, long inputPosi | |
| return operatorMetrics.mergeWith(new Metrics(ImmutableMap.of("Input distribution", new TDigestHistogram(digest)))); | ||
| } | ||
|
|
||
| public static Metrics getConnectorMetrics(Metrics connectorMetrics, long physicalInputReadTimeNanos) | ||
| { | ||
| if (physicalInputReadTimeNanos == 0) { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am not a big fan of using 0 as a special marker. Can we use OptionalLong. Or if maybe -1 so we use value which is out of domain for valid values?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I will leave it as is. It's not a maker really, but rather skipping meaningless stat. If I used |
||
| return connectorMetrics; | ||
| } | ||
|
|
||
| return connectorMetrics.mergeWith(new Metrics(ImmutableMap.of( | ||
| "Physical input read time", new DurationTiming(new Duration(physicalInputReadTimeNanos, NANOSECONDS))))); | ||
| } | ||
|
|
||
| public <C, R> R accept(QueryContextVisitor<C, R> visitor, C context) | ||
| { | ||
| return visitor.visitOperatorContext(this, context); | ||
|
|
@@ -573,7 +585,7 @@ private OperatorStats getOperatorStats() | |
|
|
||
| dynamicFilterSplitsProcessed.get(), | ||
| getOperatorMetrics(metrics.get(), inputPositionsCount), | ||
| connectorMetrics.get(), | ||
| getConnectorMetrics(connectorMetrics.get(), physicalInputReadTimeNanos.get()), | ||
|
|
||
| DataSize.ofBytes(physicalWrittenDataSize.get()), | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,22 @@ | ||
| /* | ||
| * 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 io.trino.spi.metrics; | ||
|
|
||
| import java.time.Duration; | ||
|
|
||
| public interface Timing<T> | ||
| extends Metric<T> | ||
| { | ||
| Duration getDuration(); | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,86 @@ | ||
| /* | ||
| * 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 io.trino.plugin.base.metrics; | ||
|
|
||
| import com.fasterxml.jackson.annotation.JsonCreator; | ||
| import com.fasterxml.jackson.annotation.JsonProperty; | ||
| import io.airlift.units.Duration; | ||
| import io.trino.spi.metrics.Timing; | ||
|
|
||
| import java.util.Objects; | ||
|
|
||
| import static com.google.common.base.MoreObjects.toStringHelper; | ||
| import static java.util.Objects.requireNonNull; | ||
| import static java.util.concurrent.TimeUnit.NANOSECONDS; | ||
|
|
||
| public class DurationTiming | ||
| implements Timing<DurationTiming> | ||
| { | ||
| // use Airlift duration for more human-friendly serialization format | ||
| // and to match duration serialization in other JSON objects | ||
| private final Duration duration; | ||
|
|
||
| @JsonCreator | ||
| public DurationTiming(Duration duration) | ||
| { | ||
| this.duration = requireNonNull(duration, "duration is null"); | ||
| } | ||
|
|
||
| @JsonProperty("duration") | ||
| public Duration getAirliftDuration() | ||
| { | ||
| return duration; | ||
| } | ||
|
|
||
| @Override | ||
| public java.time.Duration getDuration() | ||
|
sopel39 marked this conversation as resolved.
Outdated
|
||
| { | ||
| return java.time.Duration.ofNanos(duration.roundTo(NANOSECONDS)); | ||
| } | ||
|
|
||
| @Override | ||
| public DurationTiming mergeWith(DurationTiming other) | ||
| { | ||
| long durationNanos = duration.roundTo(NANOSECONDS); | ||
| long otherDurationNanos = other.getAirliftDuration().roundTo(NANOSECONDS); | ||
| return new DurationTiming(new Duration(durationNanos + otherDurationNanos, NANOSECONDS).convertToMostSuccinctTimeUnit()); | ||
| } | ||
|
|
||
| @Override | ||
| public boolean equals(Object o) | ||
| { | ||
| if (this == o) { | ||
| return true; | ||
| } | ||
| if (o == null || getClass() != o.getClass()) { | ||
| return false; | ||
| } | ||
| DurationTiming that = (DurationTiming) o; | ||
| return duration.equals(that.duration); | ||
| } | ||
|
|
||
| @Override | ||
| public int hashCode() | ||
| { | ||
| return Objects.hash(duration); | ||
| } | ||
|
|
||
| @Override | ||
| public String toString() | ||
| { | ||
| return toStringHelper("") | ||
| .add("duration", duration) | ||
| .toString(); | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.