diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java index b2d4e24d3f32..8173f7a53330 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java @@ -761,7 +761,7 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S realtimeRoutingTable = null; } } - BrokerResponseNative brokerResponse; + BrokerResponse brokerResponse; if (_queriesById != null) { // Start to track the running query for cancellation just before sending it out to servers to avoid any // potential failures that could happen before sending it out, like failures to calculate the routing table etc. @@ -798,7 +798,6 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S _brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.BROKER_RESPONSES_WITH_NUM_GROUPS_LIMIT_REACHED, 1); } - brokerResponse.setPartialResult(isPartialResult(brokerResponse)); // Set total query processing time long totalTimeMs = TimeUnit.NANOSECONDS.toMillis(executionEndTimeNs - compilationStartTimeNs); @@ -1870,7 +1869,7 @@ private static void attachTimeBoundary(PinotQuery pinotQuery, TimeBoundaryInfo t * Processes the optimized broker requests for both OFFLINE and REALTIME table. * TODO: Directly take PinotQuery */ - protected abstract BrokerResponseNative processBrokerRequest(long requestId, BrokerRequest originalBrokerRequest, + protected abstract BrokerResponse processBrokerRequest(long requestId, BrokerRequest originalBrokerRequest, BrokerRequest serverBrokerRequest, @Nullable BrokerRequest offlineBrokerRequest, @Nullable Map, List>> offlineRoutingTable, @Nullable BrokerRequest realtimeBrokerRequest, @@ -1878,11 +1877,6 @@ protected abstract BrokerResponseNative processBrokerRequest(long requestId, Bro ServerStats serverStats, RequestContext requestContext) throws Exception; - protected static boolean isPartialResult(BrokerResponse brokerResponse) { - return brokerResponse.isNumGroupsLimitReached() || brokerResponse.isMaxRowsInJoinReached() - || brokerResponse.getExceptionsSize() > 0; - } - protected static void augmentStatistics(RequestContext statistics, BrokerResponse response) { statistics.setTotalDocs(response.getTotalDocs()); statistics.setNumDocsScanned(response.getNumDocsScanned()); diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java index 01bfe456b5b1..078ef0651c23 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java @@ -19,14 +19,13 @@ package org.apache.pinot.broker.requesthandler; import com.fasterxml.jackson.databind.JsonNode; -import com.google.common.collect.Maps; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; import javax.annotation.Nullable; import javax.ws.rs.WebApplicationException; import javax.ws.rs.core.HttpHeaders; @@ -48,7 +47,6 @@ import org.apache.pinot.common.response.BrokerResponse; import org.apache.pinot.common.response.broker.BrokerResponseNative; import org.apache.pinot.common.response.broker.BrokerResponseNativeV2; -import org.apache.pinot.common.response.broker.BrokerResponseStats; import org.apache.pinot.common.response.broker.QueryProcessingException; import org.apache.pinot.common.response.broker.ResultTable; import org.apache.pinot.common.utils.DataSchema; @@ -58,14 +56,17 @@ import org.apache.pinot.common.utils.request.RequestUtils; import org.apache.pinot.core.auth.Actions; import org.apache.pinot.core.auth.TargetType; -import org.apache.pinot.core.query.reduce.ExecutionStatsAggregator; import org.apache.pinot.core.transport.ServerInstance; import org.apache.pinot.query.QueryEnvironment; import org.apache.pinot.query.catalog.PinotCatalog; import org.apache.pinot.query.mailbox.MailboxService; +import org.apache.pinot.query.planner.PlanFragment; import org.apache.pinot.query.planner.physical.DispatchablePlanFragment; import org.apache.pinot.query.planner.physical.DispatchableSubPlan; +import org.apache.pinot.query.planner.plannode.PlanNode; import org.apache.pinot.query.routing.WorkerManager; +import org.apache.pinot.query.runtime.MultiStageStatsTreeBuilder; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; import org.apache.pinot.query.service.dispatch.QueryDispatcher; import org.apache.pinot.query.type.TypeFactory; import org.apache.pinot.query.type.TypeSystem; @@ -195,24 +196,12 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S } Map queryOptions = sqlNodeAndOptions.getOptions(); - boolean traceEnabled = Boolean.parseBoolean(queryOptions.get(CommonConstants.Broker.Request.TRACE)); - Map stageIdStatsMap; - if (!traceEnabled) { - stageIdStatsMap = Collections.singletonMap(0, new ExecutionStatsAggregator(false)); - } else { - List stagePlans = dispatchableSubPlan.getQueryStageList(); - int numStages = stagePlans.size(); - stageIdStatsMap = Maps.newHashMapWithExpectedSize(numStages); - for (int stageId = 0; stageId < numStages; stageId++) { - stageIdStatsMap.put(stageId, new ExecutionStatsAggregator(true)); - } - } long executionStartTimeNs = System.nanoTime(); - ResultTable queryResults; + QueryDispatcher.QueryResult queryResults; try { - queryResults = _queryDispatcher.submitAndReduce(requestContext, dispatchableSubPlan, queryTimeoutMs, queryOptions, - stageIdStatsMap); + queryResults = + _queryDispatcher.submitAndReduce(requestContext, dispatchableSubPlan, queryTimeoutMs, queryOptions); } catch (TimeoutException e) { for (String table : tableNames) { _brokerMetrics.addMeteredTableValue(table, BrokerMeter.BROKER_RESPONSES_WITH_TIMEOUTS, 1); @@ -231,7 +220,7 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S updatePhaseTimingForTables(tableNames, BrokerQueryPhase.QUERY_EXECUTION, executionEndTimeNs - executionStartTimeNs); BrokerResponseNativeV2 brokerResponse = new BrokerResponseNativeV2(); - brokerResponse.setResultTable(queryResults); + brokerResponse.setResultTable(queryResults.getResultTable()); // Attach unavailable segments int numUnavailableSegments = 0; @@ -243,27 +232,7 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S String.format("Find unavailable segments: %s for table: %s", unavailableSegments, tableName))); } - for (Map.Entry entry : stageIdStatsMap.entrySet()) { - if (entry.getKey() == 0) { - // Root stats are aggregated and added separately to broker response for backward compatibility - entry.getValue().setStats(brokerResponse); - continue; - } - - BrokerResponseStats brokerResponseStats = new BrokerResponseStats(); - if (!tableNames.isEmpty()) { - //TODO: Only using first table to assign broker metrics - // find a way to split metrics in case of multiple table - String rawTableName = TableNameBuilder.extractRawTableName(tableNames.iterator().next()); - entry.getValue().setStageLevelStats(rawTableName, brokerResponseStats, _brokerMetrics); - } else { - entry.getValue().setStageLevelStats(null, brokerResponseStats, null); - } - brokerResponse.addStageStat(entry.getKey(), brokerResponseStats); - } - - // Set partial result flag - brokerResponse.setPartialResult(isPartialResult(brokerResponse)); + fillOldBrokerResponseStats(brokerResponse, queryResults.getQueryStats(), dispatchableSubPlan); // Set total query processing time // TODO: Currently we don't emit metric for QUERY_TOTAL_TIME_MS @@ -271,7 +240,6 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S sqlNodeAndOptions.getParseTimeNs() + (executionEndTimeNs - compilationStartTimeNs)); brokerResponse.setTimeUsedMs(totalTimeMs); requestContext.setQueryProcessingTime(totalTimeMs); - requestContext.setTraceInfo(brokerResponse.getTraceInfo()); augmentStatistics(requestContext, brokerResponse); // Log query and stats @@ -282,6 +250,22 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S return brokerResponse; } + private void fillOldBrokerResponseStats(BrokerResponseNativeV2 brokerResponse, + List queryStats, DispatchableSubPlan dispatchableSubPlan) { + List planNodes = dispatchableSubPlan.getQueryStageList().stream() + .map(DispatchablePlanFragment::getPlanFragment) + .map(PlanFragment::getFragmentRoot) + .collect(Collectors.toList()); + MultiStageStatsTreeBuilder treeBuilder = new MultiStageStatsTreeBuilder(planNodes, queryStats); + brokerResponse.setStageStats(treeBuilder.jsonStatsByStage(0)); + + for (MultiStageQueryStats.StageStats.Closed stageStats : queryStats) { + if (stageStats != null) { // for example pipeline breaker may not have stats + stageStats.forEach((type, stats) -> type.mergeInto(brokerResponse, stats)); + } + } + } + /** * Validates whether the requester has access to all the tables. */ @@ -325,7 +309,7 @@ private void updatePhaseTimingForTables(Set tableNames, BrokerQueryPhase } } - private BrokerResponseNative constructMultistageExplainPlan(String sql, String plan) { + private BrokerResponse constructMultistageExplainPlan(String sql, String plan) { BrokerResponseNative brokerResponse = BrokerResponseNative.empty(); List rows = new ArrayList<>(); rows.add(new Object[]{sql, plan}); @@ -336,7 +320,7 @@ private BrokerResponseNative constructMultistageExplainPlan(String sql, String p } @Override - protected BrokerResponseNative processBrokerRequest(long requestId, BrokerRequest originalBrokerRequest, + protected BrokerResponse processBrokerRequest(long requestId, BrokerRequest originalBrokerRequest, BrokerRequest serverBrokerRequest, @Nullable BrokerRequest offlineBrokerRequest, @Nullable Map, List>> offlineRoutingTable, @Nullable BrokerRequest realtimeBrokerRequest, diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java index 68ae70f9eb57..b305f060faf2 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java @@ -40,6 +40,7 @@ import org.apache.pinot.common.metrics.BrokerMetrics; import org.apache.pinot.common.metrics.BrokerQueryPhase; import org.apache.pinot.common.request.BrokerRequest; +import org.apache.pinot.common.response.BrokerResponse; import org.apache.pinot.common.response.broker.BrokerResponseNative; import org.apache.pinot.common.response.broker.QueryProcessingException; import org.apache.pinot.common.utils.HashUtil; @@ -100,7 +101,7 @@ public synchronized void shutDown() { } @Override - protected BrokerResponseNative processBrokerRequest(long requestId, BrokerRequest originalBrokerRequest, + protected BrokerResponse processBrokerRequest(long requestId, BrokerRequest originalBrokerRequest, BrokerRequest serverBrokerRequest, @Nullable BrokerRequest offlineBrokerRequest, @Nullable Map, List>> offlineRoutingTable, @Nullable BrokerRequest realtimeBrokerRequest, diff --git a/pinot-common/src/main/java/org/apache/pinot/common/datablock/BaseDataBlock.java b/pinot-common/src/main/java/org/apache/pinot/common/datablock/BaseDataBlock.java index 8c6410b75d86..333ac673b6e9 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/datablock/BaseDataBlock.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/datablock/BaseDataBlock.java @@ -93,7 +93,6 @@ public abstract class BaseDataBlock implements DataBlock { protected ByteBuffer _fixedSizeData; protected byte[] _variableSizeDataBytes; protected ByteBuffer _variableSizeData; - protected Map _metadata; /** * construct a base data block. @@ -114,7 +113,6 @@ public BaseDataBlock(int numRows, @Nullable DataSchema dataSchema, String[] stri _fixedSizeData = ByteBuffer.wrap(fixedSizeDataBytes); _variableSizeDataBytes = variableSizeDataBytes; _variableSizeData = ByteBuffer.wrap(variableSizeDataBytes); - _metadata = new HashMap<>(); _errCodeToExceptionMap = new HashMap<>(); } @@ -131,7 +129,6 @@ public BaseDataBlock() { _fixedSizeData = null; _variableSizeDataBytes = null; _variableSizeData = null; - _metadata = new HashMap<>(); _errCodeToExceptionMap = new HashMap<>(); } @@ -195,10 +192,7 @@ public BaseDataBlock(ByteBuffer byteBuffer) _variableSizeData = ByteBuffer.wrap(_variableSizeDataBytes); // Read metadata. - int metadataLength = byteBuffer.getInt(); - if (metadataLength != 0) { - _metadata = deserializeMetadata(byteBuffer); - } + deserializeMetadata(byteBuffer); } @Override @@ -232,7 +226,7 @@ public int getVersion() { @Override public Map getMetadata() { - return _metadata; + return Collections.emptyMap(); } @Override @@ -432,6 +426,11 @@ public Map getExceptions() { return _errCodeToExceptionMap; } + /** + * Serialize this data block to a byte array. + *

+ * In order to deserialize it, {@link DataBlockUtils#getDataBlock(ByteBuffer)} should be used. + */ @Override public byte[] toBytes() throws IOException { @@ -444,9 +443,7 @@ public byte[] toBytes() // Write metadata: length followed by actual metadata bytes. // NOTE: We ignore metadata serialization time in "responseSerializationCpuTimeNs" as it's negligible while // considering it will bring a lot code complexity. - byte[] metadataBytes = serializeMetadata(); - dataOutputStream.writeInt(metadataBytes.length); - dataOutputStream.write(metadataBytes); + serializeMetadata(dataOutputStream); return byteArrayOutputStream.toByteArray(); } @@ -525,14 +522,30 @@ private void writeLeadingSections(DataOutputStream dataOutputStream) } } - private byte[] serializeMetadata() + /** + * Writes the metadata section to the given data output stream. + */ + protected void serializeMetadata(DataOutputStream dataOutputStream) throws IOException { - return new byte[0]; + dataOutputStream.writeInt(0); } - private Map deserializeMetadata(ByteBuffer buffer) + /** + * Deserializes the metadata section from the given byte buffer. + *

+ * This is the counterpart of {@link #serializeMetadata(DataOutputStream)} and it is guaranteed that the buffer will + * be positioned at the start of the metadata section when this method is called. + *

+ * Important: It is mandatory for implementations to leave the cursor at the end of the metadata, in + * the exact same position as it was when {@link #serializeMetadata(DataOutputStream)} was called. + *

+ * Important: This method will be called at the end of the BaseDataConstructor constructor to read + * the metadata section. This means that it will be called before the subclass have been constructor + * have been called. Therefore it is not possible to use any subclass fields in this method. + */ + protected void deserializeMetadata(ByteBuffer buffer) throws IOException { - return Collections.emptyMap(); + buffer.getInt(); } private byte[] serializeExceptions() @@ -572,14 +585,9 @@ private Map deserializeExceptions(ByteBuffer buffer) @Override public String toString() { if (_dataSchema == null) { - return _metadata.toString(); + return "{}"; } else { - StringBuilder stringBuilder = new StringBuilder(); - stringBuilder.append("resultSchema:").append('\n'); - stringBuilder.append(_dataSchema).append('\n'); - stringBuilder.append("numRows: ").append(_numRows).append('\n'); - stringBuilder.append("metadata: ").append(_metadata.toString()).append('\n'); - return stringBuilder.toString(); + return "resultSchema:" + '\n' + _dataSchema + '\n' + "numRows: " + _numRows + '\n'; } } } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/datablock/ColumnarDataBlock.java b/pinot-common/src/main/java/org/apache/pinot/common/datablock/ColumnarDataBlock.java index 216f4d9d9131..51b01e6b69bf 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/datablock/ColumnarDataBlock.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/datablock/ColumnarDataBlock.java @@ -20,6 +20,8 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Objects; import org.apache.pinot.common.utils.DataSchema; @@ -27,7 +29,7 @@ * Column-wise data table. It stores data in columnar-major format. */ public class ColumnarDataBlock extends BaseDataBlock { - private static final int VERSION = 1; + private static final int VERSION = 2; protected int[] _cumulativeColumnOffsetSizeInBytes; protected int[] _columnSizeInBytes; @@ -80,17 +82,21 @@ protected int positionOffsetInVariableBufferAndGetLength(int rowId, int colId) { } @Override - public ColumnarDataBlock toMetadataOnlyDataTable() { - ColumnarDataBlock metadataOnlyDataTable = new ColumnarDataBlock(); - metadataOnlyDataTable._metadata.putAll(_metadata); - metadataOnlyDataTable._errCodeToExceptionMap.putAll(_errCodeToExceptionMap); - return metadataOnlyDataTable; + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof ColumnarDataBlock)) { + return false; + } + ColumnarDataBlock that = (ColumnarDataBlock) o; + return Objects.deepEquals(_cumulativeColumnOffsetSizeInBytes, that._cumulativeColumnOffsetSizeInBytes) + && Objects.deepEquals(_columnSizeInBytes, that._columnSizeInBytes); } @Override - public ColumnarDataBlock toDataOnlyDataTable() { - return new ColumnarDataBlock(_numRows, _dataSchema, _stringDictionary, _fixedSizeDataBytes, _variableSizeDataBytes); + public int hashCode() { + return Objects.hash(Arrays.hashCode(_cumulativeColumnOffsetSizeInBytes), Arrays.hashCode(_columnSizeInBytes)); } - - // TODO: add whole-column access methods. +// TODO: add whole-column access methods. } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/datablock/DataBlock.java b/pinot-common/src/main/java/org/apache/pinot/common/datablock/DataBlock.java index 418426b4ac6b..768d6cdb885d 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/datablock/DataBlock.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/datablock/DataBlock.java @@ -80,10 +80,6 @@ byte[] toBytes() @Nullable RoaringBitmap getNullRowIds(int colId); - DataBlock toMetadataOnlyDataTable(); - - DataBlock toDataOnlyDataTable(); - enum Type { ROW(0), COLUMNAR(1), diff --git a/pinot-common/src/main/java/org/apache/pinot/common/datablock/DataBlockUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/datablock/DataBlockUtils.java index 27f114032849..1a67f980b115 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/datablock/DataBlockUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/datablock/DataBlockUtils.java @@ -26,15 +26,20 @@ import org.apache.pinot.common.response.ProcessingException; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public final class DataBlockUtils { + private static final Logger LOGGER = LoggerFactory.getLogger(DataBlockUtils.class); + private DataBlockUtils() { } static final int VERSION_TYPE_SHIFT = 5; public static MetadataBlock getErrorDataBlock(Exception e) { + LOGGER.info("Caught exception while processing query", e); if (e instanceof ProcessingException) { return getErrorDataBlock(Collections.singletonMap(((ProcessingException) e).getErrorCode(), extractErrorMsg(e))); } else { @@ -51,33 +56,40 @@ private static String extractErrorMsg(Throwable t) { } public static MetadataBlock getErrorDataBlock(Map exceptions) { - MetadataBlock errorBlock = new MetadataBlock(MetadataBlock.MetadataBlockType.ERROR); - for (Map.Entry exception : exceptions.entrySet()) { - errorBlock.addException(exception.getKey(), exception.getValue()); - } - return errorBlock; + return MetadataBlock.newError(exceptions); + } + + /** + * Reads an integer from the given byte buffer. + *

+ * The returned integer contains both the version and the type of the data block. + * {@link #getVersion(int)} and {@link #getType(int)} can be used to extract the version and the type. + * @param byteBuffer byte buffer to read from. A single int will be read + */ + public static int readVersionType(ByteBuffer byteBuffer) { + return byteBuffer.getInt(); } - public static MetadataBlock getEndOfStreamDataBlock() { - return new MetadataBlock(MetadataBlock.MetadataBlockType.EOS); + public static int getVersion(int versionType) { + return versionType & ((1 << VERSION_TYPE_SHIFT) - 1); } - public static MetadataBlock getEndOfStreamDataBlock(Map stats) { - return new MetadataBlock(MetadataBlock.MetadataBlockType.EOS, stats); + public static DataBlock.Type getType(int versionType) { + return DataBlock.Type.fromOrdinal(versionType >> VERSION_TYPE_SHIFT); } public static DataBlock getDataBlock(ByteBuffer byteBuffer) throws IOException { - int versionType = byteBuffer.getInt(); - int version = versionType & ((1 << VERSION_TYPE_SHIFT) - 1); - DataBlock.Type type = DataBlock.Type.fromOrdinal(versionType >> VERSION_TYPE_SHIFT); + int versionType = readVersionType(byteBuffer); + int version = getVersion(versionType); + DataBlock.Type type = getType(versionType); switch (type) { case COLUMNAR: return new ColumnarDataBlock(byteBuffer); case ROW: return new RowDataBlock(byteBuffer); case METADATA: - return new MetadataBlock(byteBuffer); + return MetadataBlock.deserialize(byteBuffer, version); default: throw new UnsupportedOperationException("Unsupported data table version: " + version + " with type: " + type); } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/datablock/MetadataBlock.java b/pinot-common/src/main/java/org/apache/pinot/common/datablock/MetadataBlock.java index 41c2e9d0ea8c..164ca5157701 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/datablock/MetadataBlock.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/datablock/MetadataBlock.java @@ -18,16 +18,19 @@ */ package org.apache.pinot.common.datablock; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonIgnoreProperties; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; +import java.io.DataOutputStream; import java.io.IOException; +import java.nio.BufferUnderflowException; import java.nio.ByteBuffer; -import java.util.HashMap; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import java.util.Map; +import java.util.Objects; +import javax.annotation.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -36,108 +39,116 @@ */ public class MetadataBlock extends BaseDataBlock { - private static final ObjectMapper JSON = new ObjectMapper(); - + private static final Logger LOGGER = LoggerFactory.getLogger(MetadataBlock.class); @VisibleForTesting - static final int VERSION = 1; - - public enum MetadataBlockType { - /** - * Indicates that this block is the final block to be sent - * (End Of Stream) as part of an operator chain computation. - */ - EOS, + static final int VERSION = 2; + @Nullable + private List _statsByStage; - /** - * An {@code ERROR} metadata block indicates that there was - * some error during computation. To retrieve the error that - * occurred, use {@link MetadataBlock#getExceptions()} - */ - ERROR + private MetadataBlock() { + this(Collections.emptyList()); } - /** - * Used to serialize the contents of the metadata block conveniently and in - * a backwards compatible way. Use JSON because the performance of metadata block - * SerDe should not be a bottleneck. - */ - @JsonIgnoreProperties(ignoreUnknown = true) - @VisibleForTesting - static class Contents { - - private String _type; - private Map _stats; - - @JsonCreator - public Contents(@JsonProperty("type") String type, @JsonProperty("stats") Map stats) { - _type = type; - _stats = stats; - } - - @JsonCreator - public Contents() { - this(null, new HashMap<>()); - } - - public String getType() { - return _type; - } - - public void setType(String type) { - _type = type; - } - - public Map getStats() { - return _stats; - } + public static MetadataBlock newEos() { + return new MetadataBlock(); + } - public void setStats(Map stats) { - _stats = stats; + public static MetadataBlock newError(Map exceptions) { + MetadataBlock errorBlock = new MetadataBlock(); + for (Map.Entry exception : exceptions.entrySet()) { + errorBlock.addException(exception.getKey(), exception.getValue()); } + return errorBlock; } - private final Contents _contents; + public MetadataBlock(List statsByStage) { + super(0, null, new String[0], new byte[0], new byte[0]); + _statsByStage = statsByStage; + } - public MetadataBlock(MetadataBlockType type) { - this(type, new HashMap<>()); + MetadataBlock(ByteBuffer byteBuffer) + throws IOException { + super(byteBuffer); } - public MetadataBlock(MetadataBlockType type, Map stats) { - super(0, null, new String[0], new byte[]{0}, toContents(new Contents(type.name(), stats))); - _contents = new Contents(type.name(), stats); + @Override + protected void serializeMetadata(DataOutputStream output) + throws IOException { + if (_statsByStage == null) { + output.writeInt(0); + return; + } + int size = _statsByStage.size(); + output.writeInt(size); + if (size > 0) { + byte[] bytes = new byte[4096]; + for (ByteBuffer stat : _statsByStage) { + if (stat == null) { + output.writeBoolean(false); + } else { + output.writeBoolean(true); + output.writeInt(stat.remaining()); + ByteBuffer duplicate = stat.duplicate(); + while (duplicate.hasRemaining()) { + int length = Math.min(duplicate.remaining(), bytes.length); + duplicate.get(bytes, 0, length); + output.write(bytes, 0, length); + } + } + } + } } - private static byte[] toContents(Contents type) { - try { - return JSON.writeValueAsBytes(type); - } catch (JsonProcessingException e) { - throw new RuntimeException(e); + public static MetadataBlock deserialize(ByteBuffer byteBuffer, int version) + throws IOException { + switch (version) { + case 1: + case 2: + return new MetadataBlock(byteBuffer); + default: + throw new IOException("Unsupported metadata block version: " + version); } } - public MetadataBlock(ByteBuffer byteBuffer) + @Override + protected void deserializeMetadata(ByteBuffer buffer) throws IOException { - super(byteBuffer); - if (_variableSizeDataBytes != null && _variableSizeDataBytes.length > 0) { - _contents = JSON.readValue(_variableSizeDataBytes, Contents.class); - } else { - _contents = new Contents(); + try { + int statsSize = buffer.getInt(); + + List stats = new ArrayList<>(statsSize); + + for (int i = 0; i < statsSize; i++) { + if (buffer.get() != 0) { + int length = buffer.getInt(); + buffer.limit(buffer.position() + length); + stats.add(buffer.slice()); + buffer.position(buffer.limit()); + buffer.limit(buffer.capacity()); + } else { + stats.add(null); + } + } + _statsByStage = stats; + } catch (BufferUnderflowException e) { + LOGGER.info("Failed to read stats from metadata block. Considering it empty", e);; + } catch (RuntimeException e) { + LOGGER.warn("Failed to read stats from metadata block. Considering it empty", e);; } } public MetadataBlockType getType() { - String type = _contents.getType(); - - // if type is null, then we're reading a legacy block where we didn't encode any - // data. assume that it is an EOS block if there's no exceptions and an ERROR block - // otherwise - return type == null - ? (getExceptions().isEmpty() ? MetadataBlockType.EOS : MetadataBlockType.ERROR) - : MetadataBlockType.valueOf(type); + return _errCodeToExceptionMap.isEmpty() ? MetadataBlockType.EOS : MetadataBlockType.ERROR; } - public Map getStats() { - return _contents.getStats() != null ? _contents.getStats() : new HashMap<>(); + /** + * Returns the list of serialized stats. + *

+ * The returned list may contain nulls, which would mean that no stats were available for that stage. + */ + @Nullable + public List getStatsByStage() { + return _statsByStage; } @Override @@ -156,12 +167,35 @@ protected int positionOffsetInVariableBufferAndGetLength(int rowId, int colId) { } @Override - public MetadataBlock toMetadataOnlyDataTable() { - return this; + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof MetadataBlock)) { + return false; + } + MetadataBlock that = (MetadataBlock) o; + return Objects.equals(_statsByStage, that._statsByStage) + && _errCodeToExceptionMap.equals(that._errCodeToExceptionMap); } @Override - public MetadataBlock toDataOnlyDataTable() { - throw new UnsupportedOperationException(); + public int hashCode() { + return Objects.hash(_statsByStage, _errCodeToExceptionMap); + } + + public enum MetadataBlockType { + /** + * Indicates that this block is the final block to be sent + * (End Of Stream) as part of an operator chain computation. + */ + EOS, + + /** + * An {@code ERROR} metadata block indicates that there was + * some error during computation. To retrieve the error that + * occurred, use {@link MetadataBlock#getExceptions()} + */ + ERROR } } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/datablock/RowDataBlock.java b/pinot-common/src/main/java/org/apache/pinot/common/datablock/RowDataBlock.java index f5aa80648936..33a5e77cf764 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/datablock/RowDataBlock.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/datablock/RowDataBlock.java @@ -20,6 +20,8 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Objects; import org.apache.pinot.common.utils.DataSchema; @@ -27,7 +29,7 @@ * Wrapper for row-wise data table. It stores data in row-major format. */ public class RowDataBlock extends BaseDataBlock { - private static final int VERSION = 1; + private static final int VERSION = 2; protected int[] _columnOffsets; protected int _rowSizeInBytes; @@ -72,22 +74,25 @@ protected int positionOffsetInVariableBufferAndGetLength(int rowId, int colId) { return _fixedSizeData.getInt(offset + 4); } - @Override - public RowDataBlock toMetadataOnlyDataTable() { - RowDataBlock metadataOnlyDataTable = new RowDataBlock(); - metadataOnlyDataTable._metadata.putAll(_metadata); - metadataOnlyDataTable._errCodeToExceptionMap.putAll(_errCodeToExceptionMap); - return metadataOnlyDataTable; + public int getRowSizeInBytes() { + return _rowSizeInBytes; } @Override - public RowDataBlock toDataOnlyDataTable() { - return new RowDataBlock(_numRows, _dataSchema, _stringDictionary, _fixedSizeDataBytes, _variableSizeDataBytes); + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof RowDataBlock)) { + return false; + } + RowDataBlock that = (RowDataBlock) o; + return _rowSizeInBytes == that._rowSizeInBytes && Objects.deepEquals(_columnOffsets, that._columnOffsets); } - public int getRowSizeInBytes() { - return _rowSizeInBytes; + @Override + public int hashCode() { + return Objects.hash(Arrays.hashCode(_columnOffsets), _rowSizeInBytes); } - - // TODO: add whole-row access methods. +// TODO: add whole-row access methods. } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTable.java b/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTable.java index 72121611eb04..5fb1018dceae 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTable.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTable.java @@ -110,6 +110,8 @@ enum MetadataKey { NUM_SEGMENTS_PROCESSED(6, "numSegmentsProcessed", MetadataValueType.INT), NUM_SEGMENTS_MATCHED(7, "numSegmentsMatched", MetadataValueType.INT), NUM_CONSUMING_SEGMENTS_QUERIED(8, "numConsumingSegmentsQueried", MetadataValueType.INT), + // the timestamp indicating the freshness of the data queried in consuming segments. + // This can be ingestion timestamp if provided by the stream, or the last index time MIN_CONSUMING_FRESHNESS_TIME_MS(9, "minConsumingFreshnessTimeMs", MetadataValueType.LONG), TOTAL_DOCS(10, "totalDocs", MetadataValueType.LONG), NUM_GROUPS_LIMIT_REACHED(11, "numGroupsLimitReached", MetadataValueType.STRING), @@ -135,6 +137,7 @@ enum MetadataKey { OPERATOR_ID(31, "operatorId", MetadataValueType.STRING), OPERATOR_EXEC_START_TIME_MS(32, "operatorExecStartTimeMs", MetadataValueType.LONG), OPERATOR_EXEC_END_TIME_MS(33, "operatorExecEndTimeMs", MetadataValueType.LONG), + // Not actually used MAX_ROWS_IN_JOIN_REACHED(34, "maxRowsInJoinReached", MetadataValueType.STRING); // We keep this constant to track the max id added so far for backward compatibility. diff --git a/pinot-common/src/main/java/org/apache/pinot/common/datatable/StatMap.java b/pinot-common/src/main/java/org/apache/pinot/common/datatable/StatMap.java new file mode 100644 index 000000000000..9d2818958bb7 --- /dev/null +++ b/pinot-common/src/main/java/org/apache/pinot/common/datatable/StatMap.java @@ -0,0 +1,498 @@ +/** + * 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.pinot.common.datatable; + +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.google.common.base.Preconditions; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Collections; +import java.util.EnumMap; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import javax.annotation.Nullable; +import org.apache.pinot.spi.utils.JsonUtils; + + +/** + * A map that stores statistics. + *

+ * Statistics must be keyed by an enum that implements {@link StatMap.Key}. + *

+ * A stat map efficiently store, serialize and deserialize these statistics. + *

+ * Serialization and deserialization is backward and forward compatible as long as the only change in the keys are: + *

    + *
  • Adding new keys
  • + *
  • Change the name of the keys
  • + *
+ * + * Any other change (like changing the type of key, changing their literal order are not supported or removing keys) + * are backward incompatible changes. + * @param + */ +public class StatMap & StatMap.Key> { + private final Class _keyClass; + private final Map _map; + + private static final ConcurrentHashMap, Object[]> KEYS_BY_CLASS = new ConcurrentHashMap<>(); + + public StatMap(Class keyClass) { + _keyClass = keyClass; + // TODO: Study whether this is fine or we should impose a single thread policy in StatMaps + _map = Collections.synchronizedMap(new EnumMap<>(keyClass)); + } + + public int getInt(K key) { + Preconditions.checkArgument(key.getType() == Type.INT, "Key %s is of type %s, not INT", key, key.getType()); + Object o = _map.get(key); + return o == null ? 0 : (Integer) o; + } + + public StatMap merge(K key, int value) { + if (key.getType() == Type.LONG) { + merge(key, (long) value); + return this; + } + int oldValue = getInt(key); + int newValue = key.merge(oldValue, value); + if (newValue == 0) { + _map.remove(key); + } else { + _map.put(key, newValue); + } + return this; + } + + public long getLong(K key) { + if (key.getType() == Type.INT) { + return getInt(key); + } + Preconditions.checkArgument(key.getType() == Type.LONG, "Key %s is of type %s, not LONG", key, key.getType()); + Object o = _map.get(key); + return o == null ? 0L : (Long) o; + } + + public StatMap merge(K key, long value) { + Preconditions.checkArgument(key.getType() == Type.LONG, "Key %s is of type %s, not LONG", key, key.getType()); + long oldValue = getLong(key); + long newValue = key.merge(oldValue, value); + if (newValue == 0) { + _map.remove(key); + } else { + _map.put(key, newValue); + } + return this; + } + + public boolean getBoolean(K key) { + Preconditions.checkArgument(key.getType() == Type.BOOLEAN, "Key %s is of type %s, not BOOLEAN", + key, key.getType()); + Object o = _map.get(key); + return o != null && (Boolean) o; + } + + public StatMap merge(K key, boolean value) { + boolean oldValue = getBoolean(key); + boolean newValue = key.merge(oldValue, value); + if (!newValue) { + _map.remove(key); + } else { + _map.put(key, Boolean.TRUE); + } + return this; + } + + public String getString(K key) { + Preconditions.checkArgument(key.getType() == Type.STRING, "Key %s is of type %s, not STRING", key, key.getType()); + Object o = _map.get(key); + return o == null ? null : (String) o; + } + + public StatMap merge(K key, String value) { + String oldValue = getString(key); + String newValue = key.merge(oldValue, value); + if (newValue == null) { + _map.remove(key); + } else { + _map.put(key, newValue); + } + return this; + } + + /** + * Returns the value associated with the key. + *

+ * Primitives will be boxed, so it is recommended to use the specific methods for each type. + */ + public Object getAny(K key) { + switch (key.getType()) { + case BOOLEAN: + return getBoolean(key); + case INT: + return getInt(key); + case LONG: + return getLong(key); + case STRING: + return getString(key); + default: + throw new IllegalArgumentException("Unsupported type: " + key.getType()); + } + } + + /** + * Modifies this object to merge the values of the other object. + * + * @param other The object to merge with. This argument will not be modified. + * @return this object once it is modified. + */ + public StatMap merge(StatMap other) { + Preconditions.checkState(_keyClass.equals(other._keyClass), + "Different key classes %s and %s", _keyClass, other._keyClass); + for (Map.Entry entry : other._map.entrySet()) { + K key = entry.getKey(); + Object value = entry.getValue(); + if (value == null) { + continue; + } + switch (key.getType()) { + case BOOLEAN: + merge(key, (boolean) value); + break; + case INT: + merge(key, (int) value); + break; + case LONG: + merge(key, (long) value); + break; + case STRING: + merge(key, (String) value); + break; + default: + throw new IllegalArgumentException("Unsupported type: " + key.getType()); + } + } + return this; + } + + public StatMap merge(DataInput input) + throws IOException { + byte serializedKeys = input.readByte(); + + K[] keys = (K[]) KEYS_BY_CLASS.computeIfAbsent(_keyClass, k -> k.getEnumConstants()); + for (byte i = 0; i < serializedKeys; i++) { + int ordinal = input.readByte(); + K key = keys[ordinal]; + switch (key.getType()) { + case BOOLEAN: + merge(key, true); + break; + case INT: + merge(key, input.readInt()); + break; + case LONG: + merge(key, input.readLong()); + break; + case STRING: + merge(key, input.readUTF()); + break; + default: + throw new IllegalStateException("Unknown type " + key.getType()); + } + } + return this; + } + + public ObjectNode asJson() { + ObjectNode node = JsonUtils.newObjectNode(); + + for (Map.Entry entry : _map.entrySet()) { + K key = entry.getKey(); + Object value = entry.getValue(); + switch (key.getType()) { + case BOOLEAN: + if (value == null) { + if (key.includeDefaultInJson()) { + node.put(key.getStatName(), false); + } + } else { + node.put(key.getStatName(), (boolean) value); + } + break; + case INT: + if (value == null) { + if (key.includeDefaultInJson()) { + node.put(key.getStatName(), 0); + } + } else { + node.put(key.getStatName(), (int) value); + } + break; + case LONG: + if (value == null) { + if (key.includeDefaultInJson()) { + node.put(key.getStatName(), 0L); + } + } else { + node.put(key.getStatName(), (long) value); + } + break; + case STRING: + if (value == null) { + if (key.includeDefaultInJson()) { + node.put(key.getStatName(), ""); + } + } else { + node.put(key.getStatName(), (String) value); + } + break; + default: + throw new IllegalArgumentException("Unsupported type: " + key.getType()); + } + } + + return node; + } + + public void serialize(DataOutput output) + throws IOException { + + assert checkContainsNoDefault() : "No default value should be stored in the map"; + output.writeByte(_map.size()); + + // We use written keys just to fail fast in tests if the number of keys written + // is not the same as the number of keys + int writtenKeys = 0; + K[] keys = (K[]) KEYS_BY_CLASS.computeIfAbsent(_keyClass, k -> k.getEnumConstants()); + for (int ordinal = 0; ordinal < keys.length; ordinal++) { + K key = keys[ordinal]; + switch (key.getType()) { + case BOOLEAN: { + if (getBoolean(key)) { + writtenKeys++; + output.writeByte(ordinal); + } + break; + } + case INT: { + int value = getInt(key); + if (value != 0) { + writtenKeys++; + output.writeByte(ordinal); + output.writeInt(value); + } + break; + } + case LONG: { + long value = getLong(key); + if (value != 0) { + writtenKeys++; + output.writeByte(ordinal); + output.writeLong(value); + } + break; + } + case STRING: { + String value = getString(key); + if (value != null) { + writtenKeys++; + output.writeByte(ordinal); + output.writeUTF(value); + } + break; + } + default: + throw new IllegalStateException("Unknown type " + key.getType()); + } + } + assert writtenKeys == _map.size() : "Written keys " + writtenKeys + " but map size " + _map.size(); + } + + private boolean checkContainsNoDefault() { + for (Map.Entry entry : _map.entrySet()) { + K key = entry.getKey(); + Object value = entry.getValue(); + switch (key.getType()) { + case BOOLEAN: + if (value == null || !(boolean) value) { + throw new IllegalStateException("Boolean value must be true but " + value + " is stored for key " + key); + } + break; + case INT: + if (value == null || (int) value == 0) { + throw new IllegalStateException("Int value must be non-zero but " + value + " is stored for key " + key); + } + break; + case LONG: + if (value == null || (long) value == 0) { + throw new IllegalStateException("Long value must be non-zero but " + value + " is stored for key " + key); + } + break; + case STRING: + if (value == null) { + throw new IllegalStateException("String value must be non-null but null is stored for key " + key); + } + break; + default: + throw new IllegalArgumentException("Unsupported type: " + key.getType()); + } + } + return true; + } + + public static String getDefaultStatName(Key key) { + String name = key.name(); + StringBuilder result = new StringBuilder(); + boolean capitalizeNext = false; + + for (char c : name.toCharArray()) { + if (c == '_') { + capitalizeNext = true; + } else { + if (capitalizeNext) { + result.append(c); + capitalizeNext = false; + } else { + result.append(Character.toLowerCase(c)); + } + } + } + + return result.toString(); + } + + public static & Key> StatMap deserialize(DataInput input, Class keyClass) + throws IOException { + StatMap result = new StatMap<>(keyClass); + result.merge(input); + return result; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + StatMap statMap = (StatMap) o; + return Objects.equals(_map, statMap._map); + } + + @Override + public int hashCode() { + return Objects.hash(_map); + } + + @Override + public String toString() { + return asJson().toString(); + } + + public Class getKeyClass() { + return _keyClass; + } + + public boolean isEmpty() { + return _map.isEmpty(); + } + + public Iterable keySet() { + return _map.keySet(); + } + + public interface Key { + String name(); + + /** + * The name of the stat used to report it. Names must be unique on the same key family. + */ + default String getStatName() { + return getDefaultStatName(this); + } + + default int merge(int value1, int value2) { + return value1 + value2; + } + + default long merge(long value1, long value2) { + return value1 + value2; + } + + default boolean merge(boolean value1, boolean value2) { + return value1 || value2; + } + + default String merge(@Nullable String value1, @Nullable String value2) { + return value2 != null ? value2 : value1; + } + + /** + * The type of the values associated to this key. + */ + Type getType(); + + default boolean includeDefaultInJson() { + return false; + } + + static int minPositive(int value1, int value2) { + if (value1 == 0 && value2 >= 0) { + return value2; + } + if (value2 == 0 && value1 >= 0) { + return value1; + } + return Math.min(value1, value2); + } + + static long minPositive(long value1, long value2) { + if (value1 == 0 && value2 >= 0) { + return value2; + } + if (value2 == 0 && value1 >= 0) { + return value1; + } + return Math.min(value1, value2); + } + + static int eqNotZero(int value1, int value2) { + if (value1 != value2) { + if (value1 == 0) { + return value2; + } else if (value2 == 0) { + return value1; + } else { + throw new IllegalStateException("Cannot merge non-zero values: " + value1 + " and " + value2); + } + } + return value1; + } + } + + public enum Type { + BOOLEAN, + INT, + LONG, + STRING + } +} diff --git a/pinot-common/src/main/java/org/apache/pinot/common/response/BrokerResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/response/BrokerResponse.java index 06f65fc2aa25..0d7ec268a749 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/response/BrokerResponse.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/response/BrokerResponse.java @@ -18,9 +18,12 @@ */ package org.apache.pinot.common.response; +import java.io.IOException; import java.util.List; +import javax.annotation.Nullable; import org.apache.pinot.common.response.broker.QueryProcessingException; import org.apache.pinot.common.response.broker.ResultTable; +import org.apache.pinot.spi.utils.JsonUtils; /** @@ -33,6 +36,8 @@ public interface BrokerResponse { */ void setExceptions(List exceptions); + void addToExceptions(QueryProcessingException processingException); + /** * Set the number of servers got queried by the broker. * @@ -47,21 +52,20 @@ public interface BrokerResponse { */ void setNumServersResponded(int numServersResponded); + long getTimeUsedMs(); + /** * Set the total time used in request handling, into the broker response. */ void setTimeUsedMs(long timeUsedMs); - /** - * Set the total number of rows in result set - */ - void setNumRowsResultSet(int numRowsResultSet); - /** * Convert the broker response to JSON String. */ - String toJsonString() - throws Exception; + default String toJsonString() + throws IOException { + return JsonUtils.objectToString(this); + } /** * Returns the number of servers queried. @@ -147,12 +151,13 @@ String toJsonString() * set the result table. * @param resultTable result table to be set. */ - void setResultTable(ResultTable resultTable); + void setResultTable(@Nullable ResultTable resultTable); /** * Get the result table. * @return result table. */ + @Nullable ResultTable getResultTable(); /** @@ -163,12 +168,10 @@ String toJsonString() /** * Get the total number of rows in result set */ - int getNumRowsResultSet(); - - /** - * Set the total thread cpu time used against offline table in request handling, into the broker response. - */ - void setOfflineThreadCpuTimeNs(long offlineThreadCpuTimeNs); + default int getNumRowsResultSet() { + ResultTable resultTable = getResultTable(); + return resultTable == null ? 0 : resultTable.getRows().size(); + } /** * Get the thread cpu time used against offline table in request handling, from the broker response. @@ -180,74 +183,43 @@ String toJsonString() */ long getRealtimeThreadCpuTimeNs(); - /** - * Set the total thread cpu time used against realtime table in request handling, into the broker response. - */ - void setRealtimeThreadCpuTimeNs(long realtimeThreadCpuTimeNs); - /** * Get the system activities cpu time used against offline table in request handling, from the broker response. */ long getOfflineSystemActivitiesCpuTimeNs(); - /** - * Set the system activities cpu time used against offline table in request handling, into the broker response. - */ - void setOfflineSystemActivitiesCpuTimeNs(long offlineSystemActivitiesCpuTimeNs); - /** * Get the system activities cpu time used against realtime table in request handling, from the broker response. */ long getRealtimeSystemActivitiesCpuTimeNs(); - /** - * Set the system activities cpu time used against realtime table in request handling, into the broker response. - */ - void setRealtimeSystemActivitiesCpuTimeNs(long realtimeSystemActivitiesCpuTimeNs); - /** * Get the response serialization cpu time used against offline table in request handling, from the broker response. */ long getOfflineResponseSerializationCpuTimeNs(); - /** - * Set the response serialization cpu time used against offline table in request handling, into the broker response. - */ - void setOfflineResponseSerializationCpuTimeNs(long offlineResponseSerializationCpuTimeNs); - /** * Get the response serialization cpu time used against realtime table in request handling, from the broker response. */ long getRealtimeResponseSerializationCpuTimeNs(); - /** - * Set the response serialization cpu time used against realtime table in request handling, into the broker response. - */ - void setRealtimeResponseSerializationCpuTimeNs(long realtimeResponseSerializationCpuTimeNs); - /** * Get the total cpu time(thread cpu time + system activities cpu time + response serialization cpu time) used * against offline table in request handling, from the broker response. */ - long getOfflineTotalCpuTimeNs(); - - /** - * Set the total cpu time(thread cpu time + system activities cpu time + response serialization cpu time) used - * against offline table in request handling, into the broker response. - */ - void setOfflineTotalCpuTimeNs(long offlineTotalCpuTimeNs); + default long getOfflineTotalCpuTimeNs() { + return getOfflineThreadCpuTimeNs() + getOfflineSystemActivitiesCpuTimeNs() + + getOfflineResponseSerializationCpuTimeNs(); + } /** * Get the total cpu time(thread cpu time + system activities cpu time + response serialization cpu time) used * against realtime table in request handling, from the broker response. */ - long getRealtimeTotalCpuTimeNs(); - - /** - * Set the total cpu time(thread cpu time + system activities cpu time + response serialization cpu time) used - * against realtime table in request handling, into the broker response. - */ - void setRealtimeTotalCpuTimeNs(long realtimeTotalCpuTimeNs); + default long getRealtimeTotalCpuTimeNs() { + return getRealtimeThreadCpuTimeNs() + getRealtimeSystemActivitiesCpuTimeNs() + + getRealtimeResponseSerializationCpuTimeNs(); + } /** * Get the total number of segments pruned on the Broker side @@ -264,11 +236,6 @@ String toJsonString() */ long getNumSegmentsPrunedByServer(); - /** - * Set the total number of segments pruned on the Server side - */ - void setNumSegmentsPrunedByServer(long numSegmentsPrunedByServer); - /** * Get the total number of segments pruned due to invalid data or schema. * @@ -276,13 +243,6 @@ String toJsonString() */ long getNumSegmentsPrunedInvalid(); - /** - * Set the total number of segments pruned due to invalid data or schema. - * - * This value is always lower or equal than {@link #getNumSegmentsPrunedByServer()} - */ - void setNumSegmentsPrunedInvalid(long numSegmentsPrunedInvalid); - /** * Get the total number of segments pruned by applying the limit optimization. * @@ -290,13 +250,6 @@ String toJsonString() */ long getNumSegmentsPrunedByLimit(); - /** - * Set the total number of segments pruned by applying the limit optimization. - * - * This value is always lower or equal than {@link #getNumSegmentsPrunedByServer()} - */ - void setNumSegmentsPrunedByLimit(long numSegmentsPrunedByLimit); - /** * Get the total number of segments pruned applying value optimizations, like bloom filters. * @@ -304,33 +257,16 @@ String toJsonString() */ long getNumSegmentsPrunedByValue(); - /** - * Set the total number of segments pruned applying value optimizations, like bloom filters. - * - * This value is always lower or equal than {@link #getNumSegmentsPrunedByServer()} - */ - void setNumSegmentsPrunedByValue(long numSegmentsPrunedByValue); - /** * Get the total number of segments with an EmptyFilterOperator when Explain Plan is called */ long getExplainPlanNumEmptyFilterSegments(); - /** - * Set the total number of segments with an EmptyFilterOperator when Explain Plan is called - */ - void setExplainPlanNumEmptyFilterSegments(long explainPlanNumEmptyFilterSegments); - /** * Get the total number of segments with a MatchAllFilterOperator when Explain Plan is called */ long getExplainPlanNumMatchAllFilterSegments(); - /** - * Set the total number of segments with a MatchAllFilterOperator when Explain Plan is called - */ - void setExplainPlanNumMatchAllFilterSegments(long explainPlanNumMatchAllFilterSegments); - /** * get request ID for the query */ @@ -354,4 +290,6 @@ String toJsonString() long getBrokerReduceTimeMs(); void setBrokerReduceTimeMs(long brokerReduceTimeMs); + + boolean isPartialResult(); } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/response/broker/BrokerResponseNative.java b/pinot-common/src/main/java/org/apache/pinot/common/response/broker/BrokerResponseNative.java index 9fe098e26d51..d22ee51c89e1 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/response/broker/BrokerResponseNative.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/response/broker/BrokerResponseNative.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyOrder; +import com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -37,7 +38,7 @@ /** * This class implements pinot-broker's response format for any given query. * All fields either primitive data types, or native objects (as opposed to JSONObjects). - * + *

* Supports serialization via JSON. */ @JsonPropertyOrder({ @@ -78,8 +79,6 @@ public class BrokerResponseNative implements BrokerResponse { private long _totalDocs = 0L; private boolean _numGroupsLimitReached = false; - private boolean _maxRowsInJoinReached = false; - private boolean _partialResult = false; private long _timeUsedMs = 0L; private long _offlineThreadCpuTimeNs = 0L; private long _realtimeThreadCpuTimeNs = 0L; @@ -131,6 +130,7 @@ public static BrokerResponseNative empty() { return new BrokerResponseNative(); } + @VisibleForTesting public static BrokerResponseNative fromJsonString(String jsonString) throws IOException { return JsonUtils.stringToObject(jsonString, BrokerResponseNative.class); @@ -143,7 +143,6 @@ public long getOfflineSystemActivitiesCpuTimeNs() { } @JsonProperty("offlineSystemActivitiesCpuTimeNs") - @Override public void setOfflineSystemActivitiesCpuTimeNs(long offlineSystemActivitiesCpuTimeNs) { _offlineSystemActivitiesCpuTimeNs = offlineSystemActivitiesCpuTimeNs; } @@ -155,7 +154,6 @@ public long getRealtimeSystemActivitiesCpuTimeNs() { } @JsonProperty("realtimeSystemActivitiesCpuTimeNs") - @Override public void setRealtimeSystemActivitiesCpuTimeNs(long realtimeSystemActivitiesCpuTimeNs) { _realtimeSystemActivitiesCpuTimeNs = realtimeSystemActivitiesCpuTimeNs; } @@ -167,7 +165,6 @@ public long getOfflineThreadCpuTimeNs() { } @JsonProperty("offlineThreadCpuTimeNs") - @Override public void setOfflineThreadCpuTimeNs(long timeUsedMs) { _offlineThreadCpuTimeNs = timeUsedMs; } @@ -179,7 +176,6 @@ public long getRealtimeThreadCpuTimeNs() { } @JsonProperty("realtimeThreadCpuTimeNs") - @Override public void setRealtimeThreadCpuTimeNs(long timeUsedMs) { _realtimeThreadCpuTimeNs = timeUsedMs; } @@ -191,7 +187,6 @@ public long getOfflineResponseSerializationCpuTimeNs() { } @JsonProperty("offlineResponseSerializationCpuTimeNs") - @Override public void setOfflineResponseSerializationCpuTimeNs(long offlineResponseSerializationCpuTimeNs) { _offlineResponseSerializationCpuTimeNs = offlineResponseSerializationCpuTimeNs; } @@ -203,7 +198,6 @@ public long getRealtimeResponseSerializationCpuTimeNs() { } @JsonProperty("realtimeResponseSerializationCpuTimeNs") - @Override public void setRealtimeResponseSerializationCpuTimeNs(long realtimeResponseSerializationCpuTimeNs) { _realtimeResponseSerializationCpuTimeNs = realtimeResponseSerializationCpuTimeNs; } @@ -215,7 +209,6 @@ public long getOfflineTotalCpuTimeNs() { } @JsonProperty("offlineTotalCpuTimeNs") - @Override public void setOfflineTotalCpuTimeNs(long offlineTotalCpuTimeNs) { _offlineTotalCpuTimeNs = offlineTotalCpuTimeNs; } @@ -227,7 +220,6 @@ public long getRealtimeTotalCpuTimeNs() { } @JsonProperty("realtimeTotalCpuTimeNs") - @Override public void setRealtimeTotalCpuTimeNs(long realtimeTotalCpuTimeNs) { _realtimeTotalCpuTimeNs = realtimeTotalCpuTimeNs; } @@ -251,7 +243,6 @@ public long getNumSegmentsPrunedByServer() { } @JsonProperty("numSegmentsPrunedByServer") - @Override public void setNumSegmentsPrunedByServer(long numSegmentsPrunedByServer) { _numSegmentsPrunedByServer = numSegmentsPrunedByServer; } @@ -263,7 +254,6 @@ public long getNumSegmentsPrunedInvalid() { } @JsonProperty("numSegmentsPrunedInvalid") - @Override public void setNumSegmentsPrunedInvalid(long numSegmentsPrunedInvalid) { _numSegmentsPrunedInvalid = numSegmentsPrunedInvalid; } @@ -275,7 +265,6 @@ public long getNumSegmentsPrunedByLimit() { } @JsonProperty("numSegmentsPrunedByLimit") - @Override public void setNumSegmentsPrunedByLimit(long numSegmentsPrunedByLimit) { _numSegmentsPrunedByLimit = numSegmentsPrunedByLimit; } @@ -287,7 +276,6 @@ public long getNumSegmentsPrunedByValue() { } @JsonProperty("numSegmentsPrunedByValue") - @Override public void setNumSegmentsPrunedByValue(long numSegmentsPrunedByValue) { _numSegmentsPrunedByValue = numSegmentsPrunedByValue; } @@ -299,7 +287,6 @@ public long getExplainPlanNumEmptyFilterSegments() { } @JsonProperty("explainPlanNumEmptyFilterSegments") - @Override public void setExplainPlanNumEmptyFilterSegments(long explainPlanNumEmptyFilterSegments) { _explainPlanNumEmptyFilterSegments = explainPlanNumEmptyFilterSegments; } @@ -311,7 +298,6 @@ public long getExplainPlanNumMatchAllFilterSegments() { } @JsonProperty("explainPlanNumMatchAllFilterSegments") - @Override public void setExplainPlanNumMatchAllFilterSegments(long explainPlanNumMatchAllFilterSegments) { _explainPlanNumMatchAllFilterSegments = explainPlanNumMatchAllFilterSegments; } @@ -350,7 +336,6 @@ public int getNumServersQueried() { } @JsonProperty("numServersQueried") - @Override public void setNumServersQueried(int numServersQueried) { _numServersQueried = numServersQueried; } @@ -448,6 +433,7 @@ public void setNumConsumingSegmentsQueried(long numConsumingSegmentsQueried) { public long getNumConsumingSegmentsProcessed() { return _numConsumingSegmentsProcessed; } + @JsonProperty("numConsumingSegmentsProcessed") public void setNumConsumingSegmentsProcessed(long numConsumingSegmentsProcessed) { _numConsumingSegmentsProcessed = numConsumingSegmentsProcessed; @@ -497,27 +483,18 @@ public void setNumGroupsLimitReached(boolean numGroupsLimitReached) { _numGroupsLimitReached = numGroupsLimitReached; } - @JsonProperty("maxRowsInJoinReached") + @JsonProperty(access = JsonProperty.Access.READ_ONLY) public boolean isMaxRowsInJoinReached() { - return _maxRowsInJoinReached; - } - - @JsonProperty("maxRowsInJoinReached") - public void setMaxRowsInJoinReached(boolean maxRowsInJoinReached) { - _maxRowsInJoinReached = maxRowsInJoinReached; + return false; } - @JsonProperty("partialResult") + @JsonProperty(access = JsonProperty.Access.READ_ONLY) public boolean isPartialResult() { - return _partialResult; - } - - @JsonProperty("partialResult") - public void setPartialResult(boolean partialResult) { - _partialResult = partialResult; + return isNumGroupsLimitReached() || getExceptionsSize() > 0 || isMaxRowsInJoinReached(); } @JsonProperty("timeUsedMs") + @Override public long getTimeUsedMs() { return _timeUsedMs; } @@ -528,16 +505,10 @@ public void setTimeUsedMs(long timeUsedMs) { _timeUsedMs = timeUsedMs; } - @JsonProperty("numRowsResultSet") + @JsonProperty(access = JsonProperty.Access.READ_ONLY) @Override public int getNumRowsResultSet() { - return _numRowsResultSet; - } - - @JsonProperty("numRowsResultSet") - @Override - public void setNumRowsResultSet(int numRowsResultSet) { - _numRowsResultSet = numRowsResultSet; + return BrokerResponse.super.getNumRowsResultSet(); } @JsonProperty("segmentStatistics") @@ -560,15 +531,11 @@ public void setTraceInfo(Map traceInfo) { _traceInfo = traceInfo; } - @Override - public String toJsonString() - throws IOException { - return JsonUtils.objectToString(this); - } - @JsonIgnore @Override public void setExceptions(List exceptions) { + // TODO: This is incorrect. It is adding and not setting the exceptions + // But there is some code that seems to depend on this. for (ProcessingException exception : exceptions) { _processingExceptions.add(new QueryProcessingException(exception.getErrorCode(), exception.getMessage())); } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/response/broker/BrokerResponseNativeV2.java b/pinot-common/src/main/java/org/apache/pinot/common/response/broker/BrokerResponseNativeV2.java index 60a22460f39a..40f521e25176 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/response/broker/BrokerResponseNativeV2.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/response/broker/BrokerResponseNativeV2.java @@ -18,16 +18,17 @@ */ package org.apache.pinot.common.response.broker; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyOrder; -import java.io.IOException; +import com.fasterxml.jackson.databind.node.ObjectNode; import java.util.ArrayList; -import java.util.HashMap; +import java.util.Collections; import java.util.List; -import java.util.Map; +import javax.annotation.Nullable; +import org.apache.pinot.common.datatable.StatMap; +import org.apache.pinot.common.response.BrokerResponse; import org.apache.pinot.common.response.ProcessingException; -import org.apache.pinot.common.utils.DataSchema; -import org.apache.pinot.spi.utils.JsonUtils; /** @@ -46,28 +47,36 @@ "realtimeResponseSerializationCpuTimeNs", "offlineTotalCpuTimeNs", "realtimeTotalCpuTimeNs", "brokerReduceTimeMs", "segmentStatistics", "traceInfo", "partialResult" }) -public class BrokerResponseNativeV2 extends BrokerResponseNative { +public class BrokerResponseNativeV2 implements BrokerResponse { - private final Map _stageIdStats = new HashMap<>(); + private ObjectNode _stageStats = null; + /** + * The max number of rows seen at runtime. + *

+ * In single-stage this doesn't make sense given it is the max number of rows read from the table. But in multi-stage + * virtual rows can be generated. For example, in a join query, the number of rows can be more than the number of rows + * in the table. + */ + private long _maxRowsInOperator = 0; + private final StatMap _brokerStats = new StatMap<>(StatKey.class); + private final List _processingExceptions; + private ResultTable _resultTable; + private String _requestId; + private String _brokerId; public BrokerResponseNativeV2() { + _processingExceptions = new ArrayList<>(); } public BrokerResponseNativeV2(ProcessingException exception) { - super(exception); + this(Collections.singletonList(exception)); } public BrokerResponseNativeV2(List exceptions) { - super(exceptions); - } - - /** Generate EXPLAIN PLAN output when queries are evaluated by Broker without going to the Server. */ - private static BrokerResponseNativeV2 getBrokerResponseExplainPlanOutput() { - BrokerResponseNativeV2 brokerResponse = BrokerResponseNativeV2.empty(); - List rows = new ArrayList<>(); - rows.add(new Object[]{"BROKER_EVALUATE", 0, -1}); - brokerResponse.setResultTable(new ResultTable(DataSchema.EXPLAIN_RESULT_SCHEMA, rows)); - return brokerResponse; + _processingExceptions = new ArrayList<>(exceptions.size()); + for (ProcessingException exception : exceptions) { + _processingExceptions.add(new QueryProcessingException(exception.getErrorCode(), exception.getMessage())); + } } /** @@ -77,20 +86,338 @@ public static BrokerResponseNativeV2 empty() { return new BrokerResponseNativeV2(); } - public static BrokerResponseNativeV2 fromJsonString(String jsonString) - throws IOException { - return JsonUtils.stringToObject(jsonString, BrokerResponseNativeV2.class); + public void setStageStats(ObjectNode stageStats) { + _stageStats = stageStats; + } + + @JsonProperty + public ObjectNode getStageStats() { + return _stageStats; + } + + /** + * Get the max number of rows seen by a single operator in the query processing chain. + */ + @JsonProperty(access = JsonProperty.Access.READ_ONLY) + public long getMaxRowsInOperator() { + return _maxRowsInOperator; + } + + public void mergeMaxRowsInOperator(long maxRows) { + _maxRowsInOperator = Math.max(_maxRowsInOperator, maxRows); + } + + @Override + public long getTimeUsedMs() { + return _brokerStats.getLong(StatKey.TIME_USED_MS); + } + + @Override + public void setTimeUsedMs(long timeUsedMs) { + _brokerStats.merge(StatKey.TIME_USED_MS, timeUsedMs); + } + + @Override + public long getNumDocsScanned() { + return _brokerStats.getLong(StatKey.NUM_DOCS_SCANNED); + } + + @Override + public long getNumEntriesScannedInFilter() { + return _brokerStats.getLong(StatKey.NUM_ENTRIES_SCANNED_IN_FILTER); + } + + @Override + public long getNumEntriesScannedPostFilter() { + return _brokerStats.getLong(StatKey.NUM_ENTRIES_SCANNED_POST_FILTER); + } + + @Override + public long getNumSegmentsQueried() { + return _brokerStats.getLong(StatKey.NUM_SEGMENTS_QUERIED); + } + + @Override + public long getNumSegmentsProcessed() { + return _brokerStats.getLong(StatKey.NUM_SEGMENTS_PROCESSED); + } + + @Override + public long getNumSegmentsMatched() { + return _brokerStats.getLong(StatKey.NUM_SEGMENTS_MATCHED); + } + + @Override + public long getNumConsumingSegmentsQueried() { + return _brokerStats.getLong(StatKey.NUM_CONSUMING_SEGMENTS_QUERIED); + } + + @Override + public long getNumConsumingSegmentsProcessed() { + return _brokerStats.getLong(StatKey.NUM_CONSUMING_SEGMENTS_PROCESSED); + } + + @Override + public long getNumConsumingSegmentsMatched() { + return _brokerStats.getLong(StatKey.NUM_CONSUMING_SEGMENTS_MATCHED); + } + + @Override + public long getMinConsumingFreshnessTimeMs() { + return _brokerStats.getLong(StatKey.MIN_CONSUMING_FRESHNESS_TIME_MS); + } + + @Override + public long getTotalDocs() { + return _brokerStats.getLong(StatKey.TOTAL_DOCS); + } + + @Override + public boolean isNumGroupsLimitReached() { + return _brokerStats.getBoolean(StatKey.NUM_GROUPS_LIMIT_REACHED); + } + + public void mergeNumGroupsLimitReached(boolean numGroupsLimitReached) { + _brokerStats.merge(StatKey.NUM_GROUPS_LIMIT_REACHED, numGroupsLimitReached); + } + + @Override + public long getNumSegmentsPrunedByServer() { + return _brokerStats.getLong(StatKey.NUM_SEGMENTS_PRUNED_BY_SERVER); } - public void addStageStat(Integer stageId, BrokerResponseStats brokerResponseStats) { - // StageExecutionWallTime will always be there, other stats are optional such as OperatorStats - if (brokerResponseStats.getStageExecWallTimeMs() != -1) { - _stageIdStats.put(stageId, brokerResponseStats); + @Override + public long getNumSegmentsPrunedInvalid() { + return _brokerStats.getLong(StatKey.NUM_SEGMENTS_PRUNED_INVALID); + } + + @Override + public long getNumSegmentsPrunedByLimit() { + return _brokerStats.getLong(StatKey.NUM_SEGMENTS_PRUNED_BY_LIMIT); + } + + @Override + public long getNumSegmentsPrunedByValue() { + return _brokerStats.getLong(StatKey.NUM_SEGMENTS_PRUNED_BY_VALUE); + } + + @Override + public long getExplainPlanNumEmptyFilterSegments() { + return _brokerStats.getLong(StatKey.EXPLAIN_PLAN_NUM_EMPTY_FILTER_SEGMENTS); + } + + @Override + public long getExplainPlanNumMatchAllFilterSegments() { + return _brokerStats.getLong(StatKey.EXPLAIN_PLAN_NUM_MATCH_ALL_FILTER_SEGMENTS); + } + + @Override + public long getOfflineTotalCpuTimeNs() { + return getOfflineThreadCpuTimeNs() + getOfflineSystemActivitiesCpuTimeNs() + + getOfflineResponseSerializationCpuTimeNs(); + } + + @Override + public long getRealtimeTotalCpuTimeNs() { + return getRealtimeThreadCpuTimeNs() + getRealtimeSystemActivitiesCpuTimeNs() + + getRealtimeResponseSerializationCpuTimeNs(); + } + + @Override + public void setExceptions(List exceptions) { + for (ProcessingException exception : exceptions) { + _processingExceptions.add(new QueryProcessingException(exception.getErrorCode(), exception.getMessage())); } } - @JsonProperty("stageStats") - public Map getStageIdStats() { - return _stageIdStats; + public void addToExceptions(QueryProcessingException processingException) { + _processingExceptions.add(processingException); + } + + @Override + public int getNumServersQueried() { + return _brokerStats.getInt(StatKey.NUM_SERVERS_QUERIED); + } + + @Override + public void setNumServersQueried(int numServersQueried) { + _brokerStats.merge(StatKey.NUM_SERVERS_QUERIED, numServersQueried); + } + + @Override + public int getNumServersResponded() { + return _brokerStats.getInt(StatKey.NUM_SERVERS_RESPONDED); + } + + @Override + public void setNumServersResponded(int numServersResponded) { + _brokerStats.merge(StatKey.NUM_SERVERS_RESPONDED, numServersResponded); + } + + @JsonProperty("maxRowsInJoinReached") + public boolean isMaxRowsInJoinReached() { + return _brokerStats.getBoolean(StatKey.MAX_ROWS_IN_JOIN_REACHED); + } + + @JsonProperty("maxRowsInJoinReached") + public void mergeMaxRowsInJoinReached(boolean maxRowsInJoinReached) { + _brokerStats.merge(StatKey.MAX_ROWS_IN_JOIN_REACHED, maxRowsInJoinReached); + } + + @Override + public int getExceptionsSize() { + return _processingExceptions.size(); + } + + @Override + public void setResultTable(@Nullable ResultTable resultTable) { + _resultTable = resultTable; + } + + @Nullable + @Override + @JsonInclude(JsonInclude.Include.NON_NULL) + public ResultTable getResultTable() { + return _resultTable; + } + + @JsonProperty("exceptions") + @Override + public List getProcessingExceptions() { + return List.of(); + } + + @JsonProperty(access = JsonProperty.Access.READ_ONLY) + @Override + public int getNumRowsResultSet() { + return BrokerResponse.super.getNumRowsResultSet(); + } + + @Override + public long getOfflineThreadCpuTimeNs() { + return 0; + } + + @Override + public long getRealtimeThreadCpuTimeNs() { + return 0; + } + + @Override + public long getOfflineSystemActivitiesCpuTimeNs() { + return 0; + } + + @Override + public long getRealtimeSystemActivitiesCpuTimeNs() { + return 0; + } + + @Override + public long getOfflineResponseSerializationCpuTimeNs() { + return 0; + } + + @Override + public long getRealtimeResponseSerializationCpuTimeNs() { + return 0; + } + + @Override + public long getNumSegmentsPrunedByBroker() { + return _brokerStats.getInt(StatKey.NUM_SEGMENTS_PRUNED_BY_BROKER); + } + + @Override + public void setNumSegmentsPrunedByBroker(long numSegmentsPrunedByBroker) { + _brokerStats.merge(StatKey.NUM_SEGMENTS_PRUNED_BY_BROKER, (int) numSegmentsPrunedByBroker); + } + + @Override + public String getRequestId() { + return _requestId; + } + + @Override + public void setRequestId(String requestId) { + _requestId = requestId; + } + + @Override + public String getBrokerId() { + return _brokerId; + } + + @Override + public void setBrokerId(String requestId) { + _brokerId = requestId; + } + + @Override + public long getBrokerReduceTimeMs() { + return _brokerStats.getLong(StatKey.BROKER_REDUCE_TIME_MS); + } + + @Override + public void setBrokerReduceTimeMs(long brokerReduceTimeMs) { + _brokerStats.merge(StatKey.BROKER_REDUCE_TIME_MS, brokerReduceTimeMs); + } + + @JsonProperty(access = JsonProperty.Access.READ_ONLY) + @Override + public boolean isPartialResult() { + return isNumGroupsLimitReached() || getExceptionsSize() > 0 || isMaxRowsInJoinReached(); + } + + public void addServerStats(StatMap serverStats) { + // Set execution statistics. + _brokerStats.merge(serverStats); + } + + public enum StatKey implements StatMap.Key { + TIME_USED_MS(StatMap.Type.LONG), + NUM_DOCS_SCANNED(StatMap.Type.LONG), + NUM_ENTRIES_SCANNED_IN_FILTER(StatMap.Type.LONG), + NUM_ENTRIES_SCANNED_POST_FILTER(StatMap.Type.LONG), + NUM_SEGMENTS_QUERIED(StatMap.Type.INT), + NUM_SEGMENTS_PROCESSED(StatMap.Type.INT), + NUM_SEGMENTS_MATCHED(StatMap.Type.INT), + NUM_CONSUMING_SEGMENTS_QUERIED(StatMap.Type.INT), + NUM_CONSUMING_SEGMENTS_PROCESSED(StatMap.Type.INT), + NUM_CONSUMING_SEGMENTS_MATCHED(StatMap.Type.INT), + MIN_CONSUMING_FRESHNESS_TIME_MS(StatMap.Type.LONG) { + @Override + public long merge(long value1, long value2) { + return StatMap.Key.minPositive(value1, value2); + } + }, + TOTAL_DOCS(StatMap.Type.LONG), + NUM_GROUPS_LIMIT_REACHED(StatMap.Type.BOOLEAN), + NUM_SEGMENTS_PRUNED_BY_SERVER(StatMap.Type.INT), + NUM_SEGMENTS_PRUNED_INVALID(StatMap.Type.INT), + NUM_SEGMENTS_PRUNED_BY_LIMIT(StatMap.Type.INT), + NUM_SEGMENTS_PRUNED_BY_VALUE(StatMap.Type.INT), + NUM_SERGMENTS_PRUNED_BY_BROKER(StatMap.Type.INT), + EXPLAIN_PLAN_NUM_EMPTY_FILTER_SEGMENTS(StatMap.Type.INT), + EXPLAIN_PLAN_NUM_MATCH_ALL_FILTER_SEGMENTS(StatMap.Type.INT), + THREAD_CPU_TIME_NS(StatMap.Type.LONG), + SYSTEM_ACTIVITIES_CPU_TIME_NS(StatMap.Type.LONG), + RESPONSE_SER_CPU_TIME_NS(StatMap.Type.LONG), + NUM_SEGMENTS_PRUNED_BY_BROKER(StatMap.Type.INT), + MAX_ROWS_IN_JOIN_REACHED(StatMap.Type.BOOLEAN), + NUM_SERVERS_RESPONDED(StatMap.Type.INT), + NUM_SERVERS_QUERIED(StatMap.Type.INT), + BROKER_REDUCE_TIME_MS(StatMap.Type.LONG); + + private final StatMap.Type _type; + + StatKey(StatMap.Type type) { + _type = type; + } + + @Override + public StatMap.Type getType() { + return _type; + } } } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/response/broker/BrokerResponseStats.java b/pinot-common/src/main/java/org/apache/pinot/common/response/broker/BrokerResponseStats.java index 3acc9f334932..97631876009a 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/response/broker/BrokerResponseStats.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/response/broker/BrokerResponseStats.java @@ -21,12 +21,10 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyOrder; -import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.pinot.spi.utils.JsonUtils; // TODO: Decouple the execution stats aggregator logic and make it into a util that can aggregate 2 values with the @@ -109,11 +107,6 @@ public void setStageExecutionUnit(int stageExecutionUnit) { _stageExecutionUnit = stageExecutionUnit; } - public String toJsonString() - throws IOException { - return JsonUtils.objectToString(this); - } - @JsonProperty("operatorStats") public Map> getOperatorStats() { return _operatorStats; diff --git a/pinot-common/src/test/java/org/apache/pinot/common/datablock/BaseDataBlockContract.java b/pinot-common/src/test/java/org/apache/pinot/common/datablock/BaseDataBlockContract.java new file mode 100644 index 000000000000..763fd7344cae --- /dev/null +++ b/pinot-common/src/test/java/org/apache/pinot/common/datablock/BaseDataBlockContract.java @@ -0,0 +1,42 @@ +/** + * 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.pinot.common.datablock; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import static org.testng.Assert.*; + + +public abstract class BaseDataBlockContract { + + protected abstract BaseDataBlock deserialize(ByteBuffer byteBuffer, int versionType) + throws IOException; + + public void testSerdeCorrectness(BaseDataBlock dataBlock) + throws IOException { + byte[] bytes = dataBlock.toBytes(); + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + int versionType = DataBlockUtils.readVersionType(byteBuffer); + BaseDataBlock deserialize = deserialize(byteBuffer, versionType); + + assertEquals(byteBuffer.position(), bytes.length, "Buffer position should be at the end of the buffer"); + assertEquals(deserialize, dataBlock, "Deserialized data block should be the same as the original data block"); + } +} diff --git a/pinot-common/src/test/java/org/apache/pinot/common/datablock/MetadataBlockTest.java b/pinot-common/src/test/java/org/apache/pinot/common/datablock/MetadataBlockTest.java index 0f85f4a69ee2..cfceddc65d7c 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/datablock/MetadataBlockTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/datablock/MetadataBlockTest.java @@ -18,43 +18,185 @@ */ package org.apache.pinot.common.datablock; -import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Collections; import java.util.HashMap; +import java.util.List; import org.testng.annotations.Test; import static org.testng.Assert.*; -public class MetadataBlockTest { +public class MetadataBlockTest extends BaseDataBlockContract { + @Override + protected BaseDataBlock deserialize(ByteBuffer byteBuffer, int versionType) + throws IOException { + return new MetadataBlock(byteBuffer); + } @Test - public void shouldEncodeContentsAsJSON() + public void emptyMetadataBlock() throws Exception { // Given: - MetadataBlock.MetadataBlockType type = MetadataBlock.MetadataBlockType.EOS; - // When: - MetadataBlock metadataBlock = new MetadataBlock(type); + MetadataBlock metadataBlock = MetadataBlock.newEos(); // Then: - byte[] expected = new ObjectMapper().writeValueAsBytes(new MetadataBlock.Contents("EOS", new HashMap<>())); - assertEquals(metadataBlock._variableSizeDataBytes, expected); + byte[] expectedFixed = new byte[0]; + assertEquals(metadataBlock._fixedSizeDataBytes, expectedFixed); + byte[] expectedVariable = new byte[0]; + assertEquals(metadataBlock._variableSizeDataBytes, expectedVariable); + + List statsByStage = metadataBlock.getStatsByStage(); + assertNotNull(statsByStage, "Expected stats by stage to be non-null"); + assertEquals(statsByStage.size(), 0, "Expected no stats by stage"); + } + + @Test + public void emptyDataBlockCorrectness() + throws IOException { + testSerdeCorrectness(MetadataBlock.newEos()); + } + + @Test + public void v1ErrorWithExceptionsIsDecodedAsV2ErrorWithSameExceptions() + throws IOException { + V1MetadataBlock v1MetadataBlock = new V1MetadataBlock(MetadataBlock.MetadataBlockType.ERROR); + v1MetadataBlock.addException(250, "timeout"); + v1MetadataBlock.addException(500, "server error"); + + byte[] bytes = v1MetadataBlock.toBytes(); + ByteBuffer buff = ByteBuffer.wrap(bytes); + DataBlock dataBlock = DataBlockUtils.getDataBlock(buff); + + assertTrue(dataBlock instanceof MetadataBlock, "V1MetadataBlock should be always decoded as MetadataBlock"); + MetadataBlock metadataBlock = (MetadataBlock) dataBlock; + assertEquals(metadataBlock.getType(), MetadataBlock.MetadataBlockType.ERROR, "Expected error type"); + assertEquals(metadataBlock.getStatsByStage(), Collections.emptyList(), "Expected no stats by stage"); + assertEquals(metadataBlock.getExceptions(), v1MetadataBlock.getExceptions(), "Expected exceptions"); + } + + /** + * Verifies that a V2 EOS with empty stats is read in V1 as EOS without stats + */ + @Test + public void v2EosWithoutStatsIsReadInV1AsEosWithoutStats() + throws IOException { + ByteBuffer stats = ByteBuffer.wrap(new byte[]{0, 0, 0, 0}); + MetadataBlock metadataBlock = new MetadataBlock(Lists.newArrayList(stats)); + + byte[] bytes = metadataBlock.toBytes(); + + // This is how V1 blocks were deserialized + ByteBuffer buff = ByteBuffer.wrap(bytes); + DataBlockUtils.readVersionType(buff); // consume the version information before decoding + V1MetadataBlock v1MetadataBlock = new V1MetadataBlock(buff); + + assertEquals(v1MetadataBlock.getType(), MetadataBlock.MetadataBlockType.EOS, "Expected EOS type"); + assertEquals(v1MetadataBlock.getStats(), Collections.emptyMap(), "Expected no stats by stage"); + assertEquals(v1MetadataBlock.getExceptions(), metadataBlock.getExceptions(), "Expected no exceptions"); } + /** + * Verifies that a V2 EOS with stats is read in V1 as EOS without stats + */ @Test - public void shouldDefaultToEosWithNoErrorsOnLegacyMetadataBlock() + public void v2EosWithStatsIsReadInV1AsEosWithoutStats() + throws IOException { + MetadataBlock metadataBlock = MetadataBlock.newEos(); + + byte[] bytes = metadataBlock.toBytes(); + + // This is how V1 blocks were deserialized + ByteBuffer buff = ByteBuffer.wrap(bytes); + DataBlockUtils.readVersionType(buff); // consume the version information before decoding + V1MetadataBlock v1MetadataBlock = new V1MetadataBlock(buff); + + assertEquals(v1MetadataBlock.getType(), MetadataBlock.MetadataBlockType.EOS, "Expected EOS type"); + assertEquals(v1MetadataBlock.getStats(), Collections.emptyMap(), "Expected no stats by stage"); + assertEquals(v1MetadataBlock.getExceptions(), metadataBlock.getExceptions(), "Expected no exceptions"); + } + + /** + * Verifies that a V2 error code is read in V1 as error with the same exceptions + */ + @Test + public void v2ErrorIsReadInV1AsErrorWithSameExceptions() + throws IOException { + HashMap errorMap = new HashMap<>(); + errorMap.put(250, "timeout"); + MetadataBlock metadataBlock = MetadataBlock.newError(errorMap); + + byte[] bytes = metadataBlock.toBytes(); + + // This is how V1 blocks were deserialized + ByteBuffer buff = ByteBuffer.wrap(bytes); + DataBlockUtils.readVersionType(buff); // consume the version information before decoding + V1MetadataBlock v1MetadataBlock = new V1MetadataBlock(buff); + + assertEquals(v1MetadataBlock.getType(), MetadataBlock.MetadataBlockType.ERROR, "Expected error type"); + assertEquals(v1MetadataBlock.getStats(), Collections.emptyMap(), "Expected no stats by stage"); + assertEquals(v1MetadataBlock.getExceptions(), metadataBlock.getExceptions(), "Expected exceptions"); + } + + /** + * Verifies that a V1 error code is decoded as a V2 error code with the same exceptions + */ + @Test + public void v1EosWithoutStatsIsDecodedAsV2EosWithoutStats() + throws IOException { + V1MetadataBlock v1MetadataBlock = new V1MetadataBlock(MetadataBlock.MetadataBlockType.EOS, new HashMap<>()); + + byte[] bytes = v1MetadataBlock.toBytes(); + ByteBuffer buff = ByteBuffer.wrap(bytes); + DataBlock dataBlock = DataBlockUtils.getDataBlock(buff); + + assertTrue(dataBlock instanceof MetadataBlock, "V1MetadataBlock should be always decoded as MetadataBlock"); + MetadataBlock metadataBlock = (MetadataBlock) dataBlock; + assertEquals(metadataBlock.getType(), MetadataBlock.MetadataBlockType.EOS, "Expected EOS type"); + assertEquals(metadataBlock.getStatsByStage(), Collections.emptyList(), "Expected no stats by stage"); + assertEquals(metadataBlock.getExceptions(), v1MetadataBlock.getExceptions(), "Expected exceptions"); + } + + /** + * Verifies that a V1 EOS with stats is decoded as a V2 EOS without stats + */ + @Test + public void v1EosWithStatsIsDecodedAsV2EosWithoutStats() + throws IOException { + HashMap stats = new HashMap<>(); + stats.put("foo", "bar"); + stats.put("baz", "qux"); + V1MetadataBlock v1MetadataBlock = new V1MetadataBlock(MetadataBlock.MetadataBlockType.EOS, stats); + + byte[] bytes = v1MetadataBlock.toBytes(); + ByteBuffer buff = ByteBuffer.wrap(bytes); + DataBlock dataBlock = DataBlockUtils.getDataBlock(buff); + + assertTrue(dataBlock instanceof MetadataBlock, "V1MetadataBlock should be always decoded as MetadataBlock"); + MetadataBlock metadataBlock = (MetadataBlock) dataBlock; + assertEquals(metadataBlock.getType(), MetadataBlock.MetadataBlockType.EOS, "Expected EOS type"); + assertEquals(metadataBlock.getStatsByStage(), Collections.emptyList(), "Expected no stats by stage"); + assertEquals(metadataBlock.getExceptions(), Collections.emptyMap(), "Expected no exceptions"); + } + + /** + * Verifies that a V0 EOS without exceptions is decoded as a V2 EOS without stats + */ + @Test + public void v0EosWithoutExceptionsIsDecodedAsV2EosWithoutStats() throws IOException { // Given: // MetadataBlock used to be encoded without any data, we should make sure that // during rollout or if server versions are mismatched that we can still handle // the old format - OldMetadataBlock legacyBlock = new OldMetadataBlock(); + V0MetadataBlock legacyBlock = new V0MetadataBlock(); byte[] bytes = legacyBlock.toBytes(); // When: ByteBuffer buff = ByteBuffer.wrap(bytes); - buff.getInt(); // consume the version information before decoding + DataBlockUtils.readVersionType(buff); // consume the version information before decoding MetadataBlock metadataBlock = new MetadataBlock(buff); // Then: @@ -62,29 +204,30 @@ public void shouldDefaultToEosWithNoErrorsOnLegacyMetadataBlock() } @Test - public void shouldDefaultToErrorOnLegacyMetadataBlockWithErrors() + public void v0EosWithExceptionsIsDecodedAsV2ErrorWithSameExceptions() throws IOException { // Given: // MetadataBlock used to be encoded without any data, we should make sure that // during rollout or if server versions are mismatched that we can still handle // the old format - OldMetadataBlock legacyBlock = new OldMetadataBlock(); + V0MetadataBlock legacyBlock = new V0MetadataBlock(); legacyBlock.addException(250, "timeout"); byte[] bytes = legacyBlock.toBytes(); // When: ByteBuffer buff = ByteBuffer.wrap(bytes); - buff.getInt(); // consume the version information before decoding + DataBlockUtils.readVersionType(buff); // consume the version information before decoding MetadataBlock metadataBlock = new MetadataBlock(buff); // Then: assertEquals(metadataBlock.getType(), MetadataBlock.MetadataBlockType.ERROR); + assertEquals(metadataBlock.getExceptions(), legacyBlock.getExceptions(), "Expected exceptions"); } @Test(expectedExceptions = UnsupportedOperationException.class) public void shouldThrowExceptionWhenUsingReadMethods() { // Given: - MetadataBlock block = new MetadataBlock(MetadataBlock.MetadataBlockType.EOS); + MetadataBlock block = MetadataBlock.newEos(); // When: // (should through exception) @@ -94,9 +237,9 @@ public void shouldThrowExceptionWhenUsingReadMethods() { /** * This is mostly just used as an internal serialization tool */ - private static class OldMetadataBlock extends BaseDataBlock { + private static class V0MetadataBlock extends BaseDataBlock { - public OldMetadataBlock() { + public V0MetadataBlock() { super(0, null, new String[0], new byte[0], new byte[0]); } @@ -114,15 +257,5 @@ protected int getOffsetInFixedBuffer(int rowId, int colId) { protected int positionOffsetInVariableBufferAndGetLength(int rowId, int colId) { return 0; } - - @Override - public DataBlock toMetadataOnlyDataTable() { - return null; - } - - @Override - public DataBlock toDataOnlyDataTable() { - return null; - } } } diff --git a/pinot-common/src/test/java/org/apache/pinot/common/datablock/RowDataBlockTest.java b/pinot-common/src/test/java/org/apache/pinot/common/datablock/RowDataBlockTest.java new file mode 100644 index 000000000000..b3bbe26ff613 --- /dev/null +++ b/pinot-common/src/test/java/org/apache/pinot/common/datablock/RowDataBlockTest.java @@ -0,0 +1,40 @@ +/** + * 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.pinot.common.datablock; + +import java.io.IOException; +import java.nio.ByteBuffer; +import org.apache.pinot.common.utils.DataSchema; +import org.testng.annotations.Test; + + +public class RowDataBlockTest extends BaseDataBlockContract { + @Override + protected BaseDataBlock deserialize(ByteBuffer byteBuffer, int versionType) + throws IOException { + return new RowDataBlock(byteBuffer); + } + + @Test + public void emptyDataBlockCorrectness() + throws IOException { + DataSchema dataSchema = new DataSchema(new String[0], new DataSchema.ColumnDataType[0]); + testSerdeCorrectness(new RowDataBlock(0, dataSchema, new String[0], new byte[0], new byte[0])); + } +} diff --git a/pinot-common/src/test/java/org/apache/pinot/common/datablock/V1MetadataBlock.java b/pinot-common/src/test/java/org/apache/pinot/common/datablock/V1MetadataBlock.java new file mode 100644 index 000000000000..990d9489ccc8 --- /dev/null +++ b/pinot-common/src/test/java/org/apache/pinot/common/datablock/V1MetadataBlock.java @@ -0,0 +1,150 @@ +/** + * 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.pinot.common.datablock; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; + + +/** + * The datablock used before the introduction of {@link org.apache.pinot.common.datatable.StatMap}. + *

+ * This version stored the metadata in a {@code Map} which was encoded as JSON and stored in the + * variable size data buffer. + *

+ * Instances of this class are not actually seen by the operators. Instead, they are converted to {@link MetadataBlock} + * in {@link MetadataBlock#deserialize(ByteBuffer, int)}. + *

+ * The reason to keep it here is mostly for backwards compatibility and testing. In order to simplify the code, the + * stats engine just ignores the metadata of these objects, but we need to be able to deserialize them anyway. + */ +public class V1MetadataBlock extends BaseDataBlock { + + private static final ObjectMapper JSON = new ObjectMapper(); + + @VisibleForTesting + static final int VERSION = 1; + + /** + * Used to serialize the contents of the metadata block conveniently and in + * a backwards compatible way. Use JSON because the performance of metadata block + * SerDe should not be a bottleneck. + */ + @JsonIgnoreProperties(ignoreUnknown = true) + @VisibleForTesting + static class Contents { + + private String _type; + private Map _stats; + + @JsonCreator + public Contents(@JsonProperty("type") String type, @JsonProperty("stats") Map stats) { + _type = type; + _stats = stats; + } + + @JsonCreator + public Contents() { + this(null, new HashMap<>()); + } + + public String getType() { + return _type; + } + + public void setType(String type) { + _type = type; + } + + public Map getStats() { + return _stats; + } + + public void setStats(Map stats) { + _stats = stats; + } + } + + private final Contents _contents; + + public V1MetadataBlock(MetadataBlock.MetadataBlockType type) { + this(type, new HashMap<>()); + } + + public V1MetadataBlock(MetadataBlock.MetadataBlockType type, Map stats) { + super(0, null, new String[0], new byte[]{0}, toContents(new Contents(type.name(), stats))); + _contents = new Contents(type.name(), stats); + } + + private static byte[] toContents(Contents type) { + try { + return JSON.writeValueAsBytes(type); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } + + public V1MetadataBlock(ByteBuffer byteBuffer) + throws IOException { + super(byteBuffer); + if (_variableSizeDataBytes != null && _variableSizeDataBytes.length > 0) { + _contents = JSON.readValue(_variableSizeDataBytes, Contents.class); + } else { + _contents = new Contents(); + } + } + + public MetadataBlock.MetadataBlockType getType() { + String type = _contents.getType(); + + // if type is null, then we're reading a legacy block where we didn't encode any + // data. assume that it is an EOS block if there's no exceptions and an ERROR block + // otherwise + return type == null + ? (getExceptions().isEmpty() ? MetadataBlock.MetadataBlockType.EOS : MetadataBlock.MetadataBlockType.ERROR) + : MetadataBlock.MetadataBlockType.valueOf(type); + } + + public Map getStats() { + return _contents.getStats() != null ? _contents.getStats() : new HashMap<>(); + } + + @Override + public int getDataBlockVersionType() { + return VERSION + (Type.METADATA.ordinal() << DataBlockUtils.VERSION_TYPE_SHIFT); + } + + @Override + protected int getOffsetInFixedBuffer(int rowId, int colId) { + throw new UnsupportedOperationException("Metadata block uses JSON encoding for field access"); + } + + @Override + protected int positionOffsetInVariableBufferAndGetLength(int rowId, int colId) { + throw new UnsupportedOperationException("Metadata block uses JSON encoding for field access"); + } +} diff --git a/pinot-common/src/test/java/org/apache/pinot/common/datatable/StatMapTest.java b/pinot-common/src/test/java/org/apache/pinot/common/datatable/StatMapTest.java new file mode 100644 index 000000000000..25f5ef8a07fc --- /dev/null +++ b/pinot-common/src/test/java/org/apache/pinot/common/datatable/StatMapTest.java @@ -0,0 +1,209 @@ +/** + * 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.pinot.common.datatable; + +import com.google.common.io.ByteArrayDataInput; +import com.google.common.io.ByteArrayDataOutput; +import com.google.common.io.ByteStreams; +import java.io.IOException; +import org.apache.pinot.common.response.broker.BrokerResponseNativeV2; +import org.testng.Assert; +import org.testng.SkipException; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + + +public class StatMapTest { + + @Test(dataProvider = "allTypeStats", expectedExceptions = IllegalArgumentException.class) + public void dynamicTypeCheckWhenAddInt(MyStats stat) { + if (stat.getType() == StatMap.Type.INT) { + throw new SkipException("Skipping INT test"); + } + if (stat.getType() == StatMap.Type.LONG) { + throw new SkipException("Skipping LONG test"); + } + StatMap statMap = new StatMap<>(MyStats.class); + statMap.merge(stat, 1); + } + + @Test(dataProvider = "allTypeStats", expectedExceptions = IllegalArgumentException.class) + public void dynamicTypeCheckWhenAddLong(MyStats stat) { + if (stat.getType() == StatMap.Type.LONG) { + throw new SkipException("Skipping LONG test"); + } + StatMap statMap = new StatMap<>(MyStats.class); + statMap.merge(stat, 1L); + } + + @Test(dataProvider = "allTypeStats", expectedExceptions = IllegalArgumentException.class) + public void dynamicTypeCheckPutBoolean(MyStats stat) { + if (stat.getType() == StatMap.Type.BOOLEAN) { + throw new SkipException("Skipping BOOLEAN test"); + } + StatMap statMap = new StatMap<>(MyStats.class); + statMap.merge(stat, true); + } + + @Test(dataProvider = "allTypeStats", expectedExceptions = IllegalArgumentException.class) + public void dynamicTypeCheckPutString(MyStats stat) { + if (stat.getType() == StatMap.Type.STRING) { + throw new SkipException("Skipping STRING test"); + } + StatMap statMap = new StatMap<>(MyStats.class); + statMap.merge(stat, "foo"); + } + + @Test(dataProvider = "allTypeStats") + public void singleEncodeDecode(MyStats stat) + throws IOException { + StatMap statMap = new StatMap<>(MyStats.class); + switch (stat.getType()) { + case BOOLEAN: + statMap.merge(stat, true); + break; + case INT: + statMap.merge(stat, 1); + break; + case LONG: + statMap.merge(stat, 1L); + break; + case STRING: + statMap.merge(stat, "foo"); + break; + default: + throw new IllegalStateException(); + } + testSerializeDeserialize(statMap); + } + + @Test + public void encodeDecodeAll() + throws IOException { + StatMap statMap = new StatMap<>(MyStats.class); + for (MyStats stat : MyStats.values()) { + switch (stat.getType()) { + case BOOLEAN: + statMap.merge(stat, true); + break; + case INT: + statMap.merge(stat, 1); + break; + case LONG: + statMap.merge(stat, 1L); + break; + case STRING: + statMap.merge(stat, "foo"); + break; + default: + throw new IllegalStateException(); + } + } + testSerializeDeserialize(statMap); + } + + private & StatMap.Key> void testSerializeDeserialize(StatMap statMap) + throws IOException { + ByteArrayDataOutput output = ByteStreams.newDataOutput(); + statMap.serialize(output); + + ByteArrayDataInput input = ByteStreams.newDataInput(output.toByteArray()); + StatMap deserializedStatMap = StatMap.deserialize(input, MyStats.class); + + Assert.assertEquals(statMap, deserializedStatMap); + } + + @Test(dataProvider = "complexStats") + public void mergeEquivalence(StatMap statMap) + throws IOException { + StatMap mergedOnHeap = new StatMap<>(statMap.getKeyClass()); + mergedOnHeap.merge(statMap); + + ByteArrayDataOutput output = ByteStreams.newDataOutput(); + statMap.serialize(output); + + ByteArrayDataInput input = ByteStreams.newDataInput(output.toByteArray()); + StatMap mergedSerialized = new StatMap<>(statMap.getKeyClass()); + mergedSerialized.merge(input); + + Assert.assertEquals(mergedOnHeap, mergedSerialized, + "Merging objects should be equal to merging serialized buffers"); + } + + @DataProvider(name = "complexStats") + static StatMap[] complexStats() { + return new StatMap[] { + new StatMap<>(MyStats.class) + .merge(MyStats.BOOL_KEY, true) + .merge(MyStats.LONG_KEY, 1L) + .merge(MyStats.INT_KEY, 1) + .merge(MyStats.STR_KEY, "foo"), + new StatMap<>(MyStats.class) + .merge(MyStats.BOOL_KEY, false) + .merge(MyStats.LONG_KEY, 1L) + .merge(MyStats.INT_KEY, 1) + .merge(MyStats.STR_KEY, "foo"), + new StatMap<>(MyStats.class) + .merge(MyStats.BOOL_KEY, true) + .merge(MyStats.LONG_KEY, 0L) + .merge(MyStats.INT_KEY, 1) + .merge(MyStats.STR_KEY, "foo"), + new StatMap<>(MyStats.class) + .merge(MyStats.BOOL_KEY, false) + .merge(MyStats.LONG_KEY, 1L) + .merge(MyStats.INT_KEY, 0) + .merge(MyStats.STR_KEY, "foo"), + new StatMap<>(MyStats.class) + .merge(MyStats.BOOL_KEY, false) + .merge(MyStats.LONG_KEY, 1L) + .merge(MyStats.INT_KEY, 1), + new StatMap<>(BrokerResponseNativeV2.StatKey.class) + .merge(BrokerResponseNativeV2.StatKey.NUM_SEGMENTS_QUERIED, 1) + .merge(BrokerResponseNativeV2.StatKey.NUM_SEGMENTS_PROCESSED, 1) + .merge(BrokerResponseNativeV2.StatKey.NUM_SEGMENTS_MATCHED, 1) + .merge(BrokerResponseNativeV2.StatKey.NUM_DOCS_SCANNED, 10) + .merge(BrokerResponseNativeV2.StatKey.NUM_ENTRIES_SCANNED_POST_FILTER, 5) + .merge(BrokerResponseNativeV2.StatKey.TOTAL_DOCS, 5) + .merge(BrokerResponseNativeV2.StatKey.TIME_USED_MS, 95) + }; + } + + @DataProvider(name = "allTypeStats") + static MyStats[] allTypeStats() { + return MyStats.values(); + } + + public enum MyStats implements StatMap.Key { + BOOL_KEY(StatMap.Type.BOOLEAN), + LONG_KEY(StatMap.Type.LONG), + INT_KEY(StatMap.Type.INT), + STR_KEY(StatMap.Type.STRING); + + private final StatMap.Type _type; + + MyStats(StatMap.Type type) { + _type = type; + } + + @Override + public StatMap.Type getType() { + return _type; + } + } +} diff --git a/pinot-core/src/main/java/org/apache/pinot/core/operator/InstanceResponseOperator.java b/pinot-core/src/main/java/org/apache/pinot/core/operator/InstanceResponseOperator.java index 0ef9bb22071d..5ee75e5bf177 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/operator/InstanceResponseOperator.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/operator/InstanceResponseOperator.java @@ -45,6 +45,9 @@ public class InstanceResponseOperator extends BaseOperator combineOperator, List segmentContexts, List fetchContexts, QueryContext queryContext) { _combineOperator = combineOperator; @@ -81,12 +84,24 @@ public static long calSystemActivitiesCpuTimeNs(long totalWallClockTimeNs, long @Override protected InstanceResponseBlock getNextBlock() { + BaseResultsBlock baseResultsBlock = getBaseBlock(); + return buildInstanceResponseBlock(baseResultsBlock); + } + + protected InstanceResponseBlock buildInstanceResponseBlock(BaseResultsBlock baseResultsBlock) { + InstanceResponseBlock instanceResponseBlock = new InstanceResponseBlock(baseResultsBlock); + instanceResponseBlock.addMetadata(MetadataKey.THREAD_CPU_TIME_NS.getName(), String.valueOf(_threadCpuTimeNs)); + instanceResponseBlock.addMetadata(MetadataKey.SYSTEM_ACTIVITIES_CPU_TIME_NS.getName(), + String.valueOf(_systemActivitiesCpuTimeNs)); + return instanceResponseBlock; + } + + protected BaseResultsBlock getBaseBlock() { if (ThreadResourceUsageProvider.isThreadCpuTimeMeasurementEnabled()) { long startWallClockTimeNs = System.nanoTime(); ThreadResourceUsageProvider mainThreadResourceUsageProvider = new ThreadResourceUsageProvider(); BaseResultsBlock resultsBlock = getCombinedResults(); - InstanceResponseBlock instanceResponseBlock = new InstanceResponseBlock(resultsBlock); long mainThreadCpuTimeNs = mainThreadResourceUsageProvider.getThreadTimeNs(); long totalWallClockTimeNs = System.nanoTime() - startWallClockTimeNs; @@ -97,22 +112,17 @@ protected InstanceResponseBlock getNextBlock() { */ long multipleThreadCpuTimeNs = resultsBlock.getExecutionThreadCpuTimeNs(); int numServerThreads = resultsBlock.getNumServerThreads(); - long systemActivitiesCpuTimeNs = - calSystemActivitiesCpuTimeNs(totalWallClockTimeNs, multipleThreadCpuTimeNs, mainThreadCpuTimeNs, - numServerThreads); - - long threadCpuTimeNs = mainThreadCpuTimeNs + multipleThreadCpuTimeNs; - instanceResponseBlock.addMetadata(MetadataKey.THREAD_CPU_TIME_NS.getName(), String.valueOf(threadCpuTimeNs)); - instanceResponseBlock.addMetadata(MetadataKey.SYSTEM_ACTIVITIES_CPU_TIME_NS.getName(), - String.valueOf(systemActivitiesCpuTimeNs)); + _systemActivitiesCpuTimeNs = calSystemActivitiesCpuTimeNs(totalWallClockTimeNs, multipleThreadCpuTimeNs, + mainThreadCpuTimeNs, numServerThreads); + _threadCpuTimeNs = mainThreadCpuTimeNs + multipleThreadCpuTimeNs; - return instanceResponseBlock; + return resultsBlock; } else { - return new InstanceResponseBlock(getCombinedResults()); + return getCombinedResults(); } } - private BaseResultsBlock getCombinedResults() { + protected BaseResultsBlock getCombinedResults() { try { prefetchAll(); return _combineOperator.nextBlock(); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/operator/streaming/StreamingInstanceResponseOperator.java b/pinot-core/src/main/java/org/apache/pinot/core/operator/streaming/StreamingInstanceResponseOperator.java index 61a5b8cc83d9..1130efb4d716 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/operator/streaming/StreamingInstanceResponseOperator.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/operator/streaming/StreamingInstanceResponseOperator.java @@ -36,6 +36,12 @@ import org.apache.pinot.spi.trace.Tracing; +/** + * Like {@link InstanceResponseOperator}, but instead of sending all the data to the broker at once, it streams the data + * to a given {@link ResultsBlockStreamer}. + * + * This is used in multi-stage to stream data to the receiving mailboxes. + */ public class StreamingInstanceResponseOperator extends InstanceResponseOperator { private static final String EXPLAIN_NAME = "STREAMING_INSTANCE_RESPONSE"; @@ -58,7 +64,7 @@ protected InstanceResponseBlock getNextBlock() { prefetchAll(); if (_streamingCombineOperator != null) { _streamingCombineOperator.start(); - BaseResultsBlock resultsBlock = _streamingCombineOperator.nextBlock(); + BaseResultsBlock resultsBlock = getBaseBlock(); while (!(resultsBlock instanceof MetadataResultsBlock)) { if (resultsBlock instanceof ExceptionResultsBlock) { return new InstanceResponseBlock(resultsBlock); @@ -66,20 +72,20 @@ protected InstanceResponseBlock getNextBlock() { if (resultsBlock.getNumRows() > 0) { _streamer.send(resultsBlock); } - resultsBlock = _streamingCombineOperator.nextBlock(); + resultsBlock = getBaseBlock(); } // Return a metadata-only block in the end - return new InstanceResponseBlock(resultsBlock); + return buildInstanceResponseBlock(resultsBlock); } else { // Handle single block combine operator in streaming fashion - BaseResultsBlock resultsBlock = _combineOperator.nextBlock(); + BaseResultsBlock resultsBlock = getBaseBlock(); if (resultsBlock instanceof ExceptionResultsBlock) { return new InstanceResponseBlock(resultsBlock); } if (resultsBlock.getNumRows() > 0) { _streamer.send(resultsBlock); } - return new InstanceResponseBlock(resultsBlock).toMetadataOnlyResponseBlock(); + return buildInstanceResponseBlock(resultsBlock).toMetadataOnlyResponseBlock(); } } catch (EarlyTerminationException e) { Exception killedErrorMsg = Tracing.getThreadAccountant().getErrorStatus(); @@ -96,6 +102,10 @@ protected InstanceResponseBlock getNextBlock() { } } + protected BaseResultsBlock getCombinedResults() { + return _combineOperator.nextBlock(); + } + @Override public String toExplainString() { return EXPLAIN_NAME; diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java index c05644af5fbc..c40133baab8f 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java @@ -34,7 +34,6 @@ import org.apache.pinot.common.metrics.BrokerMetrics; import org.apache.pinot.common.metrics.BrokerTimer; import org.apache.pinot.common.response.broker.BrokerResponseNative; -import org.apache.pinot.common.response.broker.BrokerResponseStats; import org.apache.pinot.common.response.broker.QueryProcessingException; import org.apache.pinot.core.transport.ServerRoutingInstance; import org.apache.pinot.spi.config.table.TableType; @@ -74,7 +73,6 @@ public class ExecutionStatsAggregator { private long _explainPlanNumEmptyFilterSegments = 0L; private long _explainPlanNumMatchAllFilterSegments = 0L; private boolean _numGroupsLimitReached = false; - private boolean _maxRowsInJoinReached = false; private int _numBlocks = 0; private int _numRows = 0; private long _stageExecutionTimeMs = 0; @@ -248,12 +246,10 @@ public synchronized void aggregate(@Nullable ServerRoutingInstance routingInstan if (numTotalDocsString != null) { _numTotalDocs += Long.parseLong(numTotalDocsString); } + _numGroupsLimitReached |= Boolean.parseBoolean(metadata.get(DataTable.MetadataKey.NUM_GROUPS_LIMIT_REACHED.getName())); - _maxRowsInJoinReached |= - Boolean.parseBoolean(metadata.get(DataTable.MetadataKey.MAX_ROWS_IN_JOIN_REACHED.getName())); - String numBlocksString = metadata.get(DataTable.MetadataKey.NUM_BLOCKS.getName()); if (numBlocksString != null) { _numBlocks += Long.parseLong(numBlocksString); @@ -309,15 +305,12 @@ public void setStats(@Nullable String rawTableName, BrokerResponseNative brokerR brokerResponseNative.setNumSegmentsMatched(_numSegmentsMatched); brokerResponseNative.setTotalDocs(_numTotalDocs); brokerResponseNative.setNumGroupsLimitReached(_numGroupsLimitReached); - brokerResponseNative.setMaxRowsInJoinReached(_maxRowsInJoinReached); brokerResponseNative.setOfflineThreadCpuTimeNs(_offlineThreadCpuTimeNs); brokerResponseNative.setRealtimeThreadCpuTimeNs(_realtimeThreadCpuTimeNs); brokerResponseNative.setOfflineSystemActivitiesCpuTimeNs(_offlineSystemActivitiesCpuTimeNs); brokerResponseNative.setRealtimeSystemActivitiesCpuTimeNs(_realtimeSystemActivitiesCpuTimeNs); brokerResponseNative.setOfflineResponseSerializationCpuTimeNs(_offlineResponseSerializationCpuTimeNs); brokerResponseNative.setRealtimeResponseSerializationCpuTimeNs(_realtimeResponseSerializationCpuTimeNs); - brokerResponseNative.setOfflineTotalCpuTimeNs(_offlineTotalCpuTimeNs); - brokerResponseNative.setRealtimeTotalCpuTimeNs(_realtimeTotalCpuTimeNs); brokerResponseNative.setNumSegmentsPrunedByServer(_numSegmentsPrunedByServer); brokerResponseNative.setNumSegmentsPrunedInvalid(_numSegmentsPrunedInvalid); brokerResponseNative.setNumSegmentsPrunedByLimit(_numSegmentsPrunedByLimit); @@ -364,25 +357,6 @@ public void setStats(@Nullable String rawTableName, BrokerResponseNative brokerR } } - public void setStageLevelStats(@Nullable String rawTableName, BrokerResponseStats brokerResponseStats, - @Nullable BrokerMetrics brokerMetrics) { - if (_enableTrace) { - setStats(rawTableName, brokerResponseStats, brokerMetrics); - brokerResponseStats.setOperatorStats(_operatorStats); - } - - brokerResponseStats.setNumBlocks(_numBlocks); - brokerResponseStats.setNumRows(_numRows); - brokerResponseStats.setMaxRowsInJoinReached(_maxRowsInJoinReached); - brokerResponseStats.setNumGroupsLimitReached(_numGroupsLimitReached); - brokerResponseStats.setStageExecutionTimeMs(_stageExecutionTimeMs); - brokerResponseStats.setStageExecutionUnit(_stageExecutionUnit); - brokerResponseStats.setTableNames(new ArrayList<>(_tableNames)); - if (_stageExecStartTimeMs >= 0 && _stageExecEndTimeMs >= 0) { - brokerResponseStats.setStageExecWallTimeMs(_stageExecEndTimeMs - _stageExecStartTimeMs); - } - } - private void withNotNullLongMetadata(Map metadata, DataTable.MetadataKey key, LongConsumer consumer) { String strValue = metadata.get(key.getName()); if (strValue != null) { diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java index e7c5af0dab9b..6bc1f0cbdae4 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java @@ -406,7 +406,7 @@ public void testBrokerResponseMetadata() for (String query : queries) { JsonNode response = postQuery(query); for (String statName : statNames) { - assertTrue(response.has(statName)); + assertTrue(response.has(statName), "Response does not contain stat: " + statName); } } } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/HybridClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/HybridClusterIntegrationTest.java index 73d0dcb9bdfc..4752de4add7d 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/HybridClusterIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/HybridClusterIntegrationTest.java @@ -317,7 +317,8 @@ public void testDropResults(boolean useMultiStageQueryEngine) final String resultTag = "resultTable"; // dropResults=true - resultTable must not be in the response - Assert.assertFalse(postQueryWithOptions(query, "dropResults=true").has(resultTag)); + JsonNode jsonNode = postQueryWithOptions(query, "dropResults=true"); + Assert.assertFalse(jsonNode.has(resultTag)); // dropResults=TrUE (case insensitive match) - resultTable must not be in the response Assert.assertFalse(postQueryWithOptions(query, "dropResults=TrUE").has(resultTag)); diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotJoinToDynamicBroadcastRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotJoinToDynamicBroadcastRule.java index 5398e58b77f7..0dc2dc588434 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotJoinToDynamicBroadcastRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotJoinToDynamicBroadcastRule.java @@ -24,6 +24,7 @@ import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.plan.RelOptRuleCall; import org.apache.calcite.plan.hep.HepRelVertex; +import org.apache.calcite.rel.RelDistribution; import org.apache.calcite.rel.RelDistributions; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.core.Exchange; @@ -164,11 +165,15 @@ public void onMatch(RelOptRuleCall call) { // 2. when hash key are the same but hash functions are different, it can be done via normal hash shuffle. boolean isColocatedJoin = PinotHintStrategyTable.isHintOptionTrue(join.getHints(), PinotHintOptions.JOIN_HINT_OPTIONS, PinotHintOptions.JoinHintOptions.IS_COLOCATED_BY_JOIN_KEYS); - PinotLogicalExchange dynamicBroadcastExchange = isColocatedJoin - ? PinotLogicalExchange.create(right.getInput(), RelDistributions.hash(join.analyzeCondition().rightKeys), - PinotRelExchangeType.PIPELINE_BREAKER) - : PinotLogicalExchange.create(right.getInput(), RelDistributions.BROADCAST_DISTRIBUTED, - PinotRelExchangeType.PIPELINE_BREAKER); + PinotLogicalExchange dynamicBroadcastExchange; + RelNode rightInput = right.getInput(); + if (isColocatedJoin) { + RelDistribution dist = RelDistributions.hash(join.analyzeCondition().rightKeys); + dynamicBroadcastExchange = PinotLogicalExchange.create(rightInput, dist, PinotRelExchangeType.PIPELINE_BREAKER); + } else { + RelDistribution dist = RelDistributions.BROADCAST_DISTRIBUTED; + dynamicBroadcastExchange = PinotLogicalExchange.create(rightInput, dist, PinotRelExchangeType.PIPELINE_BREAKER); + } Join dynamicFilterJoin = new LogicalJoin(join.getCluster(), join.getTraitSet(), left.getInput(), dynamicBroadcastExchange, join.getCondition(), join.getVariablesSet(), join.getJoinType(), join.isSemiJoinDone(), diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/GrpcSendingMailbox.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/GrpcSendingMailbox.java index 87a8bda603ea..e2bb53faf595 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/GrpcSendingMailbox.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/GrpcSendingMailbox.java @@ -24,12 +24,14 @@ import java.io.IOException; import java.util.concurrent.TimeUnit; import org.apache.pinot.common.datablock.DataBlock; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.proto.Mailbox.MailboxContent; import org.apache.pinot.common.proto.PinotMailboxGrpc; import org.apache.pinot.query.mailbox.channel.ChannelManager; import org.apache.pinot.query.mailbox.channel.MailboxStatusObserver; import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; +import org.apache.pinot.query.runtime.operator.MailboxSendOperator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,16 +47,19 @@ public class GrpcSendingMailbox implements SendingMailbox { private final String _hostname; private final int _port; private final long _deadlineMs; + private final StatMap _statMap; private final MailboxStatusObserver _statusObserver = new MailboxStatusObserver(); private StreamObserver _contentObserver; - public GrpcSendingMailbox(String id, ChannelManager channelManager, String hostname, int port, long deadlineMs) { + public GrpcSendingMailbox(String id, ChannelManager channelManager, String hostname, int port, long deadlineMs, + StatMap statMap) { _id = id; _channelManager = channelManager; _hostname = hostname; _port = port; _deadlineMs = deadlineMs; + _statMap = statMap; } @Override @@ -64,25 +69,31 @@ public void send(TransferableBlock block) return; } if (LOGGER.isDebugEnabled()) { - LOGGER.debug("==[GRPC SEND]== sending data to: " + _id); + LOGGER.debug("==[GRPC SEND]== sending message " + block + " to: " + _id); } if (_contentObserver == null) { _contentObserver = getContentObserver(); } _contentObserver.onNext(toMailboxContent(block)); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("==[GRPC SEND]== message " + block + " sent to: " + _id); + } } @Override public void complete() { if (isTerminated()) { + LOGGER.debug("Already terminated mailbox: {}", _id); return; } + LOGGER.debug("Completing mailbox: {}", _id); _contentObserver.onCompleted(); } @Override public void cancel(Throwable t) { if (isTerminated()) { + LOGGER.debug("Already terminated mailbox: {}", _id); return; } LOGGER.debug("Cancelling mailbox: {}", _id); @@ -118,9 +129,22 @@ private StreamObserver getContentObserver() { private MailboxContent toMailboxContent(TransferableBlock block) throws IOException { - DataBlock dataBlock = block.getDataBlock(); - byte[] bytes = dataBlock.toBytes(); - ByteString byteString = UnsafeByteOperations.unsafeWrap(bytes); - return MailboxContent.newBuilder().setMailboxId(_id).setPayload(byteString).build(); + _statMap.merge(MailboxSendOperator.StatKey.RAW_MESSAGES, 1); + long start = System.currentTimeMillis(); + try { + DataBlock dataBlock = block.getDataBlock(); + byte[] bytes = dataBlock.toBytes(); + ByteString byteString = UnsafeByteOperations.unsafeWrap(bytes); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Serialized block: {} to {} bytes", block, bytes.length); + } + _statMap.merge(MailboxSendOperator.StatKey.SERIALIZED_BYTES, bytes.length); + return MailboxContent.newBuilder().setMailboxId(_id).setPayload(byteString).build(); + } catch (Throwable t) { + LOGGER.warn("Caught exception while serializing block: {}", block, t); + throw t; + } finally { + _statMap.merge(MailboxSendOperator.StatKey.SERIALIZATION_TIME_MS, System.currentTimeMillis() - start); + } } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/InMemorySendingMailbox.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/InMemorySendingMailbox.java index d1452712bc33..8adf8db073b3 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/InMemorySendingMailbox.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/InMemorySendingMailbox.java @@ -19,8 +19,10 @@ package org.apache.pinot.query.mailbox; import java.util.concurrent.TimeoutException; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; +import org.apache.pinot.query.runtime.operator.MailboxSendOperator; import org.apache.pinot.spi.exception.QueryCancelledException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,11 +38,14 @@ public class InMemorySendingMailbox implements SendingMailbox { private ReceivingMailbox _receivingMailbox; private volatile boolean _isTerminated; private volatile boolean _isEarlyTerminated; + private final StatMap _statMap; - public InMemorySendingMailbox(String id, MailboxService mailboxService, long deadlineMs) { + public InMemorySendingMailbox(String id, MailboxService mailboxService, long deadlineMs, + StatMap statMap) { _id = id; _mailboxService = mailboxService; _deadlineMs = deadlineMs; + _statMap = statMap; } @Override @@ -52,8 +57,10 @@ public void send(TransferableBlock block) if (_receivingMailbox == null) { _receivingMailbox = _mailboxService.getReceivingMailbox(_id); } + _statMap.merge(MailboxSendOperator.StatKey.IN_MEMORY_MESSAGES, 1); long timeoutMs = _deadlineMs - System.currentTimeMillis(); ReceivingMailbox.ReceivingMailboxStatus status = _receivingMailbox.offer(block, timeoutMs); + switch (status) { case SUCCESS: break; diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/MailboxService.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/MailboxService.java index 7c171f1bf43b..c71af0386d13 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/MailboxService.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/MailboxService.java @@ -23,8 +23,10 @@ import com.google.common.cache.RemovalListener; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.query.mailbox.channel.ChannelManager; import org.apache.pinot.query.mailbox.channel.GrpcMailboxServer; +import org.apache.pinot.query.runtime.operator.MailboxSendOperator; import org.apache.pinot.spi.env.PinotConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -99,11 +101,12 @@ public int getPort() { * not open the underlying channel or acquire any additional resources. Instead, it will initialize lazily when the * data is sent for the first time. */ - public SendingMailbox getSendingMailbox(String hostname, int port, String mailboxId, long deadlineMs) { + public SendingMailbox getSendingMailbox(String hostname, int port, String mailboxId, long deadlineMs, + StatMap statMap) { if (_hostname.equals(hostname) && _port == port) { - return new InMemorySendingMailbox(mailboxId, this, deadlineMs); + return new InMemorySendingMailbox(mailboxId, this, deadlineMs, statMap); } else { - return new GrpcSendingMailbox(mailboxId, _channelManager, hostname, port, deadlineMs); + return new GrpcSendingMailbox(mailboxId, _channelManager, hostname, port, deadlineMs, statMap); } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java index 97c8731f95a8..e58b5f7dd58a 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java @@ -19,12 +19,19 @@ package org.apache.pinot.query.mailbox; import com.google.common.base.Preconditions; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Map; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; import javax.annotation.Nullable; +import org.apache.pinot.common.datablock.DataBlock; +import org.apache.pinot.common.datablock.DataBlockUtils; +import org.apache.pinot.common.datablock.MetadataBlock; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.slf4j.Logger; @@ -50,9 +57,11 @@ public class ReceivingMailbox { private final BlockingQueue _blocks = new ArrayBlockingQueue<>(DEFAULT_MAX_PENDING_BLOCKS); private final AtomicReference _errorBlock = new AtomicReference<>(); private volatile boolean _isEarlyTerminated = false; + private long _lastArriveTime = System.currentTimeMillis(); @Nullable private volatile Reader _reader; + private final StatMap _stats = new StatMap<>(StatKey.class); public ReceivingMailbox(String id) { _id = id; @@ -72,11 +81,53 @@ public String getId() { return _id; } + /** + * Offers a raw block into the mailbox within the timeout specified, returns whether the block is successfully added. + * If the block is not added, an error block is added to the mailbox. + *

+ * Contrary to {@link #offer(TransferableBlock, long)}, the block may be an + * {@link TransferableBlock#isErrorBlock() error block}. + */ + public ReceivingMailboxStatus offerRaw(ByteBuffer byteBuffer, long timeoutMs) + throws IOException { + TransferableBlock block; + long now = System.currentTimeMillis(); + _stats.merge(StatKey.WAIT_CPU_TIME_MS, now - _lastArriveTime); + _lastArriveTime = now; + _stats.merge(StatKey.DESERIALIZED_BYTES, byteBuffer.remaining()); + _stats.merge(StatKey.DESERIALIZED_MESSAGES, 1); + + now = System.currentTimeMillis(); + DataBlock dataBlock = DataBlockUtils.getDataBlock(byteBuffer); + _stats.merge(StatKey.DESERIALIZATION_TIME_MS, System.currentTimeMillis() - now); + + if (dataBlock instanceof MetadataBlock) { + Map exceptions = dataBlock.getExceptions(); + if (exceptions.isEmpty()) { + block = TransferableBlockUtils.wrap(dataBlock); + } else { + setErrorBlock(TransferableBlockUtils.getErrorTransferableBlock(exceptions)); + return ReceivingMailboxStatus.FIRST_ERROR; + } + } else { + block = TransferableBlockUtils.wrap(dataBlock); + } + return offerPrivate(block, timeoutMs); + } + + public ReceivingMailboxStatus offer(TransferableBlock block, long timeoutMs) { + long now = System.currentTimeMillis(); + _stats.merge(StatKey.WAIT_CPU_TIME_MS, now - _lastArriveTime); + _lastArriveTime = now; + _stats.merge(StatKey.IN_MEMORY_MESSAGES, 1); + return offerPrivate(block, timeoutMs); + } + /** * Offers a non-error block into the mailbox within the timeout specified, returns whether the block is successfully * added. If the block is not added, an error block is added to the mailbox. */ - public ReceivingMailboxStatus offer(TransferableBlock block, long timeoutMs) { + private ReceivingMailboxStatus offerPrivate(TransferableBlock block, long timeoutMs) { TransferableBlock errorBlock = _errorBlock.get(); if (errorBlock != null) { LOGGER.debug("Mailbox: {} is already cancelled or errored out, ignoring the late block", _id); @@ -90,7 +141,10 @@ public ReceivingMailboxStatus offer(TransferableBlock block, long timeoutMs) { return ReceivingMailboxStatus.TIMEOUT; } try { - if (_blocks.offer(block, timeoutMs, TimeUnit.MILLISECONDS)) { + long now = System.currentTimeMillis(); + boolean accepted = _blocks.offer(block, timeoutMs, TimeUnit.MILLISECONDS); + _stats.merge(StatKey.OFFER_CPU_TIME_MS, System.currentTimeMillis() - now); + if (accepted) { errorBlock = _errorBlock.get(); if (errorBlock == null) { if (LOGGER.isDebugEnabled()) { @@ -170,11 +224,40 @@ private void notifyReader() { } } + public StatMap getStatMap() { + return _stats; + } + public interface Reader { void blockReadyToRead(); } public enum ReceivingMailboxStatus { - SUCCESS, ERROR, TIMEOUT, CANCELLED, EARLY_TERMINATED + SUCCESS, FIRST_ERROR, ERROR, TIMEOUT, CANCELLED, EARLY_TERMINATED + } + + public enum StatKey implements StatMap.Key { + DESERIALIZED_MESSAGES(StatMap.Type.INT), + DESERIALIZED_BYTES(StatMap.Type.LONG), + DESERIALIZATION_TIME_MS(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }, + IN_MEMORY_MESSAGES(StatMap.Type.INT), + OFFER_CPU_TIME_MS(StatMap.Type.LONG), + WAIT_CPU_TIME_MS(StatMap.Type.LONG); + + private final StatMap.Type _type; + + StatKey(StatMap.Type type) { + _type = type; + } + + @Override + public StatMap.Type getType() { + return _type; + } } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/channel/MailboxContentObserver.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/channel/MailboxContentObserver.java index ce91701c3ade..1d817f91aaed 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/channel/MailboxContentObserver.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/channel/MailboxContentObserver.java @@ -20,16 +20,12 @@ import io.grpc.Context; import io.grpc.stub.StreamObserver; -import java.util.Map; +import java.nio.ByteBuffer; import java.util.concurrent.TimeUnit; -import org.apache.pinot.common.datablock.DataBlock; -import org.apache.pinot.common.datablock.DataBlockUtils; -import org.apache.pinot.common.datablock.MetadataBlock; import org.apache.pinot.common.proto.Mailbox.MailboxContent; import org.apache.pinot.common.proto.Mailbox.MailboxStatus; import org.apache.pinot.query.mailbox.MailboxService; import org.apache.pinot.query.mailbox.ReceivingMailbox; -import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,22 +58,9 @@ public void onNext(MailboxContent mailboxContent) { _mailbox = _mailboxService.getReceivingMailbox(mailboxId); } try { - TransferableBlock block; - DataBlock dataBlock = DataBlockUtils.getDataBlock(mailboxContent.getPayload().asReadOnlyByteBuffer()); - if (dataBlock instanceof MetadataBlock) { - Map exceptions = dataBlock.getExceptions(); - if (exceptions.isEmpty()) { - block = TransferableBlockUtils.getEndOfStreamTransferableBlock(((MetadataBlock) dataBlock).getStats()); - } else { - _mailbox.setErrorBlock(TransferableBlockUtils.getErrorTransferableBlock(exceptions)); - return; - } - } else { - block = new TransferableBlock(dataBlock); - } - long timeoutMs = Context.current().getDeadline().timeRemaining(TimeUnit.MILLISECONDS); - ReceivingMailbox.ReceivingMailboxStatus status = _mailbox.offer(block, timeoutMs); + ByteBuffer buffer = mailboxContent.getPayload().asReadOnlyByteBuffer(); + ReceivingMailbox.ReceivingMailboxStatus status = _mailbox.offerRaw(buffer, timeoutMs); switch (status) { case SUCCESS: _responseObserver.onNext(MailboxStatus.newBuilder().setMailboxId(mailboxId) @@ -88,6 +71,8 @@ public void onNext(MailboxContent mailboxContent) { LOGGER.warn("Mailbox: {} already cancelled from upstream", mailboxId); cancelStream(); break; + case FIRST_ERROR: + return; case ERROR: LOGGER.warn("Mailbox: {} already errored out (received error block before)", mailboxId); cancelStream(); diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/InStageStatsTreeBuilder.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/InStageStatsTreeBuilder.java new file mode 100644 index 000000000000..16cdda026e90 --- /dev/null +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/InStageStatsTreeBuilder.java @@ -0,0 +1,207 @@ +/** + * 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.pinot.query.runtime; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.function.IntFunction; +import org.apache.pinot.query.planner.plannode.AbstractPlanNode; +import org.apache.pinot.query.planner.plannode.AggregateNode; +import org.apache.pinot.query.planner.plannode.ExchangeNode; +import org.apache.pinot.query.planner.plannode.FilterNode; +import org.apache.pinot.query.planner.plannode.JoinNode; +import org.apache.pinot.query.planner.plannode.MailboxReceiveNode; +import org.apache.pinot.query.planner.plannode.MailboxSendNode; +import org.apache.pinot.query.planner.plannode.PlanNode; +import org.apache.pinot.query.planner.plannode.PlanNodeVisitor; +import org.apache.pinot.query.planner.plannode.ProjectNode; +import org.apache.pinot.query.planner.plannode.SetOpNode; +import org.apache.pinot.query.planner.plannode.SortNode; +import org.apache.pinot.query.planner.plannode.TableScanNode; +import org.apache.pinot.query.planner.plannode.ValueNode; +import org.apache.pinot.query.planner.plannode.WindowNode; +import org.apache.pinot.query.runtime.operator.MultiStageOperator; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; +import org.apache.pinot.spi.utils.JsonUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class InStageStatsTreeBuilder implements PlanNodeVisitor { + private static final Logger LOGGER = LoggerFactory.getLogger(InStageStatsTreeBuilder.class); + + private final MultiStageQueryStats.StageStats _stageStats; + private int _index; + private static final String CHILDREN_KEY = "children"; + private final IntFunction _jsonStatsByStage; + + public InStageStatsTreeBuilder(MultiStageQueryStats.StageStats stageStats, IntFunction jsonStatsByStage) { + _stageStats = stageStats; + _index = stageStats.getLastOperatorIndex(); + _jsonStatsByStage = jsonStatsByStage; + } + + private ObjectNode selfNode(MultiStageOperator.Type type) { + ObjectNode json = JsonUtils.newObjectNode(); + json.put("type", type.toString()); + Iterator> statsIt = _stageStats.getOperatorStats(_index).asJson().fields(); + while (statsIt.hasNext()) { + Map.Entry entry = statsIt.next(); + json.set(entry.getKey(), entry.getValue()); + } + return json; + } + + private ObjectNode recursiveCase(AbstractPlanNode node, MultiStageOperator.Type expectedType) { + MultiStageOperator.Type type = _stageStats.getOperatorType(_index); + /* + Sometimes the operator type is not what we expect, but we can still build the tree + This always happen in stage 0, in which case we have two operators but we only have stats for the receive + operator. + This may also happen leaf stages, in which case the all the stage but the send operator will be compiled into + a single leaf node. + */ + if (type != expectedType) { + if (type == MultiStageOperator.Type.LEAF) { + return selfNode(MultiStageOperator.Type.LEAF); + } + List inputs = node.getInputs(); + int childrenSize = inputs.size(); + switch (childrenSize) { + case 0: + return JsonUtils.newObjectNode(); + case 1: + return inputs.get(0).visit(this, null); + default: + ObjectNode json = JsonUtils.newObjectNode(); + ArrayNode children = JsonUtils.newArrayNode(); + for (int i = 0; i < childrenSize; i++) { + _index--; + if (inputs.size() > i) { + children.add(inputs.get(i).visit(this, null)); + } + } + json.set(CHILDREN_KEY, children); + return json; + } + } + ObjectNode json = selfNode(type); + List inputs = node.getInputs(); + int size = inputs.size(); + JsonNode[] childrenArr = new JsonNode[size]; + if (size > _index) { + LOGGER.warn("Operator {} has {} inputs but only {} stats are left", type, size, + _index); + return json; + } + for (int i = size - 1; i >= 0; i--) { + PlanNode planNode = inputs.get(i); + _index--; + JsonNode child = planNode.visit(this, null); + + childrenArr[i] = child; + } + json.set(CHILDREN_KEY, JsonUtils.objectToJsonNode(childrenArr)); + return json; + } + + @Override + public ObjectNode visitAggregate(AggregateNode node, Void context) { + return recursiveCase(node, MultiStageOperator.Type.AGGREGATE); + } + + @Override + public ObjectNode visitFilter(FilterNode node, Void context) { + return recursiveCase(node, MultiStageOperator.Type.FILTER); + } + + @Override + public ObjectNode visitJoin(JoinNode node, Void context) { + return recursiveCase(node, MultiStageOperator.Type.HASH_JOIN); + } + + @Override + public ObjectNode visitMailboxReceive(MailboxReceiveNode node, Void context) { + ObjectNode json = selfNode(MultiStageOperator.Type.MAILBOX_RECEIVE); + + ArrayNode children = JsonUtils.newArrayNode(); + int senderStageId = node.getSenderStageId(); + children.add(_jsonStatsByStage.apply(senderStageId)); + json.set(CHILDREN_KEY, children); + return json; + } + + @Override + public ObjectNode visitMailboxSend(MailboxSendNode node, Void context) { + return recursiveCase(node, MultiStageOperator.Type.MAILBOX_SEND); + } + + @Override + public ObjectNode visitProject(ProjectNode node, Void context) { + return recursiveCase(node, MultiStageOperator.Type.TRANSFORM); + } + + @Override + public ObjectNode visitSort(SortNode node, Void context) { + return recursiveCase(node, MultiStageOperator.Type.SORT_OR_LIMIT); + } + + @Override + public ObjectNode visitTableScan(TableScanNode node, Void context) { + return recursiveCase(node, MultiStageOperator.Type.LEAF); + } + + @Override + public ObjectNode visitValue(ValueNode node, Void context) { + return recursiveCase(node, MultiStageOperator.Type.LITERAL); + } + + @Override + public ObjectNode visitWindow(WindowNode node, Void context) { + return recursiveCase(node, MultiStageOperator.Type.WINDOW); + } + + @Override + public ObjectNode visitSetOp(SetOpNode node, Void context) { + MultiStageOperator.Type type; + switch (node.getSetOpType()) { + case UNION: + type = MultiStageOperator.Type.UNION; + break; + case INTERSECT: + type = MultiStageOperator.Type.INTERSECT; + break; + case MINUS: + type = MultiStageOperator.Type.MINUS; + break; + default: + throw new IllegalStateException("Unexpected set op type: " + node.getSetOpType()); + } + return recursiveCase(node, type); + } + + @Override + public ObjectNode visitExchange(ExchangeNode node, Void context) { + throw new UnsupportedOperationException("ExchangeNode should not be visited"); + } +} diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/MultiStageStatsTreeBuilder.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/MultiStageStatsTreeBuilder.java new file mode 100644 index 000000000000..74c81ed82060 --- /dev/null +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/MultiStageStatsTreeBuilder.java @@ -0,0 +1,43 @@ +/** + * 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.pinot.query.runtime; + +import com.fasterxml.jackson.databind.node.ObjectNode; +import java.util.List; +import org.apache.pinot.query.planner.plannode.PlanNode; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; + + +public class MultiStageStatsTreeBuilder { + private final List _planNodes; + private final List _queryStats; + + public MultiStageStatsTreeBuilder(List planNodes, + List queryStats) { + _planNodes = planNodes; + _queryStats = queryStats; + } + + public ObjectNode jsonStatsByStage(int stage) { + MultiStageQueryStats.StageStats stageStats = _queryStats.get(stage); + PlanNode planNode = _planNodes.get(stage); + InStageStatsTreeBuilder treeBuilder = new InStageStatsTreeBuilder(stageStats, this::jsonStatsByStage); + return planNode.visit(treeBuilder, null); + } +} diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java index fe078e87c6cf..0032545c0f1e 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java @@ -26,6 +26,7 @@ import java.util.concurrent.TimeoutException; import javax.annotation.Nullable; import org.apache.helix.HelixManager; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.metrics.ServerMetrics; import org.apache.pinot.common.utils.config.QueryOptionsUtils; import org.apache.pinot.core.data.manager.InstanceDataManager; @@ -42,6 +43,7 @@ import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.executor.ExecutorServiceUtils; import org.apache.pinot.query.runtime.executor.OpChainSchedulerService; +import org.apache.pinot.query.runtime.operator.MailboxSendOperator; import org.apache.pinot.query.runtime.operator.OpChain; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.apache.pinot.query.runtime.plan.PhysicalPlanVisitor; @@ -177,8 +179,9 @@ public void processQuery(WorkerMetadata workerMetadata, StagePlan stagePlan, Map receiverMailboxInfos); for (RoutingInfo routingInfo : routingInfos) { try { + StatMap statMap = new StatMap<>(MailboxSendOperator.StatKey.class); _mailboxService.getSendingMailbox(routingInfo.getHostname(), routingInfo.getPort(), - routingInfo.getMailboxId(), deadlineMs).send(errorBlock); + routingInfo.getMailboxId(), deadlineMs, statMap).send(errorBlock); } catch (TimeoutException e) { LOGGER.warn("Timed out sending error block to mailbox: {} for request: {}, stage: {}", routingInfo.getMailboxId(), requestId, stageId, e); diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/TransferableBlock.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/TransferableBlock.java index e38b1e272360..f42fed4eccba 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/TransferableBlock.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/TransferableBlock.java @@ -19,22 +19,24 @@ package org.apache.pinot.query.runtime.blocks; import com.google.common.base.Preconditions; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import javax.annotation.Nullable; import org.apache.pinot.common.datablock.ColumnarDataBlock; import org.apache.pinot.common.datablock.DataBlock; import org.apache.pinot.common.datablock.DataBlockUtils; import org.apache.pinot.common.datablock.MetadataBlock; import org.apache.pinot.common.datablock.RowDataBlock; -import org.apache.pinot.common.response.ProcessingException; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.core.common.Block; import org.apache.pinot.core.common.datablock.DataBlockBuilder; import org.apache.pinot.core.util.DataBlockExtractUtils; -import org.apache.pinot.query.runtime.operator.OperatorStats; -import org.apache.pinot.query.runtime.operator.utils.OperatorUtils; - +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; /** * A {@code TransferableBlock} is a wrapper around {@link DataBlock} for transferring data using @@ -42,12 +44,15 @@ */ public class TransferableBlock implements Block { private final DataBlock.Type _type; + @Nullable private final DataSchema _dataSchema; private final int _numRows; private List _container; private DataBlock _dataBlock; private Map _errCodeToExceptionMap; + @Nullable + private final MultiStageQueryStats _queryStats; public TransferableBlock(List container, DataSchema dataSchema, DataBlock.Type type) { _container = container; @@ -57,6 +62,7 @@ public TransferableBlock(List container, DataSchema dataSchema, DataBl _type = type; _numRows = _container.size(); _errCodeToExceptionMap = new HashMap<>(); + _queryStats = null; } public TransferableBlock(DataBlock dataBlock) { @@ -66,19 +72,49 @@ public TransferableBlock(DataBlock dataBlock) { : dataBlock instanceof RowDataBlock ? DataBlock.Type.ROW : DataBlock.Type.METADATA; _numRows = _dataBlock.getNumberOfRows(); _errCodeToExceptionMap = null; + _queryStats = null; + } + + public TransferableBlock(MultiStageQueryStats stats) { + _queryStats = stats; + _type = DataBlock.Type.METADATA; + _numRows = 0; + _dataSchema = null; + _errCodeToExceptionMap = null; } - public Map getResultMetadata() { + public List getSerializedStatsByStage() { if (isSuccessfulEndOfStreamBlock()) { - return OperatorUtils.getOperatorStatsFromMetadata((MetadataBlock) _dataBlock); + List statsByStage; + if (_dataBlock instanceof MetadataBlock) { + statsByStage = ((MetadataBlock) _dataBlock).getStatsByStage(); + if (statsByStage == null) { + return new ArrayList<>(); + } + } else { + Preconditions.checkArgument(_queryStats != null, "QueryStats is null for a successful EOS block"); + try { + statsByStage = _queryStats.serialize(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + return statsByStage; } - return new HashMap<>(); + return new ArrayList<>(); + } + + @Nullable + public MultiStageQueryStats getQueryStats() { + return _queryStats; } public int getNumRows() { return _numRows; } + @Nullable public DataSchema getDataSchema() { return _dataSchema; } @@ -112,13 +148,6 @@ public List getContainer() { return _container; } - /** - * Returns whether the data block is already constructed. - */ - public boolean isDataBlockConstructed() { - return _dataBlock != null; - } - /** * Retrieve the binary-packed version of the data block. * If not already constructed. It will use {@link DataBlockBuilder} to construct the binary-packed format from @@ -129,13 +158,23 @@ public boolean isDataBlockConstructed() { public DataBlock getDataBlock() { if (_dataBlock == null) { try { - if (_type == DataBlock.Type.ROW) { - _dataBlock = DataBlockBuilder.buildFromRows(_container, _dataSchema); - } else { - _dataBlock = DataBlockBuilder.buildFromColumns(_container, _dataSchema); + switch (_type) { + case ROW: + _dataBlock = DataBlockBuilder.buildFromRows(_container, _dataSchema); + break; + case COLUMNAR: + _dataBlock = DataBlockBuilder.buildFromColumns(_container, _dataSchema); + break; + case METADATA: + _dataBlock = new MetadataBlock(getSerializedStatsByStage()); + break; + default: + throw new UnsupportedOperationException("Unable to construct block with type: " + _type); + } + if (_errCodeToExceptionMap != null) { + _dataBlock.getExceptions().putAll(_errCodeToExceptionMap); + _errCodeToExceptionMap = null; } - _dataBlock.getExceptions().putAll(_errCodeToExceptionMap); - _errCodeToExceptionMap = null; } catch (Exception e) { throw new RuntimeException("Unable to create DataBlock", e); } @@ -147,18 +186,6 @@ public Map getExceptions() { return _dataBlock != null ? _dataBlock.getExceptions() : _errCodeToExceptionMap; } - public void addException(ProcessingException processingException) { - addException(processingException.getErrorCode(), processingException.getMessage()); - } - - public void addException(int errCode, String errMsg) { - if (_dataBlock != null) { - _dataBlock.addException(errCode, errMsg); - } else { - _errCodeToExceptionMap.put(errCode, errMsg); - } - } - /** * Return the type of block (one of ROW, COLUMNAR, or METADATA). * @@ -208,6 +235,9 @@ private boolean isType(MetadataBlock.MetadataBlockType type) { return false; } + if (_queryStats != null) { + return MetadataBlock.MetadataBlockType.EOS == type; + } MetadataBlock metadata = (MetadataBlock) _dataBlock; return metadata.getType() == type; } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/TransferableBlockUtils.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/TransferableBlockUtils.java index 355b6fe294da..2f58711033b1 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/TransferableBlockUtils.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/TransferableBlockUtils.java @@ -26,11 +26,13 @@ import java.util.Map; import org.apache.pinot.common.datablock.DataBlock; import org.apache.pinot.common.datablock.DataBlockUtils; +import org.apache.pinot.common.datablock.MetadataBlock; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; public final class TransferableBlockUtils { private static final int MEDIAN_COLUMN_SIZE_BYTES = 8; - private static final TransferableBlock EMPTY_EOS = new TransferableBlock(DataBlockUtils.getEndOfStreamDataBlock()); + private static final TransferableBlock EMPTY_EOS = new TransferableBlock(MetadataBlock.newEos()); private TransferableBlockUtils() { // do not instantiate. @@ -40,8 +42,12 @@ public static TransferableBlock getEndOfStreamTransferableBlock() { return EMPTY_EOS; } - public static TransferableBlock getEndOfStreamTransferableBlock(Map statsMap) { - return new TransferableBlock(DataBlockUtils.getEndOfStreamDataBlock(statsMap)); + public static TransferableBlock getEndOfStreamTransferableBlock(MultiStageQueryStats stats) { + return new TransferableBlock(stats); + } + + public static TransferableBlock wrap(DataBlock dataBlock) { + return new TransferableBlock(dataBlock); } public static TransferableBlock getErrorTransferableBlock(Exception e) { diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java index 440c79200b7c..ef0f9a7d12fe 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java @@ -51,7 +51,6 @@ public void runJob() { Throwable thrown = null; try { LOGGER.trace("({}): Executing", operatorChain); - operatorChain.getStats().executing(); TransferableBlock result = operatorChain.getRoot().nextBlock(); while (!result.isEndOfStreamBlock()) { result = operatorChain.getRoot().nextBlock(); @@ -59,13 +58,13 @@ public void runJob() { isFinished = true; if (result.isErrorBlock()) { returnedErrorBlock = result; - LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(), + LOGGER.error("({}): Completed erroneously {} {}", operatorChain, result.getQueryStats(), result.getExceptions()); } else { - LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats()); + LOGGER.debug("({}): Completed {}", operatorChain, result.getQueryStats()); } } catch (Exception e) { - LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e); + LOGGER.error("({}): Failed to execute operator chain!", operatorChain, e); thrown = e; } finally { _submittedOpChainMap.remove(operatorChain.getId()); diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java index 6b8bc3bcc40c..239a2e5eb9f0 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java @@ -29,7 +29,7 @@ import org.apache.calcite.sql.SqlKind; import org.apache.pinot.calcite.rel.hint.PinotHintOptions; import org.apache.pinot.common.datablock.DataBlock; -import org.apache.pinot.common.datatable.DataTable; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.request.Literal; import org.apache.pinot.common.request.context.ExpressionContext; import org.apache.pinot.common.request.context.FunctionContext; @@ -48,9 +48,10 @@ import org.apache.pinot.query.planner.plannode.AbstractPlanNode; import org.apache.pinot.query.planner.plannode.AggregateNode.AggType; import org.apache.pinot.query.runtime.blocks.TransferableBlock; -import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.roaringbitmap.RoaringBitmap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -59,6 +60,7 @@ * When the list of aggregation calls is empty, this class is used to calculate distinct result based on group by keys. */ public class AggregateOperator extends MultiStageOperator { + private static final Logger LOGGER = LoggerFactory.getLogger(AggregateOperator.class); private static final String EXPLAIN_NAME = "AGGREGATE_OPERATOR"; private static final CountAggregationFunction COUNT_STAR_AGG_FUNCTION = new CountAggregationFunction(Collections.singletonList(ExpressionContext.forIdentifier("*")), false); @@ -69,12 +71,15 @@ public class AggregateOperator extends MultiStageOperator { private final AggType _aggType; private final MultistageAggregationExecutor _aggregationExecutor; private final MultistageGroupByExecutor _groupByExecutor; + @Nullable + private TransferableBlock _eosBlock; + private final StatMap _statMap = new StatMap<>(StatKey.class); private boolean _hasConstructedAggregateBlock; - public AggregateOperator(OpChainExecutionContext context, MultiStageOperator inputOperator, DataSchema resultSchema, - List aggCalls, List groupSet, AggType aggType, List filterArgIndices, - @Nullable AbstractPlanNode.NodeHint nodeHint) { + public AggregateOperator(OpChainExecutionContext context, MultiStageOperator inputOperator, + DataSchema resultSchema, List aggCalls, List groupSet, AggType aggType, + List filterArgIndices, @Nullable AbstractPlanNode.NodeHint nodeHint) { super(context); _inputOperator = inputOperator; _resultSchema = resultSchema; @@ -118,6 +123,22 @@ public AggregateOperator(OpChainExecutionContext context, MultiStageOperator inp } } + @Override + public void registerExecution(long time, int numRows) { + _statMap.merge(StatKey.EXECUTION_TIME_MS, time); + _statMap.merge(StatKey.EMITTED_ROWS, numRows); + } + + @Override + public Type getOperatorType() { + return Type.AGGREGATE; + } + + @Override + protected Logger logger() { + return LOGGER; + } + @Override public List getChildOperators() { return ImmutableList.of(_inputOperator); @@ -132,29 +153,31 @@ public String toExplainString() { @Override protected TransferableBlock getNextBlock() { if (_hasConstructedAggregateBlock) { - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + assert _eosBlock != null; + return _eosBlock; } TransferableBlock finalBlock = _aggregationExecutor != null ? consumeAggregation() : consumeGroupBy(); // returning upstream error block if finalBlock contains error. if (finalBlock.isErrorBlock()) { return finalBlock; } - return produceAggregatedBlock(); + assert finalBlock.isSuccessfulEndOfStreamBlock() : "Final block must be EOS block"; + _eosBlock = updateEosBlock(finalBlock, _statMap); + return produceAggregatedBlock(finalBlock); } - private TransferableBlock produceAggregatedBlock() { + private TransferableBlock produceAggregatedBlock(TransferableBlock finalBlock) { _hasConstructedAggregateBlock = true; if (_aggregationExecutor != null) { return new TransferableBlock(_aggregationExecutor.getResult(), _resultSchema, DataBlock.Type.ROW); } else { List rows = _groupByExecutor.getResult(); if (rows.isEmpty()) { - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + return _eosBlock; } else { TransferableBlock dataBlock = new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW); if (_groupByExecutor.isNumGroupsLimitReached()) { - OperatorStats operatorStats = _opChainStats.getOperatorStats(_context, _operatorId); - operatorStats.recordSingleStat(DataTable.MetadataKey.NUM_GROUPS_LIMIT_REACHED.getName(), "true"); + _statMap.merge(StatKey.NUM_GROUPS_LIMIT_REACHED, true); _inputOperator.earlyTerminate(); } return dataBlock; @@ -407,4 +430,31 @@ static Object[] getIntermediateResults(AggregationFunction aggFunctions, T return DataBlockExtractUtils.extractColumn(block.getDataBlock(), colId); } } + + public enum StatKey implements StatMap.Key { + EXECUTION_TIME_MS(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }, + EMITTED_ROWS(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }, + NUM_GROUPS_LIMIT_REACHED(StatMap.Type.BOOLEAN); + + private final StatMap.Type _type; + + StatKey(StatMap.Type type) { + _type = type; + } + + @Override + public StatMap.Type getType() { + return _type; + } + } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/BaseMailboxReceiveOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/BaseMailboxReceiveOperator.java index a88e12273958..d60a0e102d30 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/BaseMailboxReceiveOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/BaseMailboxReceiveOperator.java @@ -24,6 +24,7 @@ import java.util.stream.Collectors; import javax.annotation.Nullable; import org.apache.calcite.rel.RelDistribution; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.query.mailbox.MailboxService; import org.apache.pinot.query.mailbox.ReceivingMailbox; import org.apache.pinot.query.planner.physical.MailboxIdUtils; @@ -48,6 +49,8 @@ public abstract class BaseMailboxReceiveOperator extends MultiStageOperator { protected final RelDistribution.Type _exchangeType; protected final List _mailboxIds; protected final BlockingMultiStreamConsumer.OfTransferableBlock _multiConsumer; + protected final List> _receivingStats; + protected final StatMap _statMap = new StatMap<>(StatKey.class); public BaseMailboxReceiveOperator(OpChainExecutionContext context, RelDistribution.Type exchangeType, int senderStageId) { @@ -69,8 +72,9 @@ public BaseMailboxReceiveOperator(OpChainExecutionContext context, RelDistributi List asyncStreams = _mailboxIds.stream() .map(mailboxId -> new ReadMailboxAsyncStream(_mailboxService.getReceivingMailbox(mailboxId), this)) .collect(Collectors.toList()); - _multiConsumer = - new BlockingMultiStreamConsumer.OfTransferableBlock(context.getId(), context.getDeadlineMs(), asyncStreams); + _receivingStats = asyncStreams.stream().map(stream -> stream._mailbox.getStatMap()).collect(Collectors.toList()); + _multiConsumer = new BlockingMultiStreamConsumer.OfTransferableBlock(context, asyncStreams); + _statMap.merge(StatKey.FAN_IN, _mailboxIds.size()); } @Override @@ -79,6 +83,11 @@ protected void earlyTerminate() { _multiConsumer.earlyTerminate(); } + @Override + public Type getOperatorType() { + return Type.MAILBOX_RECEIVE; + } + @Override public List getChildOperators() { return Collections.emptyList(); @@ -96,6 +105,29 @@ public void cancel(Throwable t) { _multiConsumer.cancel(t); } + @Override + protected TransferableBlock updateEosBlock(TransferableBlock upstreamEos, StatMap statMap) { + for (StatMap receivingStats : _receivingStats) { + addReceivingStats(receivingStats); + } + return super.updateEosBlock(upstreamEos, statMap); + } + + @Override + public void registerExecution(long time, int numRows) { + _statMap.merge(StatKey.EXECUTION_TIME_MS, time); + _statMap.merge(StatKey.EMITTED_ROWS, numRows); + } + + private void addReceivingStats(StatMap from) { + _statMap.merge(StatKey.RAW_MESSAGES, from.getInt(ReceivingMailbox.StatKey.DESERIALIZED_MESSAGES)); + _statMap.merge(StatKey.DESERIALIZED_BYTES, from.getLong(ReceivingMailbox.StatKey.DESERIALIZED_BYTES)); + _statMap.merge(StatKey.DESERIALIZATION_TIME_MS, from.getLong(ReceivingMailbox.StatKey.DESERIALIZATION_TIME_MS)); + _statMap.merge(StatKey.IN_MEMORY_MESSAGES, from.getInt(ReceivingMailbox.StatKey.IN_MEMORY_MESSAGES)); + _statMap.merge(StatKey.DOWNSTREAM_WAIT_MS, from.getLong(ReceivingMailbox.StatKey.OFFER_CPU_TIME_MS)); + _statMap.merge(StatKey.UPSTREAM_WAIT_MS, from.getLong(ReceivingMailbox.StatKey.WAIT_CPU_TIME_MS)); + } + private static class ReadMailboxAsyncStream implements AsyncStream { private final ReceivingMailbox _mailbox; private final BaseMailboxReceiveOperator _operator; @@ -116,7 +148,6 @@ public TransferableBlock poll() { TransferableBlock block = _mailbox.poll(); if (block != null && block.isSuccessfulEndOfStreamBlock()) { _operator._mailboxService.releaseReceivingMailbox(_mailbox); - _operator._opChainStats.getOperatorStatsMap().putAll(block.getResultMetadata()); } return block; } @@ -136,4 +167,71 @@ public void cancel() { _mailbox.cancel(); } } + + public enum StatKey implements StatMap.Key { + EXECUTION_TIME_MS(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }, + EMITTED_ROWS(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }, + /** + * How many send mailboxes are being read by this receive operator. + *

+ * Clock time will be proportional to this number and the parallelism of the stage. + */ + FAN_IN(StatMap.Type.INT) { + @Override + public int merge(int value1, int value2) { + return Math.max(value1, value2); + } + }, + /** + * How many messages have been received in heap format by this mailbox. + *

+ * The lower the relation between RAW_MESSAGES and IN_MEMORY_MESSAGES, the more efficient the exchange is. + */ + IN_MEMORY_MESSAGES(StatMap.Type.INT), + /** + * How many messages have been received in raw format and therefore deserialized by this mailbox. + *

+ * The higher the relation between RAW_MESSAGES and IN_MEMORY_MESSAGES, the less efficient the exchange is. + */ + RAW_MESSAGES(StatMap.Type.INT), + /** + * How many bytes have been deserialized by this mailbox. + *

+ * A high number here indicates that the mailbox is receiving a lot of data from other servers. + */ + DESERIALIZED_BYTES(StatMap.Type.LONG), + /** + * How long (in CPU time) it took to deserialize the raw messages received by this mailbox. + */ + DESERIALIZATION_TIME_MS(StatMap.Type.LONG), + /** + * How long (in CPU time) it took to offer the messages to downstream operator. + */ + DOWNSTREAM_WAIT_MS(StatMap.Type.LONG), + /** + * How long (in CPU time) it took to wait for the messages to be offered to downstream operator. + */ + UPSTREAM_WAIT_MS(StatMap.Type.LONG); + + private final StatMap.Type _type; + + StatKey(StatMap.Type type) { + _type = type; + } + + @Override + public StatMap.Type getType() { + return _type; + } + } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/FilterOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/FilterOperator.java index 5747d6191dbd..8cab4899bc42 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/FilterOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/FilterOperator.java @@ -24,6 +24,7 @@ import java.util.List; import javax.annotation.Nullable; import org.apache.pinot.common.datablock.DataBlock; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; import org.apache.pinot.query.planner.logical.RexExpression; @@ -32,6 +33,8 @@ import org.apache.pinot.query.runtime.operator.operands.TransformOperandFactory; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.apache.pinot.spi.utils.BooleanUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /* @@ -48,11 +51,14 @@ Note: Scalar functions are the ones we have in v1 engine and only do function name and arg # matching. */ public class FilterOperator extends MultiStageOperator { + + private static final Logger LOGGER = LoggerFactory.getLogger(FilterOperator.class); private static final String EXPLAIN_NAME = "FILTER"; private final MultiStageOperator _upstreamOperator; private final TransformOperand _filterOperand; private final DataSchema _dataSchema; + private final StatMap _statMap = new StatMap<>(StatKey.class); public FilterOperator(OpChainExecutionContext context, MultiStageOperator upstreamOperator, DataSchema dataSchema, RexExpression filter) { @@ -64,6 +70,22 @@ public FilterOperator(OpChainExecutionContext context, MultiStageOperator upstre "Filter operand must return BOOLEAN, got: %s", _filterOperand.getResultType()); } + @Override + public void registerExecution(long time, int numRows) { + _statMap.merge(StatKey.EXECUTION_TIME_MS, time); + _statMap.merge(StatKey.EMITTED_ROWS, numRows); + } + + @Override + public Type getOperatorType() { + return Type.FILTER; + } + + @Override + protected Logger logger() { + return LOGGER; + } + @Override public List getChildOperators() { return ImmutableList.of(_upstreamOperator); @@ -79,7 +101,10 @@ public String toExplainString() { protected TransferableBlock getNextBlock() { TransferableBlock block = _upstreamOperator.nextBlock(); if (block.isEndOfStreamBlock()) { - return block; + if (block.isErrorBlock()) { + return block; + } + return updateEosBlock(block, _statMap); } List resultRows = new ArrayList<>(); for (Object[] row : block.getContainer()) { @@ -90,4 +115,29 @@ protected TransferableBlock getNextBlock() { } return new TransferableBlock(resultRows, _dataSchema, DataBlock.Type.ROW); } + + public enum StatKey implements StatMap.Key { + EXECUTION_TIME_MS(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }, + EMITTED_ROWS(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }; + private final StatMap.Type _type; + + StatKey(StatMap.Type type) { + _type = type; + } + + @Override + public StatMap.Type getType() { + return _type; + } + } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/HashJoinOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/HashJoinOperator.java index 92d50dc54e49..aa2be1850c09 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/HashJoinOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/HashJoinOperator.java @@ -31,7 +31,7 @@ import org.apache.calcite.rel.core.JoinRelType; import org.apache.pinot.calcite.rel.hint.PinotHintOptions; import org.apache.pinot.common.datablock.DataBlock; -import org.apache.pinot.common.datatable.DataTable; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.exception.QueryException; import org.apache.pinot.common.response.ProcessingException; import org.apache.pinot.common.utils.DataSchema; @@ -45,9 +45,12 @@ import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.operator.operands.TransformOperand; import org.apache.pinot.query.runtime.operator.operands.TransformOperandFactory; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.apache.pinot.spi.utils.BooleanUtils; import org.apache.pinot.spi.utils.CommonConstants.MultiStageQueryRunner.JoinOverFlowMode; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -65,6 +68,8 @@ // TODO: Move inequi out of hashjoin. (https://github.com/apache/pinot/issues/9728) // TODO: Support memory size based resource limit. public class HashJoinOperator extends MultiStageOperator { + + private static final Logger LOGGER = LoggerFactory.getLogger(HashJoinOperator.class); private static final String EXPLAIN_NAME = "HASH_JOIN"; private static final int INITIAL_HEURISTIC_SIZE = 16; private static final int DEFAULT_MAX_ROWS_IN_JOIN = 1024 * 1024; // 2^20, around 1MM rows @@ -88,6 +93,7 @@ public class HashJoinOperator extends MultiStageOperator { private final int _resultColumnSize; private final List _joinClauseEvaluators; private boolean _isHashTableBuilt; + private final StatMap _statMap = new StatMap<>(StatKey.class); // Used by non-inner join. // Needed to indicate we have finished processing all results after returning last block. @@ -112,6 +118,11 @@ public class HashJoinOperator extends MultiStageOperator { private int _currentRowsInHashTable = 0; + @Nullable + private MultiStageQueryStats _rightSideStats = null; + @Nullable + private MultiStageQueryStats _leftSideStats = null; + public HashJoinOperator(OpChainExecutionContext context, MultiStageOperator leftTableOperator, MultiStageOperator rightTableOperator, DataSchema leftSchema, JoinNode node) { super(context); @@ -146,6 +157,22 @@ public HashJoinOperator(OpChainExecutionContext context, MultiStageOperator left _joinOverflowMode = getJoinOverflowMode(metadata, node.getJoinHints()); } + @Override + public void registerExecution(long time, int numRows) { + _statMap.merge(StatKey.EXECUTION_TIME_MS, time); + _statMap.merge(StatKey.EMITTED_ROWS, numRows); + } + + @Override + public Type getOperatorType() { + return Type.HASH_JOIN; + } + + @Override + protected Logger logger() { + return LOGGER; + } + private int getMaxRowInJoin(Map opChainMetadata, @Nullable AbstractPlanNode.NodeHint nodeHint) { if (nodeHint != null) { Map joinOptions = nodeHint._hintOptions.get(PinotHintOptions.JOIN_HINT_OPTIONS); @@ -191,7 +218,8 @@ public String toExplainString() { protected TransferableBlock getNextBlock() throws ProcessingException { if (_isTerminated) { - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + assert _leftSideStats != null; + return TransferableBlockUtils.getEndOfStreamTransferableBlock(_leftSideStats); } if (!_isHashTableBuilt) { // Build JOIN hash table @@ -207,6 +235,7 @@ protected TransferableBlock getNextBlock() private void buildBroadcastHashTable() throws ProcessingException { + long startTime = System.currentTimeMillis(); TransferableBlock rightBlock = _rightTableOperator.nextBlock(); while (!TransferableBlockUtils.isEndOfStream(rightBlock)) { List container = rightBlock.getContainer(); @@ -223,8 +252,7 @@ private void buildBroadcastHashTable() // Just fill up the buffer. int remainingRows = _maxRowsInHashTable - _currentRowsInHashTable; container = container.subList(0, remainingRows); - OperatorStats operatorStats = _opChainStats.getOperatorStats(_context, _operatorId); - operatorStats.recordSingleStat(DataTable.MetadataKey.MAX_ROWS_IN_JOIN_REACHED.getName(), "true"); + _statMap.merge(StatKey.MAX_ROWS_IN_JOIN_REACHED, true); // setting only the rightTableOperator to be early terminated and awaits EOS block next. _rightTableOperator.earlyTerminate(); } @@ -246,7 +274,10 @@ private void buildBroadcastHashTable() _upstreamErrorBlock = rightBlock; } else { _isHashTableBuilt = true; + _rightSideStats = rightBlock.getQueryStats(); + assert _rightSideStats != null; } + _statMap.merge(StatKey.TIME_BUILDING_HASH_TABLE_MS, System.currentTimeMillis() - startTime); } private TransferableBlock buildJoinedDataBlock(TransferableBlock leftBlock) { @@ -255,8 +286,13 @@ private TransferableBlock buildJoinedDataBlock(TransferableBlock leftBlock) { return _upstreamErrorBlock; } if (leftBlock.isSuccessfulEndOfStreamBlock()) { + assert _rightSideStats != null; + _leftSideStats = leftBlock.getQueryStats(); + assert _leftSideStats != null; + _leftSideStats.mergeInOrder(_rightSideStats, getOperatorType(), _statMap); + if (!needUnmatchedRightRows()) { - return leftBlock; + return TransferableBlockUtils.getEndOfStreamTransferableBlock(_leftSideStats); } // TODO: Moved to a different function. // Return remaining non-matched rows for non-inner join. @@ -390,4 +426,35 @@ private boolean needUnmatchedRightRows() { private boolean needUnmatchedLeftRows() { return _joinType == JoinRelType.LEFT || _joinType == JoinRelType.FULL; } + + public enum StatKey implements StatMap.Key { + EXECUTION_TIME_MS(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }, + EMITTED_ROWS(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }, + MAX_ROWS_IN_JOIN_REACHED(StatMap.Type.BOOLEAN), + /** + * How long (CPU time) has been spent on building the hash table. + */ + TIME_BUILDING_HASH_TABLE_MS(StatMap.Type.LONG); + + private final StatMap.Type _type; + + StatKey(StatMap.Type type) { + _type = type; + } + + @Override + public StatMap.Type getType() { + return _type; + } + } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/IntersectOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/IntersectOperator.java index b0ba3c6fa347..2ef003801d3e 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/IntersectOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/IntersectOperator.java @@ -23,19 +23,33 @@ import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.core.data.table.Record; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Intersect operator. */ public class IntersectOperator extends SetOperator { + private static final Logger LOGGER = LoggerFactory.getLogger(IntersectOperator.class); private static final String EXPLAIN_NAME = "INTERSECT"; - public IntersectOperator(OpChainExecutionContext opChainExecutionContext, List upstreamOperators, + public IntersectOperator(OpChainExecutionContext opChainExecutionContext, + List upstreamOperators, DataSchema dataSchema) { super(opChainExecutionContext, upstreamOperators, dataSchema); } + @Override + public Type getOperatorType() { + return Type.INTERSECT; + } + + @Override + protected Logger logger() { + return LOGGER; + } + @Nullable @Override public String toExplainString() { diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/LeafStageTransferableBlockOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/LeafStageTransferableBlockOperator.java index ea5a47df9863..75dc55a15016 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/LeafStageTransferableBlockOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/LeafStageTransferableBlockOperator.java @@ -33,10 +33,13 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import javax.annotation.Nullable; import org.apache.pinot.common.datablock.DataBlock; import org.apache.pinot.common.datablock.MetadataBlock; import org.apache.pinot.common.datatable.DataTable; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.request.context.ExpressionContext; +import org.apache.pinot.common.response.broker.BrokerResponseNativeV2; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; import org.apache.pinot.common.utils.config.QueryOptionsUtils; @@ -52,8 +55,11 @@ import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.operator.utils.TypeUtils; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.apache.pinot.spi.utils.CommonConstants.Broker.Request.QueryOptionValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -70,6 +76,8 @@ * */ public class LeafStageTransferableBlockOperator extends MultiStageOperator { + + private static final Logger LOGGER = LoggerFactory.getLogger(LeafStageTransferableBlockOperator.class); private static final String EXPLAIN_NAME = "LEAF_STAGE_TRANSFER_OPERATOR"; // Use a special results block to indicate that this is the last results block @@ -85,7 +93,7 @@ public class LeafStageTransferableBlockOperator extends MultiStageOperator { private Future _executionFuture; private volatile Map _exceptions; - private volatile Map _executionStats; + private final StatMap _statMap = new StatMap<>(StatKey.class); public LeafStageTransferableBlockOperator(OpChainExecutionContext context, List requests, DataSchema dataSchema, QueryExecutor queryExecutor, ExecutorService executorService) { @@ -99,6 +107,24 @@ public LeafStageTransferableBlockOperator(OpChainExecutionContext context, List< Integer maxStreamingPendingBlocks = QueryOptionsUtils.getMaxStreamingPendingBlocks(context.getOpChainMetadata()); _blockingQueue = new ArrayBlockingQueue<>(maxStreamingPendingBlocks != null ? maxStreamingPendingBlocks : QueryOptionValue.DEFAULT_MAX_STREAMING_PENDING_BLOCKS); + String tableName = context.getLeafStageContext().getStagePlan().getStageMetadata().getTableName(); + _statMap.merge(StatKey.TABLE, tableName); + } + + @Override + public void registerExecution(long time, int numRows) { + _statMap.merge(StatKey.EXECUTION_TIME_MS, time); + _statMap.merge(StatKey.EMITTED_ROWS, numRows); + } + + @Override + public Type getOperatorType() { + return Type.LEAF; + } + + @Override + protected Logger logger() { + return LOGGER; } @Override @@ -135,14 +161,125 @@ protected TransferableBlock getNextBlock() } } - private TransferableBlock constructMetadataBlock() { - // All data blocks have been returned. Record the stats and return EOS. - Map executionStats = _executionStats; + private void mergeExecutionStats(@Nullable Map executionStats) { if (executionStats != null) { - OperatorStats operatorStats = _opChainStats.getOperatorStats(_context, getOperatorId()); - operatorStats.recordExecutionStats(executionStats); + for (Map.Entry entry : executionStats.entrySet()) { + DataTable.MetadataKey key = DataTable.MetadataKey.getByName(entry.getKey()); + if (key == null) { + LOGGER.debug("Skipping unknown execution stat: {}", entry.getKey()); + continue; + } + switch (key) { + case UNKNOWN: + LOGGER.debug("Skipping unknown execution stat: {}", entry.getKey()); + break; + case TABLE: + _statMap.merge(StatKey.TABLE, entry.getValue()); + break; + case NUM_DOCS_SCANNED: + _statMap.merge(StatKey.NUM_DOCS_SCANNED, Long.parseLong(entry.getValue())); + break; + case NUM_ENTRIES_SCANNED_IN_FILTER: + _statMap.merge(StatKey.NUM_ENTRIES_SCANNED_IN_FILTER, Long.parseLong(entry.getValue())); + break; + case NUM_ENTRIES_SCANNED_POST_FILTER: + _statMap.merge(StatKey.NUM_ENTRIES_SCANNED_POST_FILTER, Long.parseLong(entry.getValue())); + break; + case NUM_SEGMENTS_QUERIED: + _statMap.merge(StatKey.NUM_SEGMENTS_QUERIED, Integer.parseInt(entry.getValue())); + break; + case NUM_SEGMENTS_PROCESSED: + _statMap.merge(StatKey.NUM_SEGMENTS_PROCESSED, Integer.parseInt(entry.getValue())); + break; + case NUM_SEGMENTS_MATCHED: + _statMap.merge(StatKey.NUM_SEGMENTS_MATCHED, Integer.parseInt(entry.getValue())); + break; + case NUM_CONSUMING_SEGMENTS_QUERIED: + _statMap.merge(StatKey.NUM_CONSUMING_SEGMENTS_QUERIED, Integer.parseInt(entry.getValue())); + break; + case MIN_CONSUMING_FRESHNESS_TIME_MS: + _statMap.merge(StatKey.MIN_CONSUMING_FRESHNESS_TIME_MS, Long.parseLong(entry.getValue())); + break; + case TOTAL_DOCS: + _statMap.merge(StatKey.TOTAL_DOCS, Long.parseLong(entry.getValue())); + break; + case NUM_GROUPS_LIMIT_REACHED: + _statMap.merge(StatKey.NUM_GROUPS_LIMIT_REACHED, Boolean.parseBoolean(entry.getValue())); + break; + case TIME_USED_MS: + _statMap.merge(StatKey.EXECUTION_TIME_MS, Long.parseLong(entry.getValue())); + break; + case TRACE_INFO: + LOGGER.debug("Skipping trace info: {}", entry.getValue()); + break; + case REQUEST_ID: + LOGGER.debug("Skipping request ID: {}", entry.getValue()); + break; + case NUM_RESIZES: + _statMap.merge(StatKey.NUM_RESIZES, Integer.parseInt(entry.getValue())); + break; + case RESIZE_TIME_MS: + _statMap.merge(StatKey.RESIZE_TIME_MS, Long.parseLong(entry.getValue())); + break; + case THREAD_CPU_TIME_NS: + _statMap.merge(StatKey.THREAD_CPU_TIME_NS, Long.parseLong(entry.getValue())); + break; + case SYSTEM_ACTIVITIES_CPU_TIME_NS: + _statMap.merge(StatKey.SYSTEM_ACTIVITIES_CPU_TIME_NS, Long.parseLong(entry.getValue())); + break; + case RESPONSE_SER_CPU_TIME_NS: + _statMap.merge(StatKey.RESPONSE_SER_CPU_TIME_NS, Long.parseLong(entry.getValue())); + break; + case NUM_SEGMENTS_PRUNED_BY_SERVER: + _statMap.merge(StatKey.NUM_SEGMENTS_PRUNED_BY_SERVER, Integer.parseInt(entry.getValue())); + break; + case NUM_SEGMENTS_PRUNED_INVALID: + _statMap.merge(StatKey.NUM_SEGMENTS_PRUNED_INVALID, Integer.parseInt(entry.getValue())); + break; + case NUM_SEGMENTS_PRUNED_BY_LIMIT: + _statMap.merge(StatKey.NUM_SEGMENTS_PRUNED_BY_LIMIT, Integer.parseInt(entry.getValue())); + break; + case NUM_SEGMENTS_PRUNED_BY_VALUE: + _statMap.merge(StatKey.NUM_SEGMENTS_PRUNED_BY_VALUE, Integer.parseInt(entry.getValue())); + break; + case EXPLAIN_PLAN_NUM_EMPTY_FILTER_SEGMENTS: + LOGGER.debug("Skipping empty filter segments: {}", entry.getValue()); + break; + case EXPLAIN_PLAN_NUM_MATCH_ALL_FILTER_SEGMENTS: + LOGGER.debug("Skipping match all filter segments: {}", entry.getValue()); + break; + case NUM_CONSUMING_SEGMENTS_PROCESSED: + _statMap.merge(StatKey.NUM_CONSUMING_SEGMENTS_PROCESSED, Integer.parseInt(entry.getValue())); + break; + case NUM_CONSUMING_SEGMENTS_MATCHED: + _statMap.merge(StatKey.NUM_CONSUMING_SEGMENTS_MATCHED, Integer.parseInt(entry.getValue())); + break; + case NUM_BLOCKS: + _statMap.merge(StatKey.NUM_BLOCKS, Integer.parseInt(entry.getValue())); + break; + case NUM_ROWS: + _statMap.merge(StatKey.EMITTED_ROWS, Integer.parseInt(entry.getValue())); + break; + case OPERATOR_EXECUTION_TIME_MS: + _statMap.merge(StatKey.OPERATOR_EXECUTION_TIME_MS, Long.parseLong(entry.getValue())); + break; + case OPERATOR_EXEC_START_TIME_MS: + _statMap.merge(StatKey.OPERATOR_EXEC_START_TIME_MS, Long.parseLong(entry.getValue())); + break; + case OPERATOR_EXEC_END_TIME_MS: + _statMap.merge(StatKey.OPERATOR_EXEC_END_TIME_MS, Long.parseLong(entry.getValue())); + break; + default: { + throw new IllegalArgumentException("Unhandled V1 execution stat: " + entry.getKey()); + } + } + } } - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + } + + private TransferableBlock constructMetadataBlock() { + MultiStageQueryStats multiStageQueryStats = MultiStageQueryStats.createLeaf(_context.getStageId(), _statMap); + return TransferableBlockUtils.getEndOfStreamTransferableBlock(multiStageQueryStats); } private Future startExecution() { @@ -170,11 +307,14 @@ private Future startExecution() { addResultsBlock(resultsBlock); } // Collect the execution stats - _executionStats = instanceResponseBlock.getResponseMetadata(); + mergeExecutionStats(instanceResponseBlock.getResponseMetadata()); } } else { assert _requests.size() == 2; - Future[] futures = new Future[2]; + Future>[] futures = new Future[2]; + // TODO: this latch mechanism is not the most elegant. We should change it to use a CompletionService. + // In order to interrupt the execution in case of error, we could different mechanisms like throwing in the + // future, or using a shared volatile variable. CountDownLatch latch = new CountDownLatch(2); for (int i = 0; i < 2; i++) { ServerQueryRequest request = _requests.get(i); @@ -190,6 +330,7 @@ private Future startExecution() { // Drain the latch when receiving exception block and not wait for the other thread to finish _exceptions = exceptions; latch.countDown(); + return Collections.emptyMap(); } else { // NOTE: Instance response block might contain data (not metadata only) when all the segments are // pruned. Add the results block if it contains data. @@ -198,16 +339,8 @@ private Future startExecution() { addResultsBlock(resultsBlock); } // Collect the execution stats - Map executionStats = instanceResponseBlock.getResponseMetadata(); - synchronized (LeafStageTransferableBlockOperator.this) { - if (_executionStats == null) { - _executionStats = executionStats; - } else { - aggregateExecutionStats(_executionStats, executionStats); - } - } + return instanceResponseBlock.getResponseMetadata(); } - return null; } finally { latch.countDown(); } @@ -218,9 +351,17 @@ private Future startExecution() { throw new TimeoutException("Timed out waiting for leaf stage to finish"); } // Propagate the exception thrown by the leaf stage + for (Future> future : futures) { + Map stats = + future.get(_context.getDeadlineMs() - System.currentTimeMillis(), TimeUnit.MILLISECONDS); + mergeExecutionStats(stats); + } + } catch (TimeoutException e) { + // Cancel all the futures and throw the exception for (Future future : futures) { - future.get(); + future.cancel(true); } + throw new TimeoutException("Timed out waiting for leaf stage to finish"); } finally { for (Future future : futures) { future.cancel(true); @@ -249,28 +390,16 @@ private void aggregateExecutionStats(Map stats1, Map entry : stats2.entrySet()) { String k2 = entry.getKey(); String v2 = entry.getValue(); - stats1.compute(k2, (k1, v1) -> { - if (v1 == null) { - return v2; - } + stats1.merge(k2, v2, (val1, val2) -> { try { - return Long.toString(Long.parseLong(v1) + Long.parseLong(v2)); + return Long.toString(Long.parseLong(val1) + Long.parseLong(val2)); } catch (Exception e) { - return v1 + "\n" + v2; + return val1 + "\n" + val2; } }); } } - /** - * Leaf stage operators should always collect stats for the tables used in queries - * Otherwise the Broker response will just contain zeros for every stat value - */ - @Override - protected boolean shouldCollectStats() { - return true; - } - @Override public void close() { if (_executionFuture != null) { @@ -419,4 +548,110 @@ public void send(BaseResultsBlock block) addResultsBlock(block); } } + + public enum StatKey implements StatMap.Key { + TABLE(StatMap.Type.STRING, null), + EXECUTION_TIME_MS(StatMap.Type.LONG, BrokerResponseNativeV2.StatKey.TIME_USED_MS) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }, + EMITTED_ROWS(StatMap.Type.LONG, null) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }, + NUM_DOCS_SCANNED(StatMap.Type.LONG), + NUM_ENTRIES_SCANNED_IN_FILTER(StatMap.Type.LONG), + NUM_ENTRIES_SCANNED_POST_FILTER(StatMap.Type.LONG), + NUM_SEGMENTS_QUERIED(StatMap.Type.INT), + NUM_SEGMENTS_PROCESSED(StatMap.Type.INT), + NUM_SEGMENTS_MATCHED(StatMap.Type.INT), + NUM_CONSUMING_SEGMENTS_QUERIED(StatMap.Type.INT), + // the timestamp indicating the freshness of the data queried in consuming segments. + // This can be ingestion timestamp if provided by the stream, or the last index time + MIN_CONSUMING_FRESHNESS_TIME_MS(StatMap.Type.LONG) { + @Override + public long merge(long value1, long value2) { + return StatMap.Key.minPositive(value1, value2); + } + }, + TOTAL_DOCS(StatMap.Type.LONG), + NUM_GROUPS_LIMIT_REACHED(StatMap.Type.BOOLEAN), + NUM_RESIZES(StatMap.Type.INT, null), + RESIZE_TIME_MS(StatMap.Type.LONG, null), + THREAD_CPU_TIME_NS(StatMap.Type.LONG, null), + SYSTEM_ACTIVITIES_CPU_TIME_NS(StatMap.Type.LONG, null), + RESPONSE_SER_CPU_TIME_NS(StatMap.Type.LONG, null) { + @Override + public String getStatName() { + return "responseSerializationCpuTimeNs"; + } + }, + NUM_SEGMENTS_PRUNED_BY_SERVER(StatMap.Type.INT), + NUM_SEGMENTS_PRUNED_INVALID(StatMap.Type.INT), + NUM_SEGMENTS_PRUNED_BY_LIMIT(StatMap.Type.INT), + NUM_SEGMENTS_PRUNED_BY_VALUE(StatMap.Type.INT), + NUM_CONSUMING_SEGMENTS_PROCESSED(StatMap.Type.INT), + NUM_CONSUMING_SEGMENTS_MATCHED(StatMap.Type.INT), + NUM_BLOCKS(StatMap.Type.INT, null), + OPERATOR_EXECUTION_TIME_MS(StatMap.Type.LONG, null), + OPERATOR_EXEC_START_TIME_MS(StatMap.Type.LONG, null) { + @Override + public long merge(long value1, long value2) { + return StatMap.Key.minPositive(value1, value2); + } + }, + OPERATOR_EXEC_END_TIME_MS(StatMap.Type.LONG, null) { + @Override + public long merge(long value1, long value2) { + return Math.max(value1, value2); + } + },; + private final StatMap.Type _type; + @Nullable + private final BrokerResponseNativeV2.StatKey _brokerKey; + + StatKey(StatMap.Type type) { + _type = type; + _brokerKey = BrokerResponseNativeV2.StatKey.valueOf(name()); + } + + StatKey(StatMap.Type type, @Nullable BrokerResponseNativeV2.StatKey brokerKey) { + _type = type; + _brokerKey = brokerKey; + } + + @Override + public StatMap.Type getType() { + return _type; + } + + public void updateBrokerMetadata(StatMap oldMetadata, StatMap stats) { + if (_brokerKey != null) { + switch (_type) { + case LONG: + if (_brokerKey.getType() == StatMap.Type.INT) { + oldMetadata.merge(_brokerKey, (int) stats.getLong(this)); + } else { + oldMetadata.merge(_brokerKey, stats.getLong(this)); + } + break; + case INT: + oldMetadata.merge(_brokerKey, stats.getInt(this)); + break; + case BOOLEAN: + oldMetadata.merge(_brokerKey, stats.getBoolean(this)); + break; + case STRING: + oldMetadata.merge(_brokerKey, stats.getString(this)); + break; + default: + throw new IllegalStateException("Unsupported type: " + _type); + } + } + } + } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/LiteralValueOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/LiteralValueOperator.java index ad0cf2a129c8..3279cb4cd027 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/LiteralValueOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/LiteralValueOperator.java @@ -23,10 +23,12 @@ import java.util.List; import javax.annotation.Nullable; import org.apache.pinot.common.datablock.DataBlock; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.query.planner.logical.RexExpression; import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,6 +41,7 @@ public class LiteralValueOperator extends MultiStageOperator { private final DataSchema _dataSchema; private final TransferableBlock _rexLiteralBlock; private boolean _isLiteralBlockReturned; + private final StatMap _statMap = new StatMap<>(StatKey.class); public LiteralValueOperator(OpChainExecutionContext context, DataSchema dataSchema, List> rexLiteralRows) { @@ -49,6 +52,17 @@ public LiteralValueOperator(OpChainExecutionContext context, DataSchema dataSche _isLiteralBlockReturned = context.getId().getVirtualServerId() != 0; } + @Override + public void registerExecution(long time, int numRows) { + _statMap.merge(StatKey.EXECUTION_TIME_MS, time); + _statMap.merge(StatKey.EMITTED_ROWS, numRows); + } + + @Override + protected Logger logger() { + return LOGGER; + } + @Override public List getChildOperators() { return ImmutableList.of(); @@ -66,10 +80,20 @@ protected TransferableBlock getNextBlock() { _isLiteralBlockReturned = true; return _rexLiteralBlock; } else { - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + return createEosBlock(); } } + protected TransferableBlock createEosBlock() { + return TransferableBlockUtils.getEndOfStreamTransferableBlock( + MultiStageQueryStats.createLiteral(_context.getStageId(), _statMap)); + } + + @Override + public Type getOperatorType() { + return Type.LITERAL; + } + private TransferableBlock constructBlock(List> rexLiteralRows) { List blockContent = new ArrayList<>(); for (List rexLiteralRow : rexLiteralRows) { @@ -81,4 +105,19 @@ private TransferableBlock constructBlock(List> rexLiteralRow } return new TransferableBlock(blockContent, _dataSchema, DataBlock.Type.ROW); } + + public enum StatKey implements StatMap.Key { + EXECUTION_TIME_MS(StatMap.Type.LONG), + EMITTED_ROWS(StatMap.Type.LONG); + private final StatMap.Type _type; + + StatKey(StatMap.Type type) { + _type = type; + } + + @Override + public StatMap.Type getType() { + return _type; + } + } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperator.java index 584b49640f3d..6fa7e3119bf1 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperator.java @@ -22,6 +22,8 @@ import org.apache.pinot.query.mailbox.ReceivingMailbox; import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -29,6 +31,7 @@ * {@link MultiStageOperator#getNextBlock()} API. */ public class MailboxReceiveOperator extends BaseMailboxReceiveOperator { + private static final Logger LOGGER = LoggerFactory.getLogger(MailboxReceiveOperator.class); private static final String EXPLAIN_NAME = "MAILBOX_RECEIVE"; public MailboxReceiveOperator(OpChainExecutionContext context, RelDistribution.Type exchangeType, int senderStageId) { @@ -40,6 +43,11 @@ public String toExplainString() { return EXPLAIN_NAME; } + @Override + protected Logger logger() { + return LOGGER; + } + @Override protected TransferableBlock getNextBlock() { TransferableBlock block = _multiConsumer.readBlockBlocking(); @@ -50,6 +58,9 @@ protected TransferableBlock getNextBlock() { while (_isEarlyTerminated && !block.isEndOfStreamBlock()) { block = _multiConsumer.readBlockBlocking(); } + if (block.isSuccessfulEndOfStreamBlock()) { + updateEosBlock(block, _statMap); + } return block; } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxSendOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxSendOperator.java index 5d66d6afac41..fce214e7aabf 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxSendOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxSendOperator.java @@ -19,6 +19,7 @@ package org.apache.pinot.query.runtime.operator; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; import com.google.common.base.Preconditions; import java.util.Collections; import java.util.EnumSet; @@ -28,6 +29,7 @@ import javax.annotation.Nullable; import org.apache.calcite.rel.RelDistribution; import org.apache.calcite.rel.RelFieldCollation; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.query.mailbox.MailboxService; import org.apache.pinot.query.mailbox.SendingMailbox; import org.apache.pinot.query.planner.logical.RexExpression; @@ -37,7 +39,7 @@ import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.operator.exchange.BlockExchange; -import org.apache.pinot.query.runtime.operator.utils.OperatorUtils; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.apache.pinot.spi.exception.QueryCancelledException; import org.slf4j.Logger; @@ -62,29 +64,34 @@ public class MailboxSendOperator extends MultiStageOperator { private final List _collationKeys; private final List _collationDirections; private final boolean _isSortOnSender; + private final StatMap _statMap = new StatMap<>(StatKey.class); public MailboxSendOperator(OpChainExecutionContext context, MultiStageOperator sourceOperator, RelDistribution.Type distributionType, @Nullable List distributionKeys, @Nullable List collationKeys, @Nullable List collationDirections, boolean isSortOnSender, int receiverStageId) { - this(context, sourceOperator, getBlockExchange(context, distributionType, distributionKeys, receiverStageId), + this(context, sourceOperator, + statMap -> getBlockExchange(context, distributionType, distributionKeys, receiverStageId, statMap), collationKeys, collationDirections, isSortOnSender); + _statMap.merge(StatKey.STAGE, context.getStageId()); + _statMap.merge(StatKey.PARALLELISM, 1); } @VisibleForTesting - MailboxSendOperator(OpChainExecutionContext context, MultiStageOperator sourceOperator, BlockExchange exchange, + MailboxSendOperator(OpChainExecutionContext context, MultiStageOperator sourceOperator, + Function, BlockExchange> exchangeFactory, @Nullable List collationKeys, @Nullable List collationDirections, boolean isSortOnSender) { super(context); _sourceOperator = sourceOperator; - _exchange = exchange; + _exchange = exchangeFactory.apply(_statMap); _collationKeys = collationKeys; _collationDirections = collationDirections; _isSortOnSender = isSortOnSender; } private static BlockExchange getBlockExchange(OpChainExecutionContext context, RelDistribution.Type distributionType, - @Nullable List distributionKeys, int receiverStageId) { + @Nullable List distributionKeys, int receiverStageId, StatMap statMap) { Preconditions.checkState(SUPPORTED_EXCHANGE_TYPES.contains(distributionType), "Unsupported distribution type: %s", distributionType); MailboxService mailboxService = context.getMailboxService(); @@ -97,12 +104,29 @@ private static BlockExchange getBlockExchange(OpChainExecutionContext context, R MailboxIdUtils.toRoutingInfos(requestId, context.getStageId(), context.getWorkerId(), receiverStageId, mailboxInfos); List sendingMailboxes = routingInfos.stream() - .map(v -> mailboxService.getSendingMailbox(v.getHostname(), v.getPort(), v.getMailboxId(), deadlineMs)) + .map(v -> mailboxService.getSendingMailbox(v.getHostname(), v.getPort(), v.getMailboxId(), deadlineMs, statMap)) .collect(Collectors.toList()); + statMap.merge(StatKey.FAN_OUT, sendingMailboxes.size()); return BlockExchange.getExchange(sendingMailboxes, distributionType, distributionKeys, TransferableBlockUtils::splitBlock); } + @Override + public void registerExecution(long time, int numRows) { + _statMap.merge(StatKey.EXECUTION_TIME_MS, time); + _statMap.merge(StatKey.EMITTED_ROWS, numRows); + } + + @Override + public Type getOperatorType() { + return Type.MAILBOX_SEND; + } + + @Override + protected Logger logger() { + return LOGGER; + } + @Override public List getChildOperators() { return Collections.singletonList(_sourceOperator); @@ -119,12 +143,9 @@ protected TransferableBlock getNextBlock() { try { TransferableBlock block = _sourceOperator.nextBlock(); if (block.isSuccessfulEndOfStreamBlock()) { - // Stats need to be populated here because the block is being sent to the mailbox - // and the receiving opChain will not be able to access the stats from the previous opChain - TransferableBlock eosBlockWithStats = TransferableBlockUtils.getEndOfStreamTransferableBlock( - OperatorUtils.getMetadataFromOperatorStats(_opChainStats.getOperatorStatsMap())); + updateEosBlock(block, _statMap); // no need to check early terminate signal b/c the current block is already EOS - sendTransferableBlock(eosBlockWithStats); + sendTransferableBlock(block); } else { if (sendTransferableBlock(block)) { earlyTerminate(); @@ -133,7 +154,7 @@ protected TransferableBlock getNextBlock() { return block; } catch (QueryCancelledException e) { LOGGER.debug("Query was cancelled! for opChain: {}", _context.getId()); - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + return createLeafBlock(); } catch (TimeoutException e) { LOGGER.warn("Timed out transferring data on opChain: {}", _context.getId(), e); return TransferableBlockUtils.getErrorTransferableBlock(e); @@ -149,6 +170,11 @@ protected TransferableBlock getNextBlock() { } } + protected TransferableBlock createLeafBlock() { + return TransferableBlockUtils.getEndOfStreamTransferableBlock( + MultiStageQueryStats.createCancelledSend(_context.getStageId(), _statMap)); + } + private boolean sendTransferableBlock(TransferableBlock block) throws Exception { boolean isEarlyTerminated = _exchange.send(block); @@ -158,15 +184,6 @@ private boolean sendTransferableBlock(TransferableBlock block) return isEarlyTerminated; } - /** - * This method is overridden to return true because this operator is last in the chain and needs to collect - * execution time stats - */ - @Override - protected boolean shouldCollectStats() { - return true; - } - @Override public void close() { super.close(); @@ -178,4 +195,87 @@ public void cancel(Throwable t) { super.cancel(t); _exchange.cancel(t); } + + public enum StatKey implements StatMap.Key { + EXECUTION_TIME_MS(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }, + EMITTED_ROWS(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }, + STAGE(StatMap.Type.INT) { + @Override + public int merge(int value1, int value2) { + return StatMap.Key.eqNotZero(value1, value2); + } + + @Override + public boolean includeDefaultInJson() { + return true; + } + }, + /** + * Number of parallelism of the stage this operator is the root of. + *

+ * The CPU times reported by this stage will be proportional to this number. + */ + PARALLELISM(StatMap.Type.INT), + /** + * How many receive mailboxes are being written by this send operator. + */ + FAN_OUT(StatMap.Type.INT) { + @Override + public int merge(int value1, int value2) { + return Math.max(value1, value2); + } + }, + /** + * How many messages have been sent in heap format by this mailbox. + *

+ * The lower the relation between RAW_MESSAGES and IN_MEMORY_MESSAGES, the more efficient the exchange is. + */ + IN_MEMORY_MESSAGES(StatMap.Type.INT), + /** + * How many messages have been sent in raw format and therefore serialized by this mailbox. + *

+ * The higher the relation between RAW_MESSAGES and IN_MEMORY_MESSAGES, the less efficient the exchange is. + */ + RAW_MESSAGES(StatMap.Type.INT), + /** + * How many bytes have been serialized by this mailbox. + *

+ * A high number here indicates that the mailbox is sending a lot of data to other servers. + */ + SERIALIZED_BYTES(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }, + /** + * How long (in CPU time) it took to serialize the raw messages sent by this mailbox. + */ + SERIALIZATION_TIME_MS(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + },; + private final StatMap.Type _type; + + StatKey(StatMap.Type type) { + _type = type; + } + + @Override + public StatMap.Type getType() { + return _type; + } + } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MinusOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MinusOperator.java index 3415bfb3fc4d..0085828e4d01 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MinusOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MinusOperator.java @@ -23,12 +23,15 @@ import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.core.data.table.Record; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Minus/Except operator. */ public class MinusOperator extends SetOperator { + private static final Logger LOGGER = LoggerFactory.getLogger(MinusOperator.class); private static final String EXPLAIN_NAME = "MINUS"; public MinusOperator(OpChainExecutionContext opChainExecutionContext, List upstreamOperators, @@ -36,6 +39,16 @@ public MinusOperator(OpChainExecutionContext opChainExecutionContext, List, AutoCloseable { - private static final Logger LOGGER = LoggerFactory.getLogger(MultiStageOperator.class); +public abstract class MultiStageOperator + implements Operator, AutoCloseable { protected final OpChainExecutionContext _context; protected final String _operatorId; - protected final OpChainStats _opChainStats; protected boolean _isEarlyTerminated; public MultiStageOperator(OpChainExecutionContext context) { _context = context; _operatorId = Joiner.on("_").join(getClass().getSimpleName(), _context.getStageId(), _context.getServer()); - _opChainStats = _context.getStats(); _isEarlyTerminated = false; } + /** + * Returns the logger for the operator. + *

+ * This method is used to generic multi-stage operator messages using the name of the specific operator. + * Implementations should not allocate new loggers for each call but instead reuse some (probably static and final) + * attribute. + */ + protected abstract Logger logger(); + + public abstract Type getOperatorType(); + + public abstract void registerExecution(long time, int numRows); + @Override public TransferableBlock nextBlock() { if (Tracing.ThreadAccountantOps.isInterrupted()) { throw new EarlyTerminationException("Interrupted while processing next block"); } + if (logger().isDebugEnabled()) { + logger().debug("Operator {}: Reading next block", _operatorId); + } try (InvocationScope ignored = Tracing.getTracer().createScope(getClass())) { TransferableBlock nextBlock; - if (shouldCollectStats()) { - OperatorStats operatorStats = _opChainStats.getOperatorStats(_context, _operatorId); - operatorStats.startTimer(); - try { - nextBlock = getNextBlock(); - } catch (Exception e) { - nextBlock = TransferableBlockUtils.getErrorTransferableBlock(e); - } - operatorStats.recordRow(1, nextBlock.getNumRows()); - operatorStats.endTimer(nextBlock); - } else { - try { - nextBlock = getNextBlock(); - } catch (Exception e) { - nextBlock = TransferableBlockUtils.getErrorTransferableBlock(e); - } + Stopwatch executeStopwatch = Stopwatch.createStarted(); + try { + nextBlock = getNextBlock(); + } catch (Exception e) { + nextBlock = TransferableBlockUtils.getErrorTransferableBlock(e); + } + registerExecution(executeStopwatch.elapsed(TimeUnit.MILLISECONDS), nextBlock.getNumRows()); + + if (logger().isDebugEnabled()) { + logger().debug("Operator {}. Block of type {} ready to send", _operatorId, nextBlock.getType()); } return nextBlock; } } - public String getOperatorId() { - return _operatorId; - } - // Make it protected because we should always call nextBlock() protected abstract TransferableBlock getNextBlock() throws Exception; @@ -89,14 +99,24 @@ protected void earlyTerminate() { } } - protected boolean shouldCollectStats() { - return _context.isTraceEnabled(); + /** + * Adds the current operator stats as the last operator in the open stats of the given holder. + * + * It is assumed that: + *

    + *
  1. The current stage of the holder is equal to the stage id of this operator.
  2. + *
  3. The holder already contains the stats of the previous operators of the same stage in inorder
  4. + *
+ */ + protected void addStats(MultiStageQueryStats holder, StatMap statMap) { + Preconditions.checkArgument(holder.getCurrentStageId() == _context.getStageId(), + "The holder's stage id should be the same as the current operator's stage id. Expected %s, got %s", + _context.getStageId(), holder.getCurrentStageId()); + holder.getCurrentStats().addLastOperator(getOperatorType(), statMap); } @Override - public List getChildOperators() { - throw new UnsupportedOperationException(); - } + public abstract List getChildOperators(); // TODO: Ideally close() call should finish within request deadline. // TODO: Consider passing deadline as part of the API. @@ -106,7 +126,7 @@ public void close() { try { op.close(); } catch (Exception e) { - LOGGER.error("Failed to close operator: " + op + " with exception:" + e); + logger().error("Failed to close operator: " + op + " with exception:" + e); // Continue processing because even one operator failed to be close, we should still close the rest. } } @@ -117,9 +137,177 @@ public void cancel(Throwable e) { try { op.cancel(e); } catch (Exception e2) { - LOGGER.error("Failed to cancel operator:" + op + "with error:" + e + " with exception:" + e2); + logger().error("Failed to cancel operator:" + op + "with error:" + e + " with exception:" + e2); // Continue processing because even one operator failed to be cancelled, we should still cancel the rest. } } } + + /** + * Receives the EOS block from upstream operator and updates the stats. + *

+ * The fact that the EOS belongs to the upstream operator is not an actual requirement. Actual requirements are listed + * in {@link #addStats(MultiStageQueryStats, StatMap)} + * @param upstreamEos + * @return + */ + protected TransferableBlock updateEosBlock(TransferableBlock upstreamEos, StatMap statMap) { + assert upstreamEos.isSuccessfulEndOfStreamBlock(); + MultiStageQueryStats queryStats = upstreamEos.getQueryStats(); + assert queryStats != null; + addStats(queryStats, statMap); + return upstreamEos; + } + + /** + * This enum is used to identify the operation type. + *

+ * This is mostly used in the context of stats collection, where we use this enum in the serialization form in order + * to identify the type of the stats in an efficient way. + */ + public enum Type { + AGGREGATE(AggregateOperator.StatKey.class) { + @Override + public void mergeInto(BrokerResponseNativeV2 response, StatMap map) { + @SuppressWarnings("unchecked") + StatMap stats = (StatMap) map; + response.mergeNumGroupsLimitReached(stats.getBoolean(AggregateOperator.StatKey.NUM_GROUPS_LIMIT_REACHED)); + response.mergeMaxRowsInOperator(stats.getLong(AggregateOperator.StatKey.EMITTED_ROWS)); + } + }, + FILTER(FilterOperator.StatKey.class) { + @Override + public void mergeInto(BrokerResponseNativeV2 response, StatMap map) { + @SuppressWarnings("unchecked") + StatMap stats = (StatMap) map; + response.mergeMaxRowsInOperator(stats.getLong(FilterOperator.StatKey.EMITTED_ROWS)); + } + }, + HASH_JOIN(HashJoinOperator.StatKey.class) { + @Override + public void mergeInto(BrokerResponseNativeV2 response, StatMap map) { + @SuppressWarnings("unchecked") + StatMap stats = (StatMap) map; + response.mergeMaxRowsInOperator(stats.getLong(HashJoinOperator.StatKey.EMITTED_ROWS)); + response.mergeMaxRowsInJoinReached(stats.getBoolean(HashJoinOperator.StatKey.MAX_ROWS_IN_JOIN_REACHED)); + } + }, + INTERSECT(SetOperator.StatKey.class) { + @Override + public void mergeInto(BrokerResponseNativeV2 response, StatMap map) { + @SuppressWarnings("unchecked") + StatMap stats = (StatMap) map; + response.mergeMaxRowsInOperator(stats.getLong(SetOperator.StatKey.EMITTED_ROWS)); + } + }, + LEAF(LeafStageTransferableBlockOperator.StatKey.class) { + @Override + public void mergeInto(BrokerResponseNativeV2 response, StatMap map) { + @SuppressWarnings("unchecked") + StatMap stats = + (StatMap) map; + response.mergeMaxRowsInOperator(stats.getLong(LeafStageTransferableBlockOperator.StatKey.EMITTED_ROWS)); + + StatMap brokerStats = new StatMap<>(BrokerResponseNativeV2.StatKey.class); + for (LeafStageTransferableBlockOperator.StatKey statKey : stats.keySet()) { + statKey.updateBrokerMetadata(brokerStats, stats); + } + response.addServerStats(brokerStats); + } + }, + LITERAL(LiteralValueOperator.StatKey.class) { + @Override + public void mergeInto(BrokerResponseNativeV2 response, StatMap map) { + // Do nothing + } + }, + MAILBOX_RECEIVE(BaseMailboxReceiveOperator.StatKey.class) { + @Override + public void mergeInto(BrokerResponseNativeV2 response, StatMap map) { + @SuppressWarnings("unchecked") + StatMap stats = (StatMap) map; + response.mergeMaxRowsInOperator(stats.getLong(BaseMailboxReceiveOperator.StatKey.EMITTED_ROWS)); + } + }, + MAILBOX_SEND(MailboxSendOperator.StatKey.class) { + @Override + public void mergeInto(BrokerResponseNativeV2 response, StatMap map) { + @SuppressWarnings("unchecked") + StatMap stats = (StatMap) map; + response.mergeMaxRowsInOperator(stats.getLong(MailboxSendOperator.StatKey.EMITTED_ROWS)); + } + }, + MINUS(SetOperator.StatKey.class) { + @Override + public void mergeInto(BrokerResponseNativeV2 response, StatMap map) { + @SuppressWarnings("unchecked") + StatMap stats = (StatMap) map; + response.mergeMaxRowsInOperator(stats.getLong(SetOperator.StatKey.EMITTED_ROWS)); + } + }, + PIPELINE_BREAKER(PipelineBreakerOperator.StatKey.class) { + @Override + public void mergeInto(BrokerResponseNativeV2 response, StatMap map) { + @SuppressWarnings("unchecked") + StatMap stats = (StatMap) map; + response.mergeMaxRowsInOperator(stats.getLong(PipelineBreakerOperator.StatKey.EMITTED_ROWS)); + } + }, + SORT_OR_LIMIT(SortOperator.StatKey.class) { + @Override + public void mergeInto(BrokerResponseNativeV2 response, StatMap map) { + @SuppressWarnings("unchecked") + StatMap stats = (StatMap) map; + response.mergeMaxRowsInOperator(stats.getLong(SortOperator.StatKey.EMITTED_ROWS)); + } + }, + TRANSFORM(TransformOperator.StatKey.class) { + @Override + public void mergeInto(BrokerResponseNativeV2 response, StatMap map) { + @SuppressWarnings("unchecked") + StatMap stats = (StatMap) map; + response.mergeMaxRowsInOperator(stats.getLong(TransformOperator.StatKey.EMITTED_ROWS)); + } + }, + UNION(SetOperator.StatKey.class) { + @Override + public void mergeInto(BrokerResponseNativeV2 response, StatMap map) { + @SuppressWarnings("unchecked") + StatMap stats = (StatMap) map; + response.mergeMaxRowsInOperator(stats.getLong(SetOperator.StatKey.EMITTED_ROWS)); + } + }, + WINDOW(WindowAggregateOperator.StatKey.class) { + @Override + public void mergeInto(BrokerResponseNativeV2 response, StatMap map) { + @SuppressWarnings("unchecked") + StatMap stats = (StatMap) map; + response.mergeMaxRowsInOperator(stats.getLong(WindowAggregateOperator.StatKey.EMITTED_ROWS)); + } + },; + + private final Class _statKeyClass; + + Type(Class statKeyClass) { + _statKeyClass = statKeyClass; + } + + /** + * Gets the class of the stat key for this operator type. + *

+ * Notice that this is not including the generic type parameter, because Java generic types are not expressive + * enough indicate what we want to say, so generics here are more problematic than useful. + */ + public Class getStatKeyClass() { + return _statKeyClass; + } + + /** + * Merges the stats from the given map into the given broker response. + *

+ * Each literal has its own implementation of this method, which assumes the given map is of the correct type + * (compatible with {@link #getStatKeyClass()}). This is a way to avoid casting in the caller. + */ + public abstract void mergeInto(BrokerResponseNativeV2 response, StatMap map); + } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OpChain.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OpChain.java index 360bef6324e0..5d989f169f24 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OpChain.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OpChain.java @@ -34,7 +34,6 @@ public class OpChain implements AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(OpChain.class); private final OpChainId _id; - private final OpChainStats _stats; private final MultiStageOperator _root; private final Consumer _finishCallback; @@ -45,7 +44,6 @@ public OpChain(OpChainExecutionContext context, MultiStageOperator root) { public OpChain(OpChainExecutionContext context, MultiStageOperator root, Consumer finishCallback) { _id = context.getId(); - _stats = context.getStats(); _root = root; _finishCallback = finishCallback; } @@ -54,11 +52,6 @@ public OpChainId getId() { return _id; } - // TODO: Move OperatorStats here. - public OpChainStats getStats() { - return _stats; - } - public Operator getRoot() { return _root; } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OpChainStats.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OpChainStats.java deleted file mode 100644 index 6c259eaff22c..000000000000 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OpChainStats.java +++ /dev/null @@ -1,109 +0,0 @@ -/** - * 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.pinot.query.runtime.operator; - -import com.google.common.base.Stopwatch; -import com.google.common.base.Suppliers; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Supplier; -import javax.annotation.concurrent.NotThreadSafe; -import org.apache.pinot.common.datatable.DataTable; -import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; -import org.apache.pinot.spi.accounting.ThreadResourceUsageProvider; - - -/** - * {@code OpChainStats} tracks execution statistics for {@link OpChain}s. - */ -@NotThreadSafe -public class OpChainStats { - - // use memoized supplier so that the timing doesn't start until the - // first time we get the timer - private final Supplier _exTimer = - Suppliers.memoize(ThreadResourceUsageProvider::new)::get; - - // this is used to make sure that toString() doesn't have side - // effects (accidentally starting the timer) - private volatile boolean _exTimerStarted = false; - - private final Stopwatch _executeStopwatch = Stopwatch.createUnstarted(); - private final Stopwatch _queuedStopwatch = Stopwatch.createUnstarted(); - private final AtomicLong _queuedCount = new AtomicLong(); - - private final String _id; - private final ConcurrentHashMap _operatorStatsMap = new ConcurrentHashMap<>(); - - public OpChainStats(String id) { - _id = id; - } - - public void executing() { - startExecutionTimer(); - if (_queuedStopwatch.isRunning()) { - _queuedStopwatch.stop(); - } - } - - public void queued() { - _queuedCount.incrementAndGet(); - if (!_queuedStopwatch.isRunning()) { - _queuedStopwatch.start(); - } - if (_executeStopwatch.isRunning()) { - _executeStopwatch.stop(); - } - } - - public ConcurrentHashMap getOperatorStatsMap() { - return _operatorStatsMap; - } - - public OperatorStats getOperatorStats(OpChainExecutionContext context, String operatorId) { - return _operatorStatsMap.computeIfAbsent(operatorId, (id) -> { - OperatorStats operatorStats = new OperatorStats(context); - if (context.isTraceEnabled()) { - operatorStats.recordSingleStat(DataTable.MetadataKey.OPERATOR_ID.getName(), operatorId); - } - return operatorStats; - }); - } - - private void startExecutionTimer() { - _exTimerStarted = true; - _exTimer.get(); - if (!_executeStopwatch.isRunning()) { - _executeStopwatch.start(); - } - } - - public long getExecutionTime() { - return _executeStopwatch.elapsed(TimeUnit.MILLISECONDS); - } - - @Override - public String toString() { - return String.format("(%s) Queued Count: %s, Executing Time: %sms, Queued Time: %sms", _id, _queuedCount.get(), - _exTimerStarted ? _executeStopwatch.elapsed(TimeUnit.MILLISECONDS) : 0, - _queuedStopwatch.elapsed(TimeUnit.MILLISECONDS)); - } -} diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OperatorStats.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OperatorStats.java deleted file mode 100644 index 32fa9f140f30..000000000000 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OperatorStats.java +++ /dev/null @@ -1,119 +0,0 @@ -/** - * 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.pinot.query.runtime.operator; - -import com.google.common.base.Stopwatch; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.TimeUnit; -import org.apache.pinot.common.datatable.DataTable; -import org.apache.pinot.query.routing.VirtualServerAddress; -import org.apache.pinot.query.runtime.blocks.TransferableBlock; -import org.apache.pinot.query.runtime.operator.utils.OperatorUtils; -import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; - - -public class OperatorStats { - private final Stopwatch _executeStopwatch = Stopwatch.createUnstarted(); - - // TODO: add a operatorId for better tracking purpose. - private final int _stageId; - private final long _requestId; - - private final VirtualServerAddress _serverAddress; - - private int _numBlock = 0; - private int _numRows = 0; - private long _startTimeMs = -1; - private long _endTimeMs = -1; - private final Map _executionStats; - - public OperatorStats(OpChainExecutionContext context) { - this(context.getRequestId(), context.getStageId(), context.getServer()); - } - - //TODO: remove this constructor after the context constructor can be used in serialization and deserialization - public OperatorStats(long requestId, int stageId, VirtualServerAddress serverAddress) { - _stageId = stageId; - _requestId = requestId; - _serverAddress = serverAddress; - _executionStats = new HashMap<>(); - } - - public void startTimer() { - _startTimeMs = _startTimeMs == -1 ? System.currentTimeMillis() : _startTimeMs; - if (!_executeStopwatch.isRunning()) { - _executeStopwatch.start(); - } - } - - public void endTimer(TransferableBlock block) { - if (_executeStopwatch.isRunning()) { - _executeStopwatch.stop(); - _endTimeMs = System.currentTimeMillis(); - } - } - - public void recordRow(int numBlock, int numRows) { - _numBlock += numBlock; - _numRows += numRows; - } - - public void recordSingleStat(String key, String stat) { - _executionStats.put(key, stat); - } - - public void recordExecutionStats(Map executionStats) { - _executionStats.putAll(executionStats); - } - - public Map getExecutionStats() { - _executionStats.putIfAbsent(DataTable.MetadataKey.NUM_BLOCKS.getName(), String.valueOf(_numBlock)); - _executionStats.putIfAbsent(DataTable.MetadataKey.NUM_ROWS.getName(), String.valueOf(_numRows)); - _executionStats.putIfAbsent(DataTable.MetadataKey.OPERATOR_EXECUTION_TIME_MS.getName(), - String.valueOf(_executeStopwatch.elapsed(TimeUnit.MILLISECONDS))); - // wall time are recorded slightly longer than actual execution but it is OK. - - if (_startTimeMs != -1) { - _executionStats.putIfAbsent(DataTable.MetadataKey.OPERATOR_EXEC_START_TIME_MS.getName(), - String.valueOf(_startTimeMs)); - long endTimeMs = _endTimeMs == -1 ? System.currentTimeMillis() : _endTimeMs; - _executionStats.putIfAbsent(DataTable.MetadataKey.OPERATOR_EXEC_END_TIME_MS.getName(), - String.valueOf(endTimeMs)); - } - return _executionStats; - } - - public int getStageId() { - return _stageId; - } - - public long getRequestId() { - return _requestId; - } - - public VirtualServerAddress getServerAddress() { - return _serverAddress; - } - - @Override - public String toString() { - return OperatorUtils.operatorStatsToJson(this); - } -} diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/SetOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/SetOperator.java index edf8416f0211..721c81d77f6d 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/SetOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/SetOperator.java @@ -22,13 +22,16 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import javax.annotation.Nullable; import org.apache.pinot.common.datablock.DataBlock; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.core.common.ExplainPlanRows; import org.apache.pinot.core.data.table.Record; import org.apache.pinot.core.operator.ExecutionStatistics; import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.apache.pinot.segment.spi.IndexSegment; @@ -50,8 +53,10 @@ public abstract class SetOperator extends MultiStageOperator { private final DataSchema _dataSchema; private boolean _isRightSetBuilt; - private boolean _isTerminated; - private TransferableBlock _upstreamErrorBlock; + protected TransferableBlock _upstreamErrorBlock; + @Nullable + private MultiStageQueryStats _rightQueryStats = null; + protected final StatMap _statMap = new StatMap<>(StatKey.class); public SetOperator(OpChainExecutionContext opChainExecutionContext, List upstreamOperators, DataSchema dataSchema) { @@ -62,7 +67,12 @@ public SetOperator(OpChainExecutionContext opChainExecutionContext, List(); _isRightSetBuilt = false; - _isTerminated = false; + } + + @Override + public void registerExecution(long time, int numRows) { + _statMap.merge(StatKey.EXECUTION_TIME_MS, time); + _statMap.merge(StatKey.EMITTED_ROWS, numRows); } @Override @@ -92,9 +102,6 @@ public ExecutionStatistics getExecutionStatistics() { @Override protected TransferableBlock getNextBlock() { - if (_isTerminated) { - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); - } if (!_isRightSetBuilt) { // construct a SET with all the right side rows. constructRightBlockSet(); @@ -121,6 +128,8 @@ protected void constructRightBlockSet() { _upstreamErrorBlock = block; } else { _isRightSetBuilt = true; + _rightQueryStats = block.getQueryStats(); + assert _rightQueryStats != null; } } @@ -133,7 +142,12 @@ protected TransferableBlock constructResultBlockSet(TransferableBlock leftBlock) return _upstreamErrorBlock; } if (leftBlock.isSuccessfulEndOfStreamBlock()) { - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + assert _rightQueryStats != null; + MultiStageQueryStats leftQueryStats = leftBlock.getQueryStats(); + assert leftQueryStats != null; + _rightQueryStats.mergeInOrder(leftQueryStats, getOperatorType(), _statMap); + _rightQueryStats.getCurrentStats().concat(leftQueryStats.getCurrentStats()); + return TransferableBlockUtils.getEndOfStreamTransferableBlock(_rightQueryStats); } for (Object[] row : leftBlock.getContainer()) { if (handleRowMatched(row)) { @@ -150,4 +164,29 @@ protected TransferableBlock constructResultBlockSet(TransferableBlock leftBlock) * @return true if the row is matched. */ protected abstract boolean handleRowMatched(Object[] row); + + public enum StatKey implements StatMap.Key { + EXECUTION_TIME_MS(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }, + EMITTED_ROWS(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }; + private final StatMap.Type _type; + + StatKey(StatMap.Type type) { + _type = type; + } + + @Override + public StatMap.Type getType() { + return _type; + } + } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/SortOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/SortOperator.java index b0a1923c808b..0c7ba6cf825d 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/SortOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/SortOperator.java @@ -19,6 +19,7 @@ package org.apache.pinot.query.runtime.operator; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; import java.util.ArrayList; import java.util.Arrays; @@ -27,11 +28,11 @@ import javax.annotation.Nullable; import org.apache.calcite.rel.RelFieldCollation; import org.apache.pinot.common.datablock.DataBlock; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.core.query.selection.SelectionOperatorUtils; import org.apache.pinot.query.planner.logical.RexExpression; import org.apache.pinot.query.runtime.blocks.TransferableBlock; -import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.operator.utils.SortUtils; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.apache.pinot.spi.utils.CommonConstants; @@ -50,8 +51,11 @@ public class SortOperator extends MultiStageOperator { private final PriorityQueue _priorityQueue; private final ArrayList _rows; private final int _numRowsToKeep; + private final StatMap _statMap = new StatMap<>(StatKey.class); private boolean _hasConstructedSortedBlock; + @Nullable + private TransferableBlock _eosBlock = null; public SortOperator(OpChainExecutionContext context, MultiStageOperator upstreamOperator, List collationKeys, List collationDirections, @@ -63,8 +67,8 @@ public SortOperator(OpChainExecutionContext context, MultiStageOperator upstream } @VisibleForTesting - SortOperator(OpChainExecutionContext context, MultiStageOperator upstreamOperator, List collationKeys, - List collationDirections, + SortOperator(OpChainExecutionContext context, MultiStageOperator upstreamOperator, + List collationKeys, List collationDirections, List collationNullDirections, int fetch, int offset, DataSchema dataSchema, boolean isInputSorted, int defaultHolderCapacity, int defaultResponseLimit) { super(context); @@ -91,6 +95,22 @@ public SortOperator(OpChainExecutionContext context, MultiStageOperator upstream } } + @Override + public void registerExecution(long time, int numRows) { + _statMap.merge(StatKey.EXECUTION_TIME_MS, time); + _statMap.merge(StatKey.EMITTED_ROWS, numRows); + } + + @Override + public Type getOperatorType() { + return Type.SORT_OR_LIMIT; + } + + @Override + protected Logger logger() { + return LOGGER; + } + @Override public List getChildOperators() { return ImmutableList.of(_upstreamOperator); @@ -109,13 +129,16 @@ public String toExplainString() { @Override protected TransferableBlock getNextBlock() { if (_hasConstructedSortedBlock) { - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + assert _eosBlock != null; + return _eosBlock; } TransferableBlock finalBlock = consumeInputBlocks(); // returning upstream error block if finalBlock contains error. if (finalBlock.isErrorBlock()) { return finalBlock; } + _statMap.merge(StatKey.REQUIRE_SORT, _priorityQueue != null); + _eosBlock = updateEosBlock(finalBlock, _statMap); return produceSortedBlock(); } @@ -126,12 +149,12 @@ private TransferableBlock produceSortedBlock() { List row = _rows.subList(_offset, _rows.size()); return new TransferableBlock(row, _dataSchema, DataBlock.Type.ROW); } else { - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + return _eosBlock; } } else { int resultSize = _priorityQueue.size() - _offset; if (resultSize <= 0) { - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + return _eosBlock; } Object[][] rowsArr = new Object[resultSize][]; for (int i = resultSize - 1; i >= 0; i--) { @@ -154,8 +177,13 @@ private TransferableBlock consumeInputBlocks() { _rows.addAll(container); } else { _rows.addAll(container.subList(0, _numRowsToKeep - numRows)); - LOGGER.debug("Early terminate at SortOperator - operatorId={}, opChainId={}", _operatorId, - _context.getId()); + if (LOGGER.isDebugEnabled()) { + // this operatorId is an old name. It is being kept to avoid breaking changes on the log message. + String operatorId = Joiner.on("_") + .join(getClass().getSimpleName(), _context.getStageId(), _context.getServer()); + LOGGER.debug("Early terminate at SortOperator - operatorId={}, opChainId={}", operatorId, + _context.getId()); + } // setting operator to be early terminated and awaits EOS block next. earlyTerminate(); } @@ -169,4 +197,30 @@ private TransferableBlock consumeInputBlocks() { } return block; } + + public enum StatKey implements StatMap.Key { + EXECUTION_TIME_MS(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }, + EMITTED_ROWS(StatMap.Type.LONG), + REQUIRE_SORT(StatMap.Type.BOOLEAN) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }; + private final StatMap.Type _type; + + StatKey(StatMap.Type type) { + _type = type; + } + + @Override + public StatMap.Type getType() { + return _type; + } + } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/SortedMailboxReceiveOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/SortedMailboxReceiveOperator.java index 8949ad569a40..76eb8dcc9259 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/SortedMailboxReceiveOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/SortedMailboxReceiveOperator.java @@ -33,6 +33,8 @@ import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.operator.utils.SortUtils; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -43,6 +45,8 @@ * resorting via the PriorityQueue. */ public class SortedMailboxReceiveOperator extends BaseMailboxReceiveOperator { + private static final Logger LOGGER = LoggerFactory.getLogger(SortedMailboxReceiveOperator.class); + private static final String EXPLAIN_NAME = "SORTED_MAILBOX_RECEIVE"; private final DataSchema _dataSchema; @@ -66,6 +70,11 @@ public SortedMailboxReceiveOperator(OpChainExecutionContext context, RelDistribu _isSortOnSender = isSortOnSender; } + @Override + protected Logger logger() { + return LOGGER; + } + @Nullable @Override public String toExplainString() { @@ -86,8 +95,10 @@ protected TransferableBlock getNextBlock() { return block; } else { assert block.isSuccessfulEndOfStreamBlock(); + // the multiConsumer has already merged stages from upstream, but doesn't know about this operator + // specific stats. + _eosBlock = updateEosBlock(block, _statMap); if (!_rows.isEmpty()) { - _eosBlock = block; // TODO: This might not be efficient because we are sorting all the received rows. We should use a k-way merge // when sender side is sorted. _rows.sort( diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/TransformOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/TransformOperator.java index 234f32fbcdee..143dcb05e226 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/TransformOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/TransformOperator.java @@ -24,12 +24,15 @@ import java.util.List; import javax.annotation.Nullable; import org.apache.pinot.common.datablock.DataBlock; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.query.planner.logical.RexExpression; import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.operator.operands.TransformOperand; import org.apache.pinot.query.runtime.operator.operands.TransformOperandFactory; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -43,6 +46,7 @@ * and canonicalized function name matching (lower case). */ public class TransformOperator extends MultiStageOperator { + private static final Logger LOGGER = LoggerFactory.getLogger(TransformOperator.class); private static final String EXPLAIN_NAME = "TRANSFORM"; private final MultiStageOperator _upstreamOperator; @@ -50,6 +54,7 @@ public class TransformOperator extends MultiStageOperator { private final int _resultColumnSize; // TODO: Check type matching between resultSchema and the actual result. private final DataSchema _resultSchema; + private final StatMap _statMap = new StatMap<>(StatKey.class); public TransformOperator(OpChainExecutionContext context, MultiStageOperator upstreamOperator, DataSchema resultSchema, List transforms, DataSchema upstreamDataSchema) { @@ -66,11 +71,27 @@ public TransformOperator(OpChainExecutionContext context, MultiStageOperator ups _resultSchema = resultSchema; } + @Override + public void registerExecution(long time, int numRows) { + _statMap.merge(StatKey.EXECUTION_TIME_MS, time); + _statMap.merge(StatKey.EMITTED_ROWS, numRows); + } + + @Override + protected Logger logger() { + return LOGGER; + } + @Override public List getChildOperators() { return ImmutableList.of(_upstreamOperator); } + @Override + public Type getOperatorType() { + return Type.TRANSFORM; + } + @Nullable @Override public String toExplainString() { @@ -81,7 +102,11 @@ public String toExplainString() { protected TransferableBlock getNextBlock() { TransferableBlock block = _upstreamOperator.nextBlock(); if (block.isEndOfStreamBlock()) { - return block; + if (block.isSuccessfulEndOfStreamBlock()) { + return updateEosBlock(block, _statMap); + } else { + return block; + } } List container = block.getContainer(); List resultRows = new ArrayList<>(container.size()); @@ -94,4 +119,24 @@ protected TransferableBlock getNextBlock() { } return new TransferableBlock(resultRows, _resultSchema, DataBlock.Type.ROW); } + + public enum StatKey implements StatMap.Key { + EXECUTION_TIME_MS(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }, + EMITTED_ROWS(StatMap.Type.LONG); + private final StatMap.Type _type; + + StatKey(StatMap.Type type) { + _type = type; + } + + @Override + public StatMap.Type getType() { + return _type; + } + } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/UnionOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/UnionOperator.java index 4f69b575b187..0b7106780edb 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/UnionOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/UnionOperator.java @@ -18,25 +18,43 @@ */ package org.apache.pinot.query.runtime.operator; +import com.google.common.base.Preconditions; import java.util.List; import javax.annotation.Nullable; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Union operator for UNION ALL queries. */ public class UnionOperator extends SetOperator { + private static final Logger LOGGER = LoggerFactory.getLogger(UnionOperator.class); private static final String EXPLAIN_NAME = "UNION"; + @Nullable + private MultiStageQueryStats _queryStats = null; + private int _finishedChildren = 0; public UnionOperator(OpChainExecutionContext opChainExecutionContext, List upstreamOperators, DataSchema dataSchema) { super(opChainExecutionContext, upstreamOperators, dataSchema); } + @Override + protected Logger logger() { + return LOGGER; + } + + @Override + public Type getOperatorType() { + return Type.UNION; + } + @Nullable @Override public String toExplainString() { @@ -45,13 +63,41 @@ public String toExplainString() { @Override protected TransferableBlock getNextBlock() { - for (MultiStageOperator upstreamOperator : getChildOperators()) { + if (_upstreamErrorBlock != null) { + return _upstreamErrorBlock; + } + List childOperators = getChildOperators(); + for (int i = _finishedChildren; i < childOperators.size(); i++) { + MultiStageOperator upstreamOperator = childOperators.get(i); TransferableBlock block = upstreamOperator.nextBlock(); - if (!block.isEndOfStreamBlock()) { + if (block.isDataBlock()) { + return block; + } else if (block.isSuccessfulEndOfStreamBlock()) { + _finishedChildren++; + consumeEos(block); + } else { + assert block.isErrorBlock(); + _upstreamErrorBlock = block; return block; } } - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + assert _queryStats != null : "Should have at least one EOS block from the upstream operators"; + addStats(_queryStats, _statMap); + return TransferableBlockUtils.getEndOfStreamTransferableBlock(_queryStats); + } + + private void consumeEos(TransferableBlock block) { + MultiStageQueryStats queryStats = block.getQueryStats(); + assert queryStats != null; + if (_queryStats == null) { + Preconditions.checkArgument(queryStats.getCurrentStageId() == _context.getStageId(), + "The current stage id of the stats holder: %s does not match the current stage id: %s", + queryStats.getCurrentStageId(), _context.getStageId()); + _queryStats = queryStats; + } else { + _queryStats.mergeUpstream(queryStats); + _queryStats.getCurrentStats().concat(queryStats.getCurrentStats()); + } } @Override diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java index d2e37598a0f1..c7976076603d 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java @@ -35,6 +35,7 @@ import org.apache.calcite.rel.RelFieldCollation; import org.apache.commons.collections.CollectionUtils; import org.apache.pinot.common.datablock.DataBlock; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; import org.apache.pinot.core.data.table.Key; @@ -101,6 +102,9 @@ public class WindowAggregateOperator extends MultiStageOperator { private int _numRows; private boolean _hasReturnedWindowAggregateBlock; + @Nullable + private TransferableBlock _eosBlock = null; + private final StatMap _statMap = new StatMap<>(StatKey.class); public WindowAggregateOperator(OpChainExecutionContext context, MultiStageOperator inputOperator, List groupSet, List orderSet, List orderSetDirection, @@ -151,11 +155,27 @@ public WindowAggregateOperator(OpChainExecutionContext context, MultiStageOperat _hasReturnedWindowAggregateBlock = false; } + @Override + public void registerExecution(long time, int numRows) { + _statMap.merge(StatKey.EXECUTION_TIME_MS, time); + _statMap.merge(StatKey.EMITTED_ROWS, numRows); + } + + @Override + protected Logger logger() { + return LOGGER; + } + @Override public List getChildOperators() { return ImmutableList.of(_inputOperator); } + @Override + public Type getOperatorType() { + return Type.WINDOW; + } + @Nullable @Override public String toExplainString() { @@ -165,12 +185,13 @@ public String toExplainString() { @Override protected TransferableBlock getNextBlock() { if (_hasReturnedWindowAggregateBlock) { - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + return _eosBlock; } TransferableBlock finalBlock = consumeInputBlocks(); if (finalBlock.isErrorBlock()) { return finalBlock; } + _eosBlock = updateEosBlock(finalBlock, _statMap); return produceWindowAggregatedBlock(); } @@ -270,8 +291,8 @@ private TransferableBlock produceWindowAggregatedBlock() { } } _hasReturnedWindowAggregateBlock = true; - if (rows.size() == 0) { - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + if (rows.isEmpty()) { + return _eosBlock; } else { return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW); } @@ -575,4 +596,29 @@ public long getCountOfDuplicateOrderByKeys() { } } } + + public enum StatKey implements StatMap.Key { + EXECUTION_TIME_MS(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }, + EMITTED_ROWS(StatMap.Type.LONG) { + @Override + public boolean includeDefaultInJson() { + return true; + } + }; + private final StatMap.Type _type; + + StatKey(StatMap.Type type) { + _type = type; + } + + @Override + public StatMap.Type getType() { + return _type; + } + } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/exchange/BlockExchange.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/exchange/BlockExchange.java index f8d49b632848..3c470b15152d 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/exchange/BlockExchange.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/exchange/BlockExchange.java @@ -89,6 +89,7 @@ public boolean send(TransferableBlock block) // Send metadata to only one randomly picked mailbox, and empty EOS block to other mailboxes int numMailboxes = _sendingMailboxes.size(); int mailboxIdToSendMetadata = ThreadLocalRandom.current().nextInt(numMailboxes); + assert block.getQueryStats() != null; for (int i = 0; i < numMailboxes; i++) { SendingMailbox sendingMailbox = _sendingMailboxes.get(i); TransferableBlock blockToSend = diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/BlockingMultiStreamConsumer.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/BlockingMultiStreamConsumer.java index 387a44e75462..145028fc7458 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/BlockingMultiStreamConsumer.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/BlockingMultiStreamConsumer.java @@ -25,6 +25,8 @@ import org.apache.pinot.common.exception.QueryException; import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; +import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,6 +58,12 @@ public BlockingMultiStreamConsumer(Object id, long deadlineMs, List { - public OfTransferableBlock(Object id, long deadlineMs, + + private final MultiStageQueryStats _stats; + + public OfTransferableBlock(OpChainExecutionContext context, List> asyncProducers) { - super(id, deadlineMs, asyncProducers); + super(context.getId(), context.getDeadlineMs(), asyncProducers); + _stats = MultiStageQueryStats.emptyStats(context.getStageId()); } @Override @@ -240,6 +253,15 @@ protected boolean isEos(TransferableBlock element) { return element.isSuccessfulEndOfStreamBlock(); } + @Override + protected void onConsumerFinish(TransferableBlock element) { + if (element.getQueryStats() != null) { + _stats.mergeUpstream(element.getQueryStats()); + } else { + _stats.mergeUpstream(element.getSerializedStatsByStage()); + } + } + @Override protected TransferableBlock onTimeout() { return TransferableBlockUtils.getErrorTransferableBlock(QueryException.EXECUTION_TIMEOUT_ERROR); @@ -252,7 +274,7 @@ protected TransferableBlock onException(Exception e) { @Override protected TransferableBlock onEos() { - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + return TransferableBlockUtils.getEndOfStreamTransferableBlock(_stats); } } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/OperatorUtils.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/OperatorUtils.java index 7998185f8a72..a10cde39dc0d 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/OperatorUtils.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/OperatorUtils.java @@ -18,23 +18,12 @@ */ package org.apache.pinot.query.runtime.operator.utils; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.JsonNode; import java.util.HashMap; import java.util.Map; import org.apache.commons.lang.StringUtils; -import org.apache.pinot.common.datablock.MetadataBlock; -import org.apache.pinot.common.datatable.DataTable; -import org.apache.pinot.query.planner.physical.DispatchablePlanFragment; -import org.apache.pinot.query.routing.VirtualServerAddress; -import org.apache.pinot.query.runtime.operator.OperatorStats; -import org.apache.pinot.spi.utils.JsonUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class OperatorUtils { - private static final Logger LOGGER = LoggerFactory.getLogger(OperatorUtils.class); private static final Map OPERATOR_TOKEN_MAPPING = new HashMap<>(); static { @@ -66,70 +55,4 @@ public static String canonicalizeFunctionName(String functionName) { functionName = OPERATOR_TOKEN_MAPPING.getOrDefault(functionName, functionName); return functionName; } - - public static void recordTableName(OperatorStats operatorStats, DispatchablePlanFragment dispatchablePlanFragment) { - String tableName = dispatchablePlanFragment.getTableName(); - if (tableName != null) { - operatorStats.recordSingleStat(DataTable.MetadataKey.TABLE.getName(), tableName); - } - } - - public static String operatorStatsToJson(OperatorStats operatorStats) { - try { - Map jsonOut = new HashMap<>(); - jsonOut.put("requestId", operatorStats.getRequestId()); - jsonOut.put("stageId", operatorStats.getStageId()); - jsonOut.put("serverAddress", operatorStats.getServerAddress().toString()); - jsonOut.put("executionStats", operatorStats.getExecutionStats()); - return JsonUtils.objectToString(jsonOut); - } catch (Exception e) { - LOGGER.warn("Error occurred while serializing operatorStats: {}", operatorStats, e); - } - return null; - } - - public static OperatorStats operatorStatsFromJson(String json) { - try { - JsonNode operatorStatsNode = JsonUtils.stringToJsonNode(json); - long requestId = operatorStatsNode.get("requestId").asLong(); - int stageId = operatorStatsNode.get("stageId").asInt(); - String serverAddressStr = operatorStatsNode.get("serverAddress").asText(); - VirtualServerAddress serverAddress = VirtualServerAddress.parse(serverAddressStr); - - OperatorStats operatorStats = - new OperatorStats(requestId, stageId, serverAddress); - operatorStats.recordExecutionStats( - JsonUtils.jsonNodeToObject(operatorStatsNode.get("executionStats"), new TypeReference>() { - })); - - return operatorStats; - } catch (Exception e) { - LOGGER.warn("Error occurred while deserializing operatorStats: {}", json, e); - } - return null; - } - - public static Map getOperatorStatsFromMetadata(MetadataBlock metadataBlock) { - Map operatorStatsMap = new HashMap<>(); - for (Map.Entry entry : metadataBlock.getStats().entrySet()) { - try { - operatorStatsMap.put(entry.getKey(), operatorStatsFromJson(entry.getValue())); - } catch (Exception e) { - LOGGER.warn("Error occurred while fetching operator stats from metadata", e); - } - } - return operatorStatsMap; - } - - public static Map getMetadataFromOperatorStats(Map operatorStatsMap) { - Map metadataStats = new HashMap<>(); - for (Map.Entry entry : operatorStatsMap.entrySet()) { - try { - metadataStats.put(entry.getKey(), operatorStatsToJson(entry.getValue())); - } catch (Exception e) { - LOGGER.warn("Error occurred while fetching metadata from operator stats", e); - } - } - return metadataStats; - } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageQueryStats.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageQueryStats.java new file mode 100644 index 000000000000..5a5af3706043 --- /dev/null +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageQueryStats.java @@ -0,0 +1,651 @@ +/** + * 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.pinot.query.runtime.plan; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.google.common.base.Preconditions; +import java.io.DataInput; +import java.io.DataInputStream; +import java.io.DataOutput; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Function; +import javax.annotation.Nullable; +import org.apache.avro.util.ByteBufferInputStream; +import org.apache.commons.io.output.UnsynchronizedByteArrayOutputStream; +import org.apache.pinot.common.datatable.StatMap; +import org.apache.pinot.query.runtime.operator.BaseMailboxReceiveOperator; +import org.apache.pinot.query.runtime.operator.LeafStageTransferableBlockOperator; +import org.apache.pinot.query.runtime.operator.LiteralValueOperator; +import org.apache.pinot.query.runtime.operator.MailboxSendOperator; +import org.apache.pinot.query.runtime.operator.MultiStageOperator; +import org.apache.pinot.spi.utils.JsonUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * The stats of a given query. + *

+ * For the same query, multiple instances of this class may exist. Each of them will have a partial view of the stats. + * Specifically, while the query is being executed, each operator will return its own partial view of the stats when + * EOS block is sent. + *

+ * Simple operations with a single upstream, like filters or transforms, would just add their own information to the + * stats. More complex operations, like joins or receiving mailboxes, will merge the stats from all their upstreams and + * add their own stats. + *

+ * The complete stats for the query are obtained in the execution root (usually the broker) by merging the partial + * views. + *

+ * In order to reduce allocation, this class is mutable. Some operators may create their own stats, but most of them + * will receive a stats object from the upstream operator and modify it by adding their own stats and sometimes merging + * them with other upstream stats. + */ +public class MultiStageQueryStats { + private static final Logger LOGGER = LoggerFactory.getLogger(MultiStageQueryStats.class); + private final int _currentStageId; + private final StageStats.Open _currentStats; + /** + * Known stats for stages whose id is higher than the current one. + *

+ * A stage may not know all the stats whose id is higher than the current one, so this list may contain null values. + * It may also grow in size when different merge methods are called. + *

+ * For example the stats of the left hand side of a join may know stats of stages 3 and 4 and the right side may know + * stats of stages 5. When merging the stats of the join, the stats of stages 5 will be added to this list. + * + * @see #mergeUpstream(List) + * @see #mergeUpstream(MultiStageQueryStats) + * @see #mergeInOrder(MultiStageQueryStats, MultiStageOperator.Type, StatMap) + */ + private final ArrayList _closedStats; + private static final MultiStageOperator.Type[] ALL_TYPES = MultiStageOperator.Type.values(); + + private MultiStageQueryStats(int stageId) { + _currentStageId = stageId; + _currentStats = new StageStats.Open(); + _closedStats = new ArrayList<>(); + } + + private static MultiStageQueryStats create(int stageId, MultiStageOperator.Type type, @Nullable StatMap opStats) { + MultiStageQueryStats multiStageQueryStats = new MultiStageQueryStats(stageId); + multiStageQueryStats.getCurrentStats().addLastOperator(type, opStats); + return multiStageQueryStats; + } + + public static MultiStageQueryStats emptyStats(int stageId) { + return new MultiStageQueryStats(stageId); + } + + public static MultiStageQueryStats createLeaf(int stageId, + StatMap opStats) { + return create(stageId, MultiStageOperator.Type.LEAF, opStats); + } + + public static MultiStageQueryStats createLiteral(int stageId, StatMap statMap) { + return create(stageId, MultiStageOperator.Type.LITERAL, statMap); + } + + public static MultiStageQueryStats createCancelledSend(int stageId, + StatMap statMap) { + return create(stageId, MultiStageOperator.Type.MAILBOX_SEND, statMap); + } + + public static MultiStageQueryStats createReceive(int stageId, StatMap stats) { + return create(stageId, MultiStageOperator.Type.MAILBOX_RECEIVE, stats); + } + + public int getCurrentStageId() { + return _currentStageId; + } + + /** + * Serialize the current stats in a way it is compatible with {@link #mergeUpstream(List)}. + *

+ * The serialized stats are returned in a list where the index is the stage id. Stages downstream or not related to + * the current one will be null. + */ + public List serialize() + throws IOException { + + ArrayList serializedStats = new ArrayList<>(getMaxStageId()); + for (int i = 0; i < _currentStageId; i++) { + serializedStats.add(null); + } + + try (UnsynchronizedByteArrayOutputStream baos = new UnsynchronizedByteArrayOutputStream.Builder().get(); + DataOutputStream output = new DataOutputStream(baos)) { + + _currentStats.serialize(output); + ByteBuffer currentBuf = ByteBuffer.wrap(baos.toByteArray()); + + serializedStats.add(currentBuf); + + for (StageStats.Closed closedStats : _closedStats) { + if (closedStats == null) { + serializedStats.add(null); + continue; + } + baos.reset(); + closedStats.serialize(output); + ByteBuffer buf = ByteBuffer.wrap(baos.toByteArray()); + serializedStats.add(buf); + } + } + Preconditions.checkState(serializedStats.size() == getMaxStageId() + 1, + "Serialized stats size is different from expected size. Expected %s, got %s", + getMaxStageId() + 1, serializedStats.size()); + return serializedStats; + } + + public StageStats.Open getCurrentStats() { + return _currentStats; + } + + /** + * Returns the higher stage id known by this object. + */ + public int getMaxStageId() { + return _currentStageId + _closedStats.size(); + } + + /** + * Get the stats of a stage whose id is higher than the current one. + *

+ * This method returns null in case the stage id is unknown by this stage or no stats are stored for it. + */ + @Nullable + public StageStats.Closed getUpstreamStageStats(int stageId) { + if (stageId <= _currentStageId) { + throw new IllegalArgumentException("Stage " + stageId + " cannot be upstream of current stage " + + _currentStageId); + } + + int index = stageId - _currentStageId - 1; + if (index >= _closedStats.size()) { + return null; + } + return _closedStats.get(index); + } + + public void mergeInOrder(MultiStageQueryStats otherStats, MultiStageOperator.Type type, + StatMap statMap) { + Preconditions.checkArgument(_currentStageId == otherStats._currentStageId, + "Cannot merge stats from different stages (%s and %s)", _currentStageId, otherStats._currentStageId); + mergeUpstream(otherStats); + StageStats.Open currentStats = getCurrentStats(); + currentStats.concat(otherStats.getCurrentStats()); + currentStats.addLastOperator(type, statMap); + } + + private void growUpToStage(int stageId) { + _closedStats.ensureCapacity(stageId - _currentStageId); + while (getMaxStageId() < stageId) { + _closedStats.add(null); + } + } + + /** + * Merge upstream stats from another MultiStageQueryStats object into this one. + *

+ * Only the stages whose id is higher than the current one are merged. The reason to do so is that upstream stats + * should be already closed while current stage may need some extra tuning. + *

+ * For example set operations may need to merge the stats from all its upstreams before concatenating stats of the + * current stage. + */ + public void mergeUpstream(MultiStageQueryStats otherStats) { + Preconditions.checkArgument(_currentStageId <= otherStats._currentStageId, + "Cannot merge stats from early stage %s into stats of later stage %s", + otherStats._currentStageId, _currentStageId); + + growUpToStage(otherStats.getMaxStageId()); + + int currentDiff = otherStats._currentStageId - _currentStageId; + if (currentDiff > 0) { + StageStats.Closed close = otherStats._currentStats.close(); + int selfIdx = currentDiff - 1; + StageStats.Closed myStats = _closedStats.get(selfIdx); + if (myStats == null) { + _closedStats.set(selfIdx, close); + } else { + myStats.merge(close); + } + } + + for (int i = 0; i < otherStats._closedStats.size(); i++) { + StageStats.Closed otherStatsForStage = otherStats._closedStats.get(i); + if (otherStatsForStage == null) { + continue; + } + int selfIdx = i + currentDiff; + StageStats.Closed myStats = _closedStats.get(selfIdx); + try { + if (myStats == null) { + _closedStats.set(selfIdx, otherStatsForStage); + assert getUpstreamStageStats(i + otherStats._currentStageId + 1) == otherStatsForStage; + } else { + myStats.merge(otherStatsForStage); + } + } catch (IllegalArgumentException | IllegalStateException ex) { + LOGGER.warn("Error merging stats on stage " + i + ". Ignoring the new stats", ex); + } + } + } + + public void mergeUpstream(List otherStats) { + for (int i = 0; i <= _currentStageId && i < otherStats.size(); i++) { + if (otherStats.get(i) != null) { + throw new IllegalArgumentException("Cannot merge stats from early stage " + i + " into stats of " + + "later stage " + _currentStageId); + } + } + growUpToStage(otherStats.size() - 1); + + for (int i = _currentStageId + 1; i < otherStats.size(); i++) { + ByteBuffer otherBuf = otherStats.get(i); + if (otherBuf != null) { + StageStats.Closed myStats = getUpstreamStageStats(i); + try (InputStream is = new ByteBufferInputStream(Collections.singletonList(otherBuf)); + DataInputStream dis = new DataInputStream(is)) { + if (myStats == null) { + StageStats.Closed deserialized = StageStats.Closed.deserialize(dis); + _closedStats.set(i - _currentStageId - 1, deserialized); + assert getUpstreamStageStats(i) == deserialized; + } else { + myStats.merge(dis); + } + } catch (IOException ex) { + LOGGER.warn("Error deserializing stats on stage " + i + ". Considering the new stats empty", ex); + } catch (IllegalArgumentException | IllegalStateException ex) { + LOGGER.warn("Error merging stats on stage " + i + ". Ignoring the new stats", ex); + } + } + } + } + + public JsonNode asJson() { + ObjectNode node = JsonUtils.newObjectNode(); + node.put("stage", _currentStageId); + node.set("open", _currentStats.asJson()); + + ArrayNode closedStats = JsonUtils.newArrayNode(); + for (StageStats.Closed closed : _closedStats) { + if (closed == null) { + closedStats.addNull(); + } else { + closedStats.add(closed.asJson()); + } + } + node.set("closed", closedStats); + return node; + } + + @Override + public String toString() { + return asJson().toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MultiStageQueryStats that = (MultiStageQueryStats) o; + return _currentStageId == that._currentStageId && Objects.equals(_currentStats, that._currentStats) + && Objects.equals(_closedStats, that._closedStats); + } + + @Override + public int hashCode() { + return Objects.hash(_currentStageId, _currentStats, _closedStats); + } + + /** + * {@code StageStats} tracks execution statistics for a single stage. + *

+ * Instances of this class may not have the complete stat information for the stage. Specifically, while the query + * is being executed, each OpChain will contain its own partial view of the stats. + * The final stats for the stage are obtained in the execution root (usually the broker) by + * {@link Closed#merge(StageStats) merging} the partial views from all OpChains. + */ + public abstract static class StageStats { + + /** + * The types of the operators in the stage. + *

+ * The operator index used here is the index of the operator in the operator tree, in the order of the inorder + * traversal. That means that the first value is the leftmost leaf, and the last value is the root. + *

+ * This list contains no null values. + */ + protected final List _operatorTypes; + /** + * The stats associated with the given operator index. + *

+ * The operator index used here is the index of the operator in the operator tree, in the order of the inorder + * traversal. That means that the first value is the leftmost leaf, and the last value is the root. + *

+ * This list contains no null values. + */ + protected final List> _operatorStats; + + private StageStats() { + this(new ArrayList<>(), new ArrayList<>()); + } + + private StageStats(List operatorTypes, List> operatorStats) { + Preconditions.checkArgument(operatorTypes.size() == operatorStats.size(), + "Operator types and stats must have the same size (%s != %s)", + operatorTypes.size(), operatorStats.size()); + for (int i = 0; i < operatorTypes.size(); i++) { + if (operatorTypes.get(i) == null) { + throw new IllegalArgumentException("Unexpected null operator type at index " + i); + } + } + for (int i = 0; i < operatorStats.size(); i++) { + if (operatorStats.get(i) == null) { + throw new IllegalArgumentException("Unexpected null operator stats of type " + operatorTypes.get(i) + + " at index " + i); + } + } + _operatorTypes = operatorTypes; + _operatorStats = operatorStats; + } + + /** + * Return the stats associated with the given operator index. + *

+ * The operator index used here is the index of the operator in the operator tree, in the order of the inorder + * traversal. + * That means that the first value is the leftmost leaf, and the last value is the root. + *

+ * It is the operator responsibility to store here its own stats and that must be done just before the end of stream + * block is sent. This means that calling this method before the stats is added will throw an index out of bounds. + * + * @param operatorIdx The operator index in inorder traversal of the operator tree. + * @return The value of the stat or null if no stat is registered. + * @throws IndexOutOfBoundsException if there is no stats for the given operator index. + */ + public StatMap getOperatorStats(int operatorIdx) { + return _operatorStats.get(operatorIdx); + } + + public MultiStageOperator.Type getOperatorType(int index) { + return _operatorTypes.get(index); + } + + public MultiStageOperator.Type getLastType() { + return _operatorTypes.get(_operatorTypes.size() - 1); + } + + public StatMap getLastOperatorStats() { + return _operatorStats.get(_operatorStats.size() - 1); + } + + public int getLastOperatorIndex() { + return _operatorStats.size() - 1; + } + + public JsonNode asJson() { + ArrayNode json = JsonUtils.newArrayNode(); + + for (int i = 0; i < _operatorStats.size(); i++) { + ObjectNode statNode = JsonUtils.newObjectNode(); + statNode.put("type", _operatorTypes.get(i).name()); + StatMap stats = _operatorStats.get(i); + if (!stats.isEmpty()) { + statNode.set("stats", stats.asJson()); + } + json.add(statNode); + } + + return json; + } + + /** + * Serialize the stats to the given output. + *

+ * Stats can be then deserialized as {@link Closed} with {@link Closed#deserialize(DataInput)}. + */ + public void serialize(DataOutput output) + throws IOException { + // TODO: we can serialize with short or variable size + output.writeInt(_operatorTypes.size()); + assert MultiStageOperator.Type.values().length < Byte.MAX_VALUE : "Too many operator types. " + + "Need to increase the number of bytes size per operator type"; + for (int i = 0; i < _operatorTypes.size(); i++) { + output.writeByte(_operatorTypes.get(i).ordinal()); + StatMap statMap = _operatorStats.get(i); + statMap.serialize(output); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + StageStats that = (StageStats) o; + return Objects.equals(_operatorTypes, that._operatorTypes) && Objects.equals(_operatorStats, that._operatorStats); + } + + @Override + public int hashCode() { + return Objects.hash(_operatorTypes, _operatorStats); + } + + @Override + public String toString() { + return asJson().toString(); + } + + /** + * Closed stats represent the stats of upstream stages. + *

+ * These stats can be serialized or deserialized and merged with other stats of the same stage, but operators must + * never add entries for itself in upstream stats. + */ + public static class Closed extends StageStats { + public Closed(List operatorTypes, List> operatorStats) { + super(operatorTypes, operatorStats); + } + + /** + * Merges the stats from another StageStats object into this one. + *

+ * This object is modified in place while the other object is not modified. + * Both stats must belong to the same stage. + */ + // We need to deal with unchecked because Java type system is not expressive enough to handle this at static time + // But we know we are merging the same stat types because they were created for the same operation. + // There is also a dynamic check in the StatMap.merge() method. + @SuppressWarnings("unchecked") + public void merge(StageStats other) { + Preconditions.checkState(_operatorTypes.equals(other._operatorTypes), "Cannot merge stats from " + + "different stages. Found types %s and %s", _operatorTypes, other._operatorTypes); + for (int i = 0; i < _operatorStats.size(); i++) { + StatMap otherStats = other._operatorStats.get(i); + StatMap myStats = _operatorStats.get(i); + myStats.merge(otherStats); + } + } + + public void merge(DataInputStream input) + throws IOException { + int numOperators = input.readInt(); + if (numOperators != _operatorTypes.size()) { + try { + Closed deserialized = deserialize(input, numOperators); + throw new RuntimeException("Cannot merge stats from stages with different operators. Expected " + + _operatorTypes + " operators, got " + numOperators + ". Deserialized stats: " + deserialized); + } catch (IOException e) { + throw new IOException("Cannot merge stats from stages with different operators. Expected " + + _operatorTypes + " operators, got " + numOperators, e); + } catch (RuntimeException e) { + throw new RuntimeException("Cannot merge stats from stages with different operators. Expected " + + _operatorTypes + " operators, got " + numOperators, e); + } + } + for (int i = 0; i < numOperators; i++) { + byte ordinal = input.readByte(); + if (ordinal != _operatorTypes.get(i).ordinal()) { + throw new IllegalStateException("Cannot merge stats from stages with different operators. Expected " + + " operator " + _operatorTypes.get(i) + "at index " + i + ", got " + ordinal); + } + _operatorStats.get(i).merge(input); + } + } + + /** + * Same as {@link #merge(StageStats)} but reads the stats from a DataInput, so it should be slightly faster given + * it doesn't need to create new objects. + */ + public static Closed deserialize(DataInput input) + throws IOException { + return deserialize(input, input.readInt()); + } + + public void forEach(BiConsumer> consumer) { + Iterator typeIterator = _operatorTypes.iterator(); + Iterator> statIterator = _operatorStats.iterator(); + while (typeIterator.hasNext()) { + consumer.accept(typeIterator.next(), statIterator.next()); + } + } + } + + public static Closed deserialize(DataInput input, int numOperators) + throws IOException { + List operatorTypes = new ArrayList<>(numOperators); + List> operatorStats = new ArrayList<>(numOperators); + + MultiStageOperator.Type[] allTypes = ALL_TYPES; + try { + for (int i = 0; i < numOperators; i++) { + byte ordinal = input.readByte(); + if (ordinal < 0 || ordinal >= allTypes.length) { + throw new IllegalStateException( + "Invalid operator type ordinal " + ordinal + " at index " + i + ". " + "Deserialized so far: " + + new Closed(operatorTypes, operatorStats)); + } + MultiStageOperator.Type type = allTypes[ordinal]; + operatorTypes.add(type); + + @SuppressWarnings("unchecked") + StatMap opStatMap = StatMap.deserialize(input, type.getStatKeyClass()); + operatorStats.add(opStatMap); + } + return new Closed(operatorTypes, operatorStats); + } catch (IOException e) { + throw new IOException("Error deserializing stats. Deserialized so far: " + + new Closed(operatorTypes.subList(0, operatorStats.size()), operatorStats), e); + } catch (RuntimeException e) { + throw new RuntimeException("Error deserializing stats. Deserialized so far: " + + new Closed(operatorTypes.subList(0, operatorStats.size()), operatorStats), e); + } + } + + /** + * Open stats represent the stats of the current stage. + *

+ * These stats can be modified by the operator that is currently executing. Specifically they can add stats for + * the current operator or merge with other open stats from the same stage. + */ + public static class Open extends StageStats { + private Open() { + super(); + } + + public Open addLastOperator(MultiStageOperator.Type type, StatMap statMap) { + Preconditions.checkArgument(statMap.getKeyClass().equals(type.getStatKeyClass()), + "Expected stats of class %s for type %s but found class %s", + type.getStatKeyClass(), type, statMap.getKeyClass()); + if (!_operatorStats.isEmpty() && _operatorStats.get(_operatorStats.size() - 1) == statMap) { + // This is mostly useful to detect errors in the code. + // In the future we may choose to evaluate it only if asserts are enabled + throw new IllegalArgumentException("Cannot add the same stat map twice."); + } + Preconditions.checkNotNull(type, "Cannot add null operator type"); + Preconditions.checkNotNull(statMap, "Cannot add null stats"); + _operatorTypes.add(type); + _operatorStats.add(statMap); + return this; + } + + /** + * Adds the given stats at the end of this object. + */ + public void concat(StageStats.Open other) { + _operatorTypes.addAll(other._operatorTypes); + _operatorStats.addAll(other._operatorStats); + } + + public Closed close() { + return new Closed(_operatorTypes, _operatorStats); + } + } + } + + public static class Builder { + private final MultiStageQueryStats _stats; + + public Builder(int stageId) { + _stats = new MultiStageQueryStats(stageId); + } + + public Builder customizeOpen(Consumer customizer) { + customizer.accept(_stats._currentStats); + return this; + } + + /** + * Adds a new operator to the stats. + * @param consumer a function that will be called with a new and empty open stats object and returns the closed stat + * to be added. The received object can be freely modified and close. + */ + public Builder addLast(Function consumer) { + StageStats.Open open = new StageStats.Open(); + _stats._closedStats.add(consumer.apply(open)); + return this; + } + + public MultiStageQueryStats build() { + return _stats; + } + } +} diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/OpChainExecutionContext.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/OpChainExecutionContext.java index 51d61e5dbcde..3290478de898 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/OpChainExecutionContext.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/OpChainExecutionContext.java @@ -20,12 +20,12 @@ import java.util.Collections; import java.util.Map; +import javax.annotation.Nullable; import org.apache.pinot.query.mailbox.MailboxService; import org.apache.pinot.query.routing.StageMetadata; import org.apache.pinot.query.routing.VirtualServerAddress; import org.apache.pinot.query.routing.WorkerMetadata; import org.apache.pinot.query.runtime.operator.OpChainId; -import org.apache.pinot.query.runtime.operator.OpChainStats; import org.apache.pinot.query.runtime.plan.pipeline.PipelineBreakerResult; import org.apache.pinot.query.runtime.plan.server.ServerPlanRequestContext; import org.apache.pinot.spi.utils.CommonConstants; @@ -45,7 +45,7 @@ public class OpChainExecutionContext { private final WorkerMetadata _workerMetadata; private final VirtualServerAddress _server; private final OpChainId _id; - private final OpChainStats _stats; + @Nullable private final PipelineBreakerResult _pipelineBreakerResult; private final boolean _traceEnabled; @@ -53,7 +53,7 @@ public class OpChainExecutionContext { public OpChainExecutionContext(MailboxService mailboxService, long requestId, long deadlineMs, Map opChainMetadata, StageMetadata stageMetadata, WorkerMetadata workerMetadata, - PipelineBreakerResult pipelineBreakerResult) { + @Nullable PipelineBreakerResult pipelineBreakerResult) { _mailboxService = mailboxService; _requestId = requestId; _deadlineMs = deadlineMs; @@ -63,11 +63,7 @@ public OpChainExecutionContext(MailboxService mailboxService, long requestId, lo _server = new VirtualServerAddress(mailboxService.getHostname(), mailboxService.getPort(), workerMetadata.getWorkerId()); _id = new OpChainId(requestId, workerMetadata.getWorkerId(), stageMetadata.getStageId()); - _stats = new OpChainStats(_id.toString()); _pipelineBreakerResult = pipelineBreakerResult; - if (pipelineBreakerResult != null && pipelineBreakerResult.getOpChainStats() != null) { - _stats.getOperatorStatsMap().putAll(pipelineBreakerResult.getOpChainStats().getOperatorStatsMap()); - } _traceEnabled = Boolean.parseBoolean(opChainMetadata.get(CommonConstants.Broker.Request.TRACE)); } @@ -111,10 +107,7 @@ public OpChainId getId() { return _id; } - public OpChainStats getStats() { - return _stats; - } - + @Nullable public PipelineBreakerResult getPipelineBreakerResult() { return _pipelineBreakerResult; } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutor.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutor.java index a033df03d737..2e0cc7003de3 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutor.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutor.java @@ -121,7 +121,7 @@ private static PipelineBreakerResult runMailboxReceivePipelineBreaker(OpChainSch long timeoutMs = opChainExecutionContext.getDeadlineMs() - System.currentTimeMillis(); if (latch.await(timeoutMs, TimeUnit.MILLISECONDS)) { return new PipelineBreakerResult(pipelineBreakerContext.getNodeIdMap(), pipelineBreakerOperator.getResultMap(), - pipelineBreakerOperator.getErrorBlock(), pipelineBreakerOpChain.getStats()); + pipelineBreakerOperator.getErrorBlock(), pipelineBreakerOperator.getQueryStats()); } else { throw new TimeoutException( String.format("Timed out waiting for pipeline breaker results after: %dms", timeoutMs)); diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerOperator.java index 9fe25888273e..ee3fd721010f 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerOperator.java @@ -18,6 +18,7 @@ */ package org.apache.pinot.query.runtime.plan.pipeline; +import com.google.common.base.Preconditions; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collections; @@ -26,20 +27,28 @@ import java.util.Map; import java.util.Queue; import javax.annotation.Nullable; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.core.common.Operator; import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.operator.MultiStageOperator; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -class PipelineBreakerOperator extends MultiStageOperator { +public class PipelineBreakerOperator extends MultiStageOperator { + private static final Logger LOGGER = LoggerFactory.getLogger(PipelineBreakerOperator.class); private static final String EXPLAIN_NAME = "PIPELINE_BREAKER"; private final Map> _workerMap; private Map> _resultMap; private TransferableBlock _errorBlock; + @Nullable + private MultiStageQueryStats _queryStats = null; + private final StatMap _statMap = new StatMap<>(StatKey.class); public PipelineBreakerOperator(OpChainExecutionContext context, Map> workerMap) { super(context); @@ -50,6 +59,33 @@ public PipelineBreakerOperator(OpChainExecutionContext context, Map getChildOperators() { + throw new UnsupportedOperationException(); + } + + @Override + public Type getOperatorType() { + return Type.PIPELINE_BREAKER; + } + + public MultiStageQueryStats getQueryStats() { + assert _queryStats != null || _errorBlock != null + : "This method should not be called before blocks have been processed"; + return _queryStats; + } + + @Override + protected Logger logger() { + return LOGGER; + } + public Map> getResultMap() { return _resultMap; } @@ -84,6 +120,7 @@ protected TransferableBlock getNextBlock() { dataBlocks.add(block); block = operator.nextBlock(); } + _queryStats = block.getQueryStats(); } else { _resultMap = new HashMap<>(); for (int workerKey : _workerMap.keySet()) { @@ -101,9 +138,37 @@ protected TransferableBlock getNextBlock() { if (block.isDataBlock()) { _resultMap.get(entry.getKey()).add(block); entries.offer(entry); + } else if (block.isSuccessfulEndOfStreamBlock()) { + MultiStageQueryStats queryStats = block.getQueryStats(); + assert queryStats != null; + if (_queryStats == null) { + Preconditions.checkArgument(queryStats.getCurrentStageId() == _context.getStageId(), + "The current stage id of the stats holder: %s does not match the current stage id: %s", + queryStats.getCurrentStageId(), _context.getStageId()); + _queryStats = queryStats; + } else { + _queryStats.mergeUpstream(queryStats); + } } } } - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + assert _queryStats != null; + addStats(_queryStats, _statMap); + return TransferableBlockUtils.getEndOfStreamTransferableBlock(_queryStats); + } + + public enum StatKey implements StatMap.Key { + EXECUTION_TIME_MS(StatMap.Type.LONG), + EMITTED_ROWS(StatMap.Type.LONG); + private final StatMap.Type _type; + + StatKey(StatMap.Type type) { + _type = type; + } + + @Override + public StatMap.Type getType() { + return _type; + } } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerResult.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerResult.java index 2e2b003e34a6..fef6df2cb1c9 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerResult.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerResult.java @@ -23,7 +23,7 @@ import javax.annotation.Nullable; import org.apache.pinot.query.planner.plannode.PlanNode; import org.apache.pinot.query.runtime.blocks.TransferableBlock; -import org.apache.pinot.query.runtime.operator.OpChainStats; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; /** @@ -33,14 +33,14 @@ public class PipelineBreakerResult { private final Map _nodeIdMap; private final Map> _resultMap; private final TransferableBlock _errorBlock; - private final OpChainStats _opChainStats; + private final MultiStageQueryStats _multiStageQueryStats; public PipelineBreakerResult(Map nodeIdMap, Map> resultMap, - @Nullable TransferableBlock errorBlock, @Nullable OpChainStats opChainStats) { + @Nullable TransferableBlock errorBlock, @Nullable MultiStageQueryStats multiStageQueryStats) { _nodeIdMap = nodeIdMap; _resultMap = resultMap; _errorBlock = errorBlock; - _opChainStats = opChainStats; + _multiStageQueryStats = multiStageQueryStats; } public Map getNodeIdMap() { @@ -57,7 +57,7 @@ public TransferableBlock getErrorBlock() { } @Nullable - public OpChainStats getOpChainStats() { - return _opChainStats; + public MultiStageQueryStats getStageQueryStats() { + return _multiStageQueryStats; } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestContext.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestContext.java index cc9d13196276..ac91b457078b 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestContext.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestContext.java @@ -20,6 +20,7 @@ import java.util.List; import java.util.concurrent.ExecutorService; +import javax.annotation.Nullable; import org.apache.pinot.common.request.PinotQuery; import org.apache.pinot.core.query.executor.QueryExecutor; import org.apache.pinot.core.query.request.ServerQueryRequest; @@ -39,6 +40,7 @@ public class ServerPlanRequestContext { private final StagePlan _stagePlan; private final QueryExecutor _leafQueryExecutor; private final ExecutorService _executorService; + @Nullable private final PipelineBreakerResult _pipelineBreakerResult; private final PinotQuery _pinotQuery; @@ -46,7 +48,7 @@ public class ServerPlanRequestContext { private List _serverQueryRequests; public ServerPlanRequestContext(StagePlan stagePlan, QueryExecutor leafQueryExecutor, - ExecutorService executorService, PipelineBreakerResult pipelineBreakerResult) { + ExecutorService executorService, @Nullable PipelineBreakerResult pipelineBreakerResult) { _stagePlan = stagePlan; _leafQueryExecutor = leafQueryExecutor; _executorService = executorService; @@ -66,6 +68,7 @@ public ExecutorService getExecutorService() { return _executorService; } + @Nullable public PipelineBreakerResult getPipelineBreakerResult() { return _pipelineBreakerResult; } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java index 99a94d0a24e0..47b563b1fb9c 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java @@ -36,15 +36,12 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import javax.annotation.Nullable; import org.apache.calcite.runtime.PairList; -import org.apache.commons.collections.MapUtils; import org.apache.pinot.common.datablock.DataBlock; import org.apache.pinot.common.proto.Worker; import org.apache.pinot.common.response.broker.ResultTable; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; -import org.apache.pinot.core.query.reduce.ExecutionStatsAggregator; import org.apache.pinot.core.util.DataBlockExtractUtils; import org.apache.pinot.core.util.trace.TracedThreadFactory; import org.apache.pinot.query.mailbox.MailboxService; @@ -62,9 +59,7 @@ import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.operator.MailboxReceiveOperator; -import org.apache.pinot.query.runtime.operator.OpChainStats; -import org.apache.pinot.query.runtime.operator.OperatorStats; -import org.apache.pinot.query.runtime.operator.utils.OperatorUtils; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.apache.pinot.spi.trace.RequestContext; import org.apache.pinot.spi.utils.CommonConstants; @@ -89,18 +84,17 @@ public QueryDispatcher(MailboxService mailboxService) { new TracedThreadFactory(Thread.NORM_PRIORITY, false, PINOT_BROKER_QUERY_DISPATCHER_FORMAT)); } - public ResultTable submitAndReduce(RequestContext context, DispatchableSubPlan dispatchableSubPlan, long timeoutMs, - Map queryOptions, @Nullable Map executionStatsAggregator) + public QueryResult submitAndReduce(RequestContext context, DispatchableSubPlan dispatchableSubPlan, long timeoutMs, + Map queryOptions) throws Exception { long requestId = context.getRequestId(); try { submit(requestId, dispatchableSubPlan, timeoutMs, queryOptions); long reduceStartTimeNs = System.nanoTime(); - ResultTable resultTable = - runReducer(requestId, dispatchableSubPlan, timeoutMs, queryOptions, executionStatsAggregator, - _mailboxService); + QueryResult queryResult = + runReducer(requestId, dispatchableSubPlan, timeoutMs, queryOptions, _mailboxService); context.setReduceTimeNanos(System.nanoTime() - reduceStartTimeNs); - return resultTable; + return queryResult; } catch (Throwable e) { // TODO: Consider always cancel when it returns (early terminate) cancel(requestId, dispatchableSubPlan); @@ -252,9 +246,8 @@ private DispatchClient getOrCreateDispatchClient(QueryServerInstance queryServer } @VisibleForTesting - public static ResultTable runReducer(long requestId, DispatchableSubPlan dispatchableSubPlan, long timeoutMs, - Map queryOptions, @Nullable Map statsAggregatorMap, - MailboxService mailboxService) { + public static QueryResult runReducer(long requestId, DispatchableSubPlan dispatchableSubPlan, long timeoutMs, + Map queryOptions, MailboxService mailboxService) { // NOTE: Reduce stage is always stage 0 DispatchablePlanFragment dispatchableStagePlan = dispatchableSubPlan.getQueryStageList().get(0); PlanFragment planFragment = dispatchableStagePlan.getPlanFragment(); @@ -272,29 +265,10 @@ public static ResultTable runReducer(long requestId, DispatchableSubPlan dispatc MailboxReceiveOperator receiveOperator = new MailboxReceiveOperator(opChainExecutionContext, receiveNode.getDistributionType(), receiveNode.getSenderStageId()); - ResultTable resultTable = - getResultTable(receiveOperator, receiveNode.getDataSchema(), dispatchableSubPlan.getQueryResultFields()); - collectStats(dispatchableSubPlan, opChainExecutionContext.getStats(), statsAggregatorMap); - return resultTable; + return getQueryResult(receiveOperator, receiveNode.getDataSchema(), dispatchableSubPlan.getQueryResultFields()); } - private static void collectStats(DispatchableSubPlan dispatchableSubPlan, OpChainStats opChainStats, - @Nullable Map statsAggregatorMap) { - if (MapUtils.isNotEmpty(statsAggregatorMap)) { - for (OperatorStats operatorStats : opChainStats.getOperatorStatsMap().values()) { - ExecutionStatsAggregator rootStatsAggregator = statsAggregatorMap.get(0); - rootStatsAggregator.aggregate(null, operatorStats.getExecutionStats(), new HashMap<>()); - ExecutionStatsAggregator stageStatsAggregator = statsAggregatorMap.get(operatorStats.getStageId()); - if (stageStatsAggregator != null) { - OperatorUtils.recordTableName(operatorStats, - dispatchableSubPlan.getQueryStageList().get(operatorStats.getStageId())); - stageStatsAggregator.aggregate(null, operatorStats.getExecutionStats(), new HashMap<>()); - } - } - } - } - - private static ResultTable getResultTable(MailboxReceiveOperator receiveOperator, DataSchema sourceDataSchema, + private static QueryResult getQueryResult(MailboxReceiveOperator receiveOperator, DataSchema sourceDataSchema, PairList resultFields) { int numColumns = resultFields.size(); String[] columnNames = new String[numColumns]; @@ -308,6 +282,7 @@ private static ResultTable getResultTable(MailboxReceiveOperator receiveOperator ArrayList resultRows = new ArrayList<>(); TransferableBlock block = receiveOperator.nextBlock(); + while (!TransferableBlockUtils.isEndOfStream(block)) { DataBlock dataBlock = block.getDataBlock(); int numRows = dataBlock.getNumberOfRows(); @@ -328,11 +303,16 @@ private static ResultTable getResultTable(MailboxReceiveOperator receiveOperator } block = receiveOperator.nextBlock(); } + MultiStageQueryStats queryStats; if (block.isErrorBlock()) { throw new RuntimeException("Received error query execution result block: " + block.getExceptions()); + } else { + assert block.isSuccessfulEndOfStreamBlock(); + queryStats = block.getQueryStats(); + assert queryStats != null; } - return new ResultTable(resultDataSchema, resultRows); + return new QueryResult(new ResultTable(resultDataSchema, resultRows), queryStats); } public void shutdown() { @@ -341,4 +321,29 @@ public void shutdown() { } _dispatchClientMap.clear(); } + + public static class QueryResult { + private final ResultTable _resultTable; + private final List _queryStats; + + public QueryResult(ResultTable resultTable, MultiStageQueryStats queryStats) { + _resultTable = resultTable; + + Preconditions.checkArgument(queryStats.getCurrentStageId() == 0, + "Expecting query stats for stage 0, got: %s", queryStats.getCurrentStageId()); + _queryStats = new ArrayList<>(queryStats.getMaxStageId()); + _queryStats.add(queryStats.getCurrentStats().close()); + for (int i = 1; i <= queryStats.getMaxStageId(); i++) { + _queryStats.add(queryStats.getUpstreamStageStats(i)); + } + } + + public ResultTable getResultTable() { + return _resultTable; + } + + public List getQueryStats() { + return _queryStats; + } + } } diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/mailbox/MailboxServiceTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/mailbox/MailboxServiceTest.java index 3f20d33956c9..4991448fc172 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/mailbox/MailboxServiceTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/mailbox/MailboxServiceTest.java @@ -22,11 +22,13 @@ import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; import org.apache.pinot.query.planner.physical.MailboxIdUtils; import org.apache.pinot.query.runtime.blocks.TransferableBlock; -import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; +import org.apache.pinot.query.runtime.blocks.TransferableBlockTestUtils; +import org.apache.pinot.query.runtime.operator.MailboxSendOperator; import org.apache.pinot.query.runtime.operator.OperatorTestUtil; import org.apache.pinot.query.testutils.QueryTestUtils; import org.apache.pinot.spi.env.PinotConfiguration; @@ -34,6 +36,7 @@ import org.apache.pinot.util.TestUtils; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeTest; import org.testng.annotations.Test; import static org.testng.Assert.*; @@ -49,6 +52,7 @@ public class MailboxServiceTest { private MailboxService _mailboxService2; private long _requestId = 0; + private StatMap _stats; @BeforeClass public void setUp() { @@ -67,6 +71,11 @@ public void tearDown() { _mailboxService2.shutdown(); } + @BeforeTest + public void setUpStats() { + _stats = new StatMap<>(MailboxSendOperator.StatKey.class); + } + @Test public void testLocalHappyPathSendFirst() throws Exception { @@ -74,11 +83,11 @@ public void testLocalHappyPathSendFirst() // Sends are non-blocking as long as channel capacity is not breached SendingMailbox sendingMailbox = - _mailboxService1.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE); + _mailboxService1.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE, _stats); for (int i = 0; i < ReceivingMailbox.DEFAULT_MAX_PENDING_BLOCKS - 1; i++) { sendingMailbox.send(OperatorTestUtil.block(DATA_SCHEMA, new Object[]{i})); } - sendingMailbox.send(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + sendingMailbox.send(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(SENDER_STAGE_ID)); sendingMailbox.complete(); ReceivingMailbox receivingMailbox = _mailboxService1.getReceivingMailbox(mailboxId); @@ -113,11 +122,11 @@ public void testLocalHappyPathReceiveFirst() // Sends are non-blocking as long as channel capacity is not breached SendingMailbox sendingMailbox = - _mailboxService1.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE); + _mailboxService1.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE, _stats); for (int i = 0; i < ReceivingMailbox.DEFAULT_MAX_PENDING_BLOCKS - 1; i++) { sendingMailbox.send(OperatorTestUtil.block(DATA_SCHEMA, new Object[]{i})); } - sendingMailbox.send(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + sendingMailbox.send(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(SENDER_STAGE_ID)); sendingMailbox.complete(); assertEquals(numCallbacks.get(), ReceivingMailbox.DEFAULT_MAX_PENDING_BLOCKS); @@ -143,7 +152,7 @@ public void testLocalCancelledBySender() throws Exception { String mailboxId = MailboxIdUtils.toMailboxId(_requestId++, SENDER_STAGE_ID, 0, RECEIVER_STAGE_ID, 0); SendingMailbox sendingMailbox = - _mailboxService1.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE); + _mailboxService1.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE, _stats); ReceivingMailbox receivingMailbox = _mailboxService1.getReceivingMailbox(mailboxId); AtomicInteger numCallbacks = new AtomicInteger(); receivingMailbox.registeredReader(numCallbacks::getAndIncrement); @@ -169,8 +178,9 @@ public void testLocalCancelledBySender() @Test public void testLocalCancelledBySenderBeforeSend() { String mailboxId = MailboxIdUtils.toMailboxId(_requestId++, SENDER_STAGE_ID, 0, RECEIVER_STAGE_ID, 0); + StatMap stats = new StatMap<>(MailboxSendOperator.StatKey.class); SendingMailbox sendingMailbox = - _mailboxService1.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE); + _mailboxService1.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE, stats); ReceivingMailbox receivingMailbox = _mailboxService1.getReceivingMailbox(mailboxId); AtomicInteger numCallbacks = new AtomicInteger(); receivingMailbox.registeredReader(numCallbacks::getAndIncrement); @@ -197,7 +207,7 @@ public void testLocalCancelledByReceiver() throws Exception { String mailboxId = MailboxIdUtils.toMailboxId(_requestId++, SENDER_STAGE_ID, 0, RECEIVER_STAGE_ID, 0); SendingMailbox sendingMailbox = - _mailboxService1.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE); + _mailboxService1.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE, _stats); ReceivingMailbox receivingMailbox = _mailboxService1.getReceivingMailbox(mailboxId); AtomicInteger numCallbacks = new AtomicInteger(); receivingMailbox.registeredReader(numCallbacks::getAndIncrement); @@ -226,7 +236,7 @@ public void testLocalTimeOut() String mailboxId = MailboxIdUtils.toMailboxId(_requestId++, SENDER_STAGE_ID, 0, RECEIVER_STAGE_ID, 0); long deadlineMs = System.currentTimeMillis() + 1000; SendingMailbox sendingMailbox = - _mailboxService1.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, deadlineMs); + _mailboxService1.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, deadlineMs, _stats); ReceivingMailbox receivingMailbox = _mailboxService1.getReceivingMailbox(mailboxId); AtomicInteger numCallbacks = new AtomicInteger(); receivingMailbox.registeredReader(numCallbacks::getAndIncrement); @@ -261,7 +271,7 @@ public void testLocalBufferFull() String mailboxId = MailboxIdUtils.toMailboxId(_requestId++, SENDER_STAGE_ID, 0, RECEIVER_STAGE_ID, 0); SendingMailbox sendingMailbox = _mailboxService1.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, - System.currentTimeMillis() + 1000); + System.currentTimeMillis() + 1000, _stats); ReceivingMailbox receivingMailbox = _mailboxService1.getReceivingMailbox(mailboxId); AtomicInteger numCallbacks = new AtomicInteger(); receivingMailbox.registeredReader(numCallbacks::getAndIncrement); @@ -273,7 +283,7 @@ public void testLocalBufferFull() // Next send will throw exception because buffer is full try { - sendingMailbox.send(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + sendingMailbox.send(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(SENDER_STAGE_ID)); fail("Except exception when sending data after buffer is full"); } catch (Exception e) { // Expected @@ -298,7 +308,7 @@ public void testLocalEarlyTerminated() throws Exception { String mailboxId = MailboxIdUtils.toMailboxId(_requestId++, SENDER_STAGE_ID, 0, RECEIVER_STAGE_ID, 0); SendingMailbox sendingMailbox = - _mailboxService1.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE); + _mailboxService1.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE, _stats); ReceivingMailbox receivingMailbox = _mailboxService1.getReceivingMailbox(mailboxId); receivingMailbox.registeredReader(() -> { }); @@ -313,7 +323,7 @@ public void testLocalEarlyTerminated() // send another block b/c it doesn't guarantee the next block must be EOS sendingMailbox.send(OperatorTestUtil.block(DATA_SCHEMA, new Object[]{0})); // send a metadata block - sendingMailbox.send(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + sendingMailbox.send(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(SENDER_STAGE_ID)); // sending side should early terminate assertTrue(sendingMailbox.isEarlyTerminated()); @@ -326,11 +336,11 @@ public void testRemoteHappyPathSendFirst() // Sends are non-blocking as long as channel capacity is not breached SendingMailbox sendingMailbox = - _mailboxService2.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE); + _mailboxService2.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE, _stats); for (int i = 0; i < ReceivingMailbox.DEFAULT_MAX_PENDING_BLOCKS - 1; i++) { sendingMailbox.send(OperatorTestUtil.block(DATA_SCHEMA, new Object[]{i})); } - sendingMailbox.send(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + sendingMailbox.send(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(SENDER_STAGE_ID)); sendingMailbox.complete(); // Wait until all the mails are delivered @@ -374,11 +384,11 @@ public void testRemoteHappyPathReceiveFirst() // Sends are non-blocking as long as channel capacity is not breached SendingMailbox sendingMailbox = - _mailboxService2.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE); + _mailboxService2.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE, _stats); for (int i = 0; i < ReceivingMailbox.DEFAULT_MAX_PENDING_BLOCKS - 1; i++) { sendingMailbox.send(OperatorTestUtil.block(DATA_SCHEMA, new Object[]{i})); } - sendingMailbox.send(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + sendingMailbox.send(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(SENDER_STAGE_ID)); sendingMailbox.complete(); // Wait until all the mails are delivered @@ -406,7 +416,7 @@ public void testRemoteCancelledBySender() throws Exception { String mailboxId = MailboxIdUtils.toMailboxId(_requestId++, SENDER_STAGE_ID, 0, RECEIVER_STAGE_ID, 0); SendingMailbox sendingMailbox = - _mailboxService2.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE); + _mailboxService2.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE, _stats); ReceivingMailbox receivingMailbox = _mailboxService1.getReceivingMailbox(mailboxId); AtomicInteger numCallbacks = new AtomicInteger(); CountDownLatch receiveMailLatch = new CountDownLatch(2); @@ -441,7 +451,7 @@ public void testRemoteCancelledBySenderBeforeSend() throws Exception { String mailboxId = MailboxIdUtils.toMailboxId(_requestId++, SENDER_STAGE_ID, 0, RECEIVER_STAGE_ID, 0); SendingMailbox sendingMailbox = - _mailboxService2.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE); + _mailboxService2.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE, _stats); ReceivingMailbox receivingMailbox = _mailboxService1.getReceivingMailbox(mailboxId); AtomicInteger numCallbacks = new AtomicInteger(); CountDownLatch receiveMailLatch = new CountDownLatch(1); @@ -475,7 +485,7 @@ public void testRemoteCancelledByReceiver() throws Exception { String mailboxId = MailboxIdUtils.toMailboxId(_requestId++, SENDER_STAGE_ID, 0, RECEIVER_STAGE_ID, 0); SendingMailbox sendingMailbox = - _mailboxService2.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE); + _mailboxService2.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE, _stats); ReceivingMailbox receivingMailbox = _mailboxService1.getReceivingMailbox(mailboxId); AtomicInteger numCallbacks = new AtomicInteger(); CountDownLatch receiveMailLatch = new CountDownLatch(1); @@ -509,7 +519,7 @@ public void testRemoteTimeOut() String mailboxId = MailboxIdUtils.toMailboxId(_requestId++, SENDER_STAGE_ID, 0, RECEIVER_STAGE_ID, 0); long deadlineMs = System.currentTimeMillis() + 1000; SendingMailbox sendingMailbox = - _mailboxService2.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, deadlineMs); + _mailboxService2.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, deadlineMs, _stats); ReceivingMailbox receivingMailbox = _mailboxService1.getReceivingMailbox(mailboxId); AtomicInteger numCallbacks = new AtomicInteger(); CountDownLatch receiveMailLatch = new CountDownLatch(2); @@ -552,7 +562,7 @@ public void testRemoteBufferFull() String mailboxId = MailboxIdUtils.toMailboxId(_requestId++, SENDER_STAGE_ID, 0, RECEIVER_STAGE_ID, 0); SendingMailbox sendingMailbox = _mailboxService2.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, - System.currentTimeMillis() + 1000); + System.currentTimeMillis() + 1000, _stats); ReceivingMailbox receivingMailbox = _mailboxService1.getReceivingMailbox(mailboxId); AtomicInteger numCallbacks = new AtomicInteger(); CountDownLatch receiveMailLatch = new CountDownLatch(ReceivingMailbox.DEFAULT_MAX_PENDING_BLOCKS + 1); @@ -567,7 +577,7 @@ public void testRemoteBufferFull() } // Next send will be blocked on the receiver side and cause exception after timeout - sendingMailbox.send(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + sendingMailbox.send(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(SENDER_STAGE_ID)); receiveMailLatch.await(); assertEquals(numCallbacks.get(), ReceivingMailbox.DEFAULT_MAX_PENDING_BLOCKS + 1); @@ -591,7 +601,7 @@ public void testRemoteEarlyTerminated() // Sends are non-blocking as long as channel capacity is not breached SendingMailbox sendingMailbox = - _mailboxService2.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE); + _mailboxService2.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE, _stats); ReceivingMailbox receivingMailbox = _mailboxService1.getReceivingMailbox(mailboxId); receivingMailbox.registeredReader(() -> { }); @@ -608,7 +618,7 @@ public void testRemoteEarlyTerminated() // send another block b/c it doesn't guarantee the next block must be EOS sendingMailbox.send(OperatorTestUtil.block(DATA_SCHEMA, new Object[]{0})); // send a metadata block - sendingMailbox.send(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + sendingMailbox.send(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(SENDER_STAGE_ID)); sendingMailbox.complete(); // sending side should early terminate diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/blocks/TransferableBlockTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/blocks/TransferableBlockTest.java new file mode 100644 index 000000000000..8ede37c65bd6 --- /dev/null +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/blocks/TransferableBlockTest.java @@ -0,0 +1,44 @@ +/** + * 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.pinot.query.runtime.blocks; + +import java.nio.ByteBuffer; +import java.util.List; +import org.apache.pinot.common.datablock.DataBlock; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStatsTest; +import org.testng.Assert; +import org.testng.annotations.Test; + + +public class TransferableBlockTest { + + @Test(dataProviderClass = MultiStageQueryStatsTest.class, dataProvider = "stats") + public void serializeDeserialize(MultiStageQueryStats queryStats) { + TransferableBlock transferableBlock = new TransferableBlock(queryStats); + List fromStatsBytes = transferableBlock.getSerializedStatsByStage(); + + DataBlock dataBlock = transferableBlock.getDataBlock(); + + TransferableBlock fromBlock = TransferableBlockUtils.wrap(dataBlock); + List fromBlockBytes = fromBlock.getSerializedStatsByStage(); + + Assert.assertEquals(fromStatsBytes, fromBlockBytes, "Serialized bytes from stats and block should be equal"); + } +} diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/blocks/TransferableBlockTestUtils.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/blocks/TransferableBlockTestUtils.java new file mode 100644 index 000000000000..a300326fb25c --- /dev/null +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/blocks/TransferableBlockTestUtils.java @@ -0,0 +1,45 @@ +/** + * 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.pinot.query.runtime.blocks; + +import org.apache.pinot.common.datablock.DataBlock; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; +import org.testng.Assert; + + +public class TransferableBlockTestUtils { + private TransferableBlockTestUtils() { + // do not instantiate. + } + + public static TransferableBlock getEndOfStreamTransferableBlock(int stageId) { + return TransferableBlockUtils.getEndOfStreamTransferableBlock(MultiStageQueryStats.emptyStats(stageId)); + } + + public static void assertSuccessEos(TransferableBlock block) { + Assert.assertEquals(block.getType(), DataBlock.Type.METADATA, "Block type should be metadata"); + Assert.assertTrue(block.isSuccessfulEndOfStreamBlock(), "Block should be successful EOS"); + } + + public static void assertDataBlock(TransferableBlock block) { + if (block.getType() != DataBlock.Type.ROW && block.getType() != DataBlock.Type.COLUMNAR) { + Assert.fail("Block type should be row or columnar but found " + block.getType()); + } + } +} diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/blocks/TransferableBlockUtilsTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/blocks/TransferableBlockUtilsTest.java index 94900dd334b8..d136f9179546 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/blocks/TransferableBlockUtilsTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/blocks/TransferableBlockUtilsTest.java @@ -89,7 +89,7 @@ public void testNonSplittableBlock() validateNonSplittableBlock(columnarBlock); // METADATA - MetadataBlock metadataBlock = new MetadataBlock(MetadataBlock.MetadataBlockType.EOS); + MetadataBlock metadataBlock = MetadataBlock.newEos(); validateNonSplittableBlock(metadataBlock); } diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerServiceTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerServiceTest.java index 7a3520252922..aff6a68853d4 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerServiceTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerServiceTest.java @@ -28,6 +28,7 @@ import org.apache.pinot.query.mailbox.MailboxService; import org.apache.pinot.query.routing.StageMetadata; import org.apache.pinot.query.routing.WorkerMetadata; +import org.apache.pinot.query.runtime.blocks.TransferableBlockTestUtils; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.operator.MultiStageOperator; import org.apache.pinot.query.runtime.operator.OpChain; @@ -91,7 +92,7 @@ public void shouldScheduleSingleOpChainRegisteredAfterStart() CountDownLatch latch = new CountDownLatch(1); Mockito.when(_operatorA.nextBlock()).thenAnswer(inv -> { latch.countDown(); - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + return TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0); }); schedulerService.register(opChain); @@ -108,7 +109,7 @@ public void shouldScheduleSingleOpChainRegisteredBeforeStart() CountDownLatch latch = new CountDownLatch(1); Mockito.when(_operatorA.nextBlock()).thenAnswer(inv -> { latch.countDown(); - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + return TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0); }); schedulerService.register(opChain); @@ -123,7 +124,7 @@ public void shouldCallCloseOnOperatorsThatFinishSuccessfully() OpChainSchedulerService schedulerService = new OpChainSchedulerService(_executor); CountDownLatch latch = new CountDownLatch(1); - Mockito.when(_operatorA.nextBlock()).thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + Mockito.when(_operatorA.nextBlock()).thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.doAnswer(inv -> { latch.countDown(); return null; diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/AggregateOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/AggregateOperatorTest.java index c207118aaa19..790bd3a90113 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/AggregateOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/AggregateOperatorTest.java @@ -18,7 +18,6 @@ */ package org.apache.pinot.query.runtime.operator; -import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import java.util.Collections; @@ -27,7 +26,7 @@ import org.apache.calcite.rel.hint.RelHint; import org.apache.calcite.sql.SqlKind; import org.apache.pinot.calcite.rel.hint.PinotHintOptions; -import org.apache.pinot.common.datatable.DataTable; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; import org.apache.pinot.query.planner.logical.RexExpression; @@ -35,6 +34,7 @@ import org.apache.pinot.query.planner.plannode.AggregateNode.AggType; import org.apache.pinot.query.routing.VirtualServerAddress; import org.apache.pinot.query.runtime.blocks.TransferableBlock; +import org.apache.pinot.query.runtime.blocks.TransferableBlockTestUtils; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.mockito.Mock; @@ -90,7 +90,7 @@ public void shouldHandleUpstreamErrorBlocks() { DataSchema outSchema = new DataSchema(new String[]{"group", "sum"}, new ColumnDataType[]{INT, DOUBLE}); AggregateOperator operator = - new AggregateOperator(OperatorTestUtil.getDefaultContext(), _input, outSchema, calls, group, AggType.DIRECT, + new AggregateOperator(OperatorTestUtil.getTracingContext(), _input, outSchema, calls, group, AggType.DIRECT, Collections.singletonList(-1), null); // When: @@ -107,11 +107,11 @@ public void shouldHandleEndOfStreamBlockWithNoOtherInputs() { List calls = ImmutableList.of(getSum(new RexExpression.InputRef(1))); List group = ImmutableList.of(new RexExpression.InputRef(0)); - Mockito.when(_input.nextBlock()).thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + Mockito.when(_input.nextBlock()).thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "sum"}, new ColumnDataType[]{INT, DOUBLE}); AggregateOperator operator = - new AggregateOperator(OperatorTestUtil.getDefaultContext(), _input, outSchema, calls, group, AggType.DIRECT, + new AggregateOperator(OperatorTestUtil.getTracingContext(), _input, outSchema, calls, group, AggType.DIRECT, Collections.singletonList(-1), null); // When: @@ -130,11 +130,11 @@ public void testAggregateSingleInputBlock() { DataSchema inSchema = new DataSchema(new String[]{"group", "arg"}, new ColumnDataType[]{INT, DOUBLE}); Mockito.when(_input.nextBlock()).thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, 1.0})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "sum"}, new ColumnDataType[]{INT, DOUBLE}); AggregateOperator operator = - new AggregateOperator(OperatorTestUtil.getDefaultContext(), _input, outSchema, calls, group, AggType.DIRECT, + new AggregateOperator(OperatorTestUtil.getTracingContext(), _input, outSchema, calls, group, AggType.DIRECT, Collections.singletonList(-1), null); // When: @@ -158,11 +158,11 @@ public void testAggregateMultipleInputBlocks() { Mockito.when(_input.nextBlock()) .thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, 1.0}, new Object[]{2, 2.0})) .thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, 3.0})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "sum"}, new ColumnDataType[]{INT, DOUBLE}); AggregateOperator operator = - new AggregateOperator(OperatorTestUtil.getDefaultContext(), _input, outSchema, calls, group, AggType.DIRECT, + new AggregateOperator(OperatorTestUtil.getTracingContext(), _input, outSchema, calls, group, AggType.DIRECT, Collections.singletonList(-1), null); // When: @@ -189,12 +189,12 @@ public void testAggregateWithFilter() { Mockito.when(_input.nextBlock()) .thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, 1.0, 0}, new Object[]{2, 2.0, 1})) .thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, 3.0, 1})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "sum", "sumWithFilter"}, new ColumnDataType[]{INT, DOUBLE, DOUBLE}); AggregateOperator operator = - new AggregateOperator(OperatorTestUtil.getDefaultContext(), _input, outSchema, calls, group, AggType.DIRECT, + new AggregateOperator(OperatorTestUtil.getTracingContext(), _input, outSchema, calls, group, AggType.DIRECT, filterArgIds, null); // When: @@ -215,7 +215,7 @@ public void testGroupByAggregateWithHashCollision() { RexExpression.FunctionCall agg = getSum(new RexExpression.InputRef(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "sum"}, new ColumnDataType[]{STRING, DOUBLE}); AggregateOperator sum0GroupBy1 = - new AggregateOperator(OperatorTestUtil.getDefaultContext(), upstreamOperator, outSchema, + new AggregateOperator(OperatorTestUtil.getTracingContext(), upstreamOperator, outSchema, Collections.singletonList(agg), Collections.singletonList(new RexExpression.InputRef(1)), AggType.DIRECT, Collections.singletonList(-1), null); TransferableBlock result = sum0GroupBy1.getNextBlock(); @@ -239,7 +239,7 @@ public void shouldThrowOnUnknownAggFunction() { DataSchema outSchema = new DataSchema(new String[]{"unknown"}, new ColumnDataType[]{DOUBLE}); // When: - new AggregateOperator(OperatorTestUtil.getDefaultContext(), _input, outSchema, calls, group, AggType.DIRECT, + new AggregateOperator(OperatorTestUtil.getTracingContext(), _input, outSchema, calls, group, AggType.DIRECT, Collections.singletonList(-1), null); } @@ -254,11 +254,11 @@ public void shouldReturnErrorBlockOnUnexpectedInputType() { // TODO: it is necessary to produce two values here, the operator only throws on second // (see the comment in Aggregate operator) .thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, "foo"}, new Object[]{2, "foo"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"sum"}, new ColumnDataType[]{DOUBLE}); AggregateOperator operator = - new AggregateOperator(OperatorTestUtil.getDefaultContext(), _input, outSchema, calls, group, + new AggregateOperator(OperatorTestUtil.getTracingContext(), _input, outSchema, calls, group, AggType.INTERMEDIATE, Collections.singletonList(-1), null); // When: @@ -280,10 +280,10 @@ public void shouldHandleGroupLimitExceed() { Mockito.when(_input.nextBlock()) .thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, 1.0}, new Object[]{3, 2.0})) .thenReturn(OperatorTestUtil.block(inSchema, new Object[]{3, 3.0})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "sum"}, new ColumnDataType[]{INT, DOUBLE}); - OpChainExecutionContext context = OperatorTestUtil.getDefaultContext(); + OpChainExecutionContext context = OperatorTestUtil.getTracingContext(); Map hintsMap = ImmutableMap.of(PinotHintOptions.AggregateOptions.NUM_GROUPS_LIMIT, "1"); AggregateOperator operator = new AggregateOperator(context, _input, outSchema, calls, group, AggType.DIRECT, Collections.singletonList(-1), @@ -297,13 +297,12 @@ public void shouldHandleGroupLimitExceed() { Mockito.verify(_input).earlyTerminate(); // Then: - Assert.assertTrue(block1.getNumRows() == 1, "when group limit reach it should only return that many groups"); + Assert.assertEquals(block1.getNumRows(), 1, "when group limit reach it should only return that many groups"); Assert.assertTrue(block2.isEndOfStreamBlock(), "Second block is EOS (done processing)"); - String operatorId = - Joiner.on("_").join(AggregateOperator.class.getSimpleName(), context.getStageId(), context.getServer()); - OperatorStats operatorStats = context.getStats().getOperatorStats(context, operatorId); - Assert.assertEquals(operatorStats.getExecutionStats().get(DataTable.MetadataKey.NUM_GROUPS_LIMIT_REACHED.getName()), - "true"); + StatMap aggrStats = + OperatorTestUtil.getStatMap(AggregateOperator.StatKey.class, block2); + Assert.assertTrue(aggrStats.getBoolean(AggregateOperator.StatKey.NUM_GROUPS_LIMIT_REACHED), + "num groups limit should be reached"); } private static RexExpression.FunctionCall getSum(RexExpression arg) { diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/FilterOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/FilterOperatorTest.java index f68fbdea99ca..27825f916aea 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/FilterOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/FilterOperatorTest.java @@ -26,6 +26,7 @@ import org.apache.pinot.common.utils.DataSchema.ColumnDataType; import org.apache.pinot.query.planner.logical.RexExpression; import org.apache.pinot.query.runtime.blocks.TransferableBlock; +import org.apache.pinot.query.runtime.blocks.TransferableBlockTestUtils; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.mockito.Mock; import org.mockito.Mockito; @@ -61,7 +62,7 @@ public void shouldPropagateUpstreamErrorBlock() { ColumnDataType.BOOLEAN }); FilterOperator op = - new FilterOperator(OperatorTestUtil.getDefaultContext(), _upstreamOperator, inputSchema, booleanLiteral); + new FilterOperator(OperatorTestUtil.getTracingContext(), _upstreamOperator, inputSchema, booleanLiteral); TransferableBlock errorBlock = op.getNextBlock(); Assert.assertTrue(errorBlock.isErrorBlock()); Assert.assertTrue(errorBlock.getExceptions().get(QueryException.UNKNOWN_ERROR_CODE).contains("filterError")); @@ -73,9 +74,10 @@ public void shouldPropagateUpstreamEOS() { DataSchema inputSchema = new DataSchema(new String[]{"intCol"}, new ColumnDataType[]{ ColumnDataType.INT }); - Mockito.when(_upstreamOperator.nextBlock()).thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + Mockito.when(_upstreamOperator.nextBlock()) + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); FilterOperator op = - new FilterOperator(OperatorTestUtil.getDefaultContext(), _upstreamOperator, inputSchema, booleanLiteral); + new FilterOperator(OperatorTestUtil.getTracingContext(), _upstreamOperator, inputSchema, booleanLiteral); TransferableBlock dataBlock = op.getNextBlock(); Assert.assertTrue(dataBlock.isEndOfStreamBlock()); } @@ -88,9 +90,9 @@ public void shouldHandleTrueBooleanLiteralFilter() { }); Mockito.when(_upstreamOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(inputSchema, new Object[]{0}, new Object[]{1})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); FilterOperator op = - new FilterOperator(OperatorTestUtil.getDefaultContext(), _upstreamOperator, inputSchema, booleanLiteral); + new FilterOperator(OperatorTestUtil.getTracingContext(), _upstreamOperator, inputSchema, booleanLiteral); TransferableBlock dataBlock = op.getNextBlock(); Assert.assertFalse(dataBlock.isErrorBlock()); List result = dataBlock.getContainer(); @@ -108,7 +110,7 @@ public void shouldHandleFalseBooleanLiteralFilter() { Mockito.when(_upstreamOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(inputSchema, new Object[]{1}, new Object[]{2})); FilterOperator op = - new FilterOperator(OperatorTestUtil.getDefaultContext(), _upstreamOperator, inputSchema, booleanLiteral); + new FilterOperator(OperatorTestUtil.getTracingContext(), _upstreamOperator, inputSchema, booleanLiteral); TransferableBlock dataBlock = op.getNextBlock(); Assert.assertFalse(dataBlock.isErrorBlock()); List result = dataBlock.getContainer(); @@ -125,7 +127,7 @@ public void shouldThrowOnNonBooleanTypeBooleanLiteral() { Mockito.when(_upstreamOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(inputSchema, new Object[]{1}, new Object[]{2})); FilterOperator op = - new FilterOperator(OperatorTestUtil.getDefaultContext(), _upstreamOperator, inputSchema, booleanLiteral); + new FilterOperator(OperatorTestUtil.getTracingContext(), _upstreamOperator, inputSchema, booleanLiteral); op.getNextBlock(); } @@ -138,7 +140,7 @@ public void shouldThrowOnNonBooleanTypeInputRef() { }); Mockito.when(_upstreamOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(inputSchema, new Object[]{1}, new Object[]{2})); - FilterOperator op = new FilterOperator(OperatorTestUtil.getDefaultContext(), _upstreamOperator, inputSchema, ref0); + FilterOperator op = new FilterOperator(OperatorTestUtil.getTracingContext(), _upstreamOperator, inputSchema, ref0); op.getNextBlock(); } @@ -150,7 +152,7 @@ public void shouldHandleBooleanInputRef() { }); Mockito.when(_upstreamOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(inputSchema, new Object[]{1, 1}, new Object[]{2, 0})); - FilterOperator op = new FilterOperator(OperatorTestUtil.getDefaultContext(), _upstreamOperator, inputSchema, ref1); + FilterOperator op = new FilterOperator(OperatorTestUtil.getTracingContext(), _upstreamOperator, inputSchema, ref1); TransferableBlock dataBlock = op.getNextBlock(); Assert.assertFalse(dataBlock.isErrorBlock()); List result = dataBlock.getContainer(); @@ -170,7 +172,7 @@ public void shouldHandleAndFilter() { ImmutableList.of(new RexExpression.InputRef(0), new RexExpression.InputRef(1))); FilterOperator op = - new FilterOperator(OperatorTestUtil.getDefaultContext(), _upstreamOperator, inputSchema, andCall); + new FilterOperator(OperatorTestUtil.getTracingContext(), _upstreamOperator, inputSchema, andCall); TransferableBlock dataBlock = op.getNextBlock(); Assert.assertFalse(dataBlock.isErrorBlock()); List result = dataBlock.getContainer(); @@ -190,7 +192,7 @@ public void shouldHandleOrFilter() { ImmutableList.of(new RexExpression.InputRef(0), new RexExpression.InputRef(1))); FilterOperator op = - new FilterOperator(OperatorTestUtil.getDefaultContext(), _upstreamOperator, inputSchema, orCall); + new FilterOperator(OperatorTestUtil.getTracingContext(), _upstreamOperator, inputSchema, orCall); TransferableBlock dataBlock = op.getNextBlock(); Assert.assertFalse(dataBlock.isErrorBlock()); List result = dataBlock.getContainer(); @@ -212,7 +214,7 @@ public void shouldHandleNotFilter() { ImmutableList.of(new RexExpression.InputRef(0))); FilterOperator op = - new FilterOperator(OperatorTestUtil.getDefaultContext(), _upstreamOperator, inputSchema, notCall); + new FilterOperator(OperatorTestUtil.getTracingContext(), _upstreamOperator, inputSchema, notCall); TransferableBlock dataBlock = op.getNextBlock(); Assert.assertFalse(dataBlock.isErrorBlock()); List result = dataBlock.getContainer(); @@ -232,7 +234,7 @@ public void shouldHandleGreaterThanFilter() { new RexExpression.FunctionCall(SqlKind.GREATER_THAN, ColumnDataType.BOOLEAN, "greaterThan", ImmutableList.of(new RexExpression.InputRef(0), new RexExpression.InputRef(1))); FilterOperator op = - new FilterOperator(OperatorTestUtil.getDefaultContext(), _upstreamOperator, inputSchema, greaterThan); + new FilterOperator(OperatorTestUtil.getTracingContext(), _upstreamOperator, inputSchema, greaterThan); TransferableBlock dataBlock = op.getNextBlock(); Assert.assertFalse(dataBlock.isErrorBlock()); List result = dataBlock.getContainer(); @@ -252,7 +254,7 @@ public void shouldHandleBooleanFunction() { new RexExpression.FunctionCall(SqlKind.OTHER, ColumnDataType.BOOLEAN, "startsWith", ImmutableList.of(new RexExpression.InputRef(0), new RexExpression.Literal(ColumnDataType.STRING, "star"))); FilterOperator op = - new FilterOperator(OperatorTestUtil.getDefaultContext(), _upstreamOperator, inputSchema, startsWith); + new FilterOperator(OperatorTestUtil.getTracingContext(), _upstreamOperator, inputSchema, startsWith); TransferableBlock dataBlock = op.getNextBlock(); Assert.assertFalse(dataBlock.isErrorBlock()); List result = dataBlock.getContainer(); @@ -272,6 +274,6 @@ public void shouldThrowOnUnfoundFunction() { RexExpression.FunctionCall startsWith = new RexExpression.FunctionCall(SqlKind.OTHER, ColumnDataType.BOOLEAN, "startsWithError", ImmutableList.of(new RexExpression.InputRef(0), new RexExpression.Literal(ColumnDataType.STRING, "star"))); - new FilterOperator(OperatorTestUtil.getDefaultContext(), _upstreamOperator, inputSchema, startsWith); + new FilterOperator(OperatorTestUtil.getTracingContext(), _upstreamOperator, inputSchema, startsWith); } } diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/HashJoinOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/HashJoinOperatorTest.java index 48dbe8ef3e72..45afa6dbc6c8 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/HashJoinOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/HashJoinOperatorTest.java @@ -18,7 +18,6 @@ */ package org.apache.pinot.query.runtime.operator; -import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import java.util.ArrayList; @@ -30,7 +29,7 @@ import org.apache.calcite.rel.hint.RelHint; import org.apache.calcite.sql.SqlKind; import org.apache.pinot.calcite.rel.hint.PinotHintOptions; -import org.apache.pinot.common.datatable.DataTable; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.exception.QueryException; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; @@ -38,6 +37,7 @@ import org.apache.pinot.query.planner.plannode.JoinNode; import org.apache.pinot.query.routing.VirtualServerAddress; import org.apache.pinot.query.runtime.blocks.TransferableBlock; +import org.apache.pinot.query.runtime.blocks.TransferableBlockTestUtils; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.mockito.Mock; @@ -94,10 +94,10 @@ public void shouldHandleHashJoinKeyCollisionInnerJoin() { List joinClauses = new ArrayList<>(); Mockito.when(_leftOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(leftSchema, new Object[]{1, "Aa"}, new Object[]{2, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(rightSchema, new Object[]{2, "Aa"}, new Object[]{2, "BB"}, new Object[]{3, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema resultSchema = new DataSchema(new String[]{"int_col1", "string_col1", "int_col2", "string_col2"}, new ColumnDataType[]{ ColumnDataType.INT, ColumnDataType.STRING, ColumnDataType.INT, ColumnDataType.STRING @@ -105,7 +105,7 @@ public void shouldHandleHashJoinKeyCollisionInnerJoin() { JoinNode node = new JoinNode(1, resultSchema, leftSchema, rightSchema, JoinRelType.INNER, getJoinKeys(Arrays.asList(1), Arrays.asList(1)), joinClauses, Collections.emptyList()); HashJoinOperator joinOnString = - new HashJoinOperator(OperatorTestUtil.getDefaultContext(), _leftOperator, _rightOperator, leftSchema, node); + new HashJoinOperator(OperatorTestUtil.getTracingContext(), _leftOperator, _rightOperator, leftSchema, node); TransferableBlock result = joinOnString.nextBlock(); List resultRows = result.getContainer(); @@ -127,10 +127,10 @@ public void shouldHandleInnerJoinOnInt() { }); Mockito.when(_leftOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(leftSchema, new Object[]{1, "Aa"}, new Object[]{2, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(rightSchema, new Object[]{2, "Aa"}, new Object[]{2, "BB"}, new Object[]{3, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); List joinClauses = new ArrayList<>(); DataSchema resultSchema = new DataSchema(new String[]{"int_col1", "string_col1", "int_col2", "string_co2"}, new ColumnDataType[]{ @@ -139,7 +139,7 @@ public void shouldHandleInnerJoinOnInt() { JoinNode node = new JoinNode(1, resultSchema, leftSchema, rightSchema, JoinRelType.INNER, getJoinKeys(Arrays.asList(0), Arrays.asList(0)), joinClauses, Collections.emptyList()); HashJoinOperator joinOnInt = - new HashJoinOperator(OperatorTestUtil.getDefaultContext(), _leftOperator, _rightOperator, leftSchema, node); + new HashJoinOperator(OperatorTestUtil.getTracingContext(), _leftOperator, _rightOperator, leftSchema, node); TransferableBlock result = joinOnInt.nextBlock(); List resultRows = result.getContainer(); List expectedRows = Arrays.asList(new Object[]{2, "BB", 2, "Aa"}, new Object[]{2, "BB", 2, "BB"}); @@ -158,10 +158,10 @@ public void shouldHandleJoinOnEmptySelector() { }); Mockito.when(_leftOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(leftSchema, new Object[]{1, "Aa"}, new Object[]{2, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(rightSchema, new Object[]{2, "Aa"}, new Object[]{2, "BB"}, new Object[]{3, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); List joinClauses = new ArrayList<>(); DataSchema resultSchema = new DataSchema(new String[]{"int_col1", "string_col1", "int_col2", "string_co2"}, new ColumnDataType[]{ @@ -170,7 +170,7 @@ public void shouldHandleJoinOnEmptySelector() { JoinNode node = new JoinNode(1, resultSchema, leftSchema, rightSchema, JoinRelType.INNER, getJoinKeys(new ArrayList<>(), new ArrayList<>()), joinClauses, Collections.emptyList()); HashJoinOperator joinOnInt = - new HashJoinOperator(OperatorTestUtil.getDefaultContext(), _leftOperator, _rightOperator, leftSchema, node); + new HashJoinOperator(OperatorTestUtil.getTracingContext(), _leftOperator, _rightOperator, leftSchema, node); TransferableBlock result = joinOnInt.nextBlock(); List resultRows = result.getContainer(); List expectedRows = @@ -195,10 +195,10 @@ public void shouldHandleLeftJoin() { }); Mockito.when(_leftOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(leftSchema, new Object[]{1, "Aa"}, new Object[]{2, "CC"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(rightSchema, new Object[]{2, "Aa"}, new Object[]{2, "BB"}, new Object[]{3, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); List joinClauses = new ArrayList<>(); DataSchema resultSchema = @@ -208,7 +208,7 @@ public void shouldHandleLeftJoin() { JoinNode node = new JoinNode(1, resultSchema, leftSchema, rightSchema, JoinRelType.LEFT, getJoinKeys(Arrays.asList(1), Arrays.asList(1)), joinClauses, Collections.emptyList()); HashJoinOperator join = - new HashJoinOperator(OperatorTestUtil.getDefaultContext(), _leftOperator, _rightOperator, leftSchema, node); + new HashJoinOperator(OperatorTestUtil.getTracingContext(), _leftOperator, _rightOperator, leftSchema, node); TransferableBlock result = join.nextBlock(); List resultRows = result.getContainer(); @@ -226,10 +226,10 @@ public void shouldPassLeftTableEOS() { DataSchema rightSchema = new DataSchema(new String[]{"int_col", "string_col"}, new ColumnDataType[]{ ColumnDataType.INT, ColumnDataType.STRING }); - Mockito.when(_leftOperator.nextBlock()).thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + Mockito.when(_leftOperator.nextBlock()).thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(rightSchema, new Object[]{1, "BB"}, new Object[]{1, "CC"}, new Object[]{3, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema resultSchema = new DataSchema(new String[]{"int_col1", "string_col1", "int_co2", "string_col2"}, new ColumnDataType[]{ @@ -239,7 +239,7 @@ public void shouldPassLeftTableEOS() { JoinNode node = new JoinNode(1, resultSchema, leftSchema, rightSchema, JoinRelType.INNER, getJoinKeys(Arrays.asList(0), Arrays.asList(0)), joinClauses, Collections.emptyList()); HashJoinOperator join = - new HashJoinOperator(OperatorTestUtil.getDefaultContext(), _leftOperator, _rightOperator, leftSchema, node); + new HashJoinOperator(OperatorTestUtil.getTracingContext(), _leftOperator, _rightOperator, leftSchema, node); TransferableBlock result = join.nextBlock(); Assert.assertTrue(result.isEndOfStreamBlock()); @@ -254,10 +254,10 @@ public void shouldHandleLeftJoinOneToN() { ColumnDataType.INT, ColumnDataType.STRING }); Mockito.when(_leftOperator.nextBlock()).thenReturn(OperatorTestUtil.block(leftSchema, new Object[]{1, "Aa"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(rightSchema, new Object[]{1, "BB"}, new Object[]{1, "CC"}, new Object[]{3, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); List joinClauses = new ArrayList<>(); DataSchema resultSchema = @@ -267,7 +267,7 @@ public void shouldHandleLeftJoinOneToN() { JoinNode node = new JoinNode(1, resultSchema, leftSchema, rightSchema, JoinRelType.LEFT, getJoinKeys(Arrays.asList(0), Arrays.asList(0)), joinClauses, Collections.emptyList()); HashJoinOperator join = - new HashJoinOperator(OperatorTestUtil.getDefaultContext(), _leftOperator, _rightOperator, leftSchema, node); + new HashJoinOperator(OperatorTestUtil.getTracingContext(), _leftOperator, _rightOperator, leftSchema, node); TransferableBlock result = join.nextBlock(); List resultRows = result.getContainer(); @@ -287,8 +287,8 @@ public void shouldPassRightTableEOS() { }); Mockito.when(_leftOperator.nextBlock()).thenReturn( OperatorTestUtil.block(rightSchema, new Object[]{1, "BB"}, new Object[]{1, "CC"}, new Object[]{3, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); - Mockito.when(_rightOperator.nextBlock()).thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); + Mockito.when(_rightOperator.nextBlock()).thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); List joinClauses = new ArrayList<>(); DataSchema resultSchema = @@ -299,7 +299,7 @@ public void shouldPassRightTableEOS() { JoinNode node = new JoinNode(1, resultSchema, leftSchema, rightSchema, JoinRelType.INNER, getJoinKeys(Arrays.asList(0), Arrays.asList(0)), joinClauses, Collections.emptyList()); HashJoinOperator join = - new HashJoinOperator(OperatorTestUtil.getDefaultContext(), _leftOperator, _rightOperator, leftSchema, node); + new HashJoinOperator(OperatorTestUtil.getTracingContext(), _leftOperator, _rightOperator, leftSchema, node); TransferableBlock result = join.nextBlock(); List resultRows = result.getContainer(); @@ -316,10 +316,10 @@ public void shouldHandleInequiJoinOnString() { }); Mockito.when(_leftOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(leftSchema, new Object[]{1, "Aa"}, new Object[]{2, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(rightSchema, new Object[]{2, "Aa"}, new Object[]{2, "BB"}, new Object[]{3, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); List joinClauses = new ArrayList<>(); List functionOperands = new ArrayList<>(); @@ -333,7 +333,7 @@ public void shouldHandleInequiJoinOnString() { JoinNode node = new JoinNode(1, resultSchema, leftSchema, rightSchema, JoinRelType.INNER, getJoinKeys(new ArrayList<>(), new ArrayList<>()), joinClauses, Collections.emptyList()); HashJoinOperator join = - new HashJoinOperator(OperatorTestUtil.getDefaultContext(), _leftOperator, _rightOperator, leftSchema, node); + new HashJoinOperator(OperatorTestUtil.getTracingContext(), _leftOperator, _rightOperator, leftSchema, node); TransferableBlock result = join.nextBlock(); List resultRows = result.getContainer(); List expectedRows = @@ -354,10 +354,10 @@ public void shouldHandleInequiJoinOnInt() { }); Mockito.when(_leftOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(leftSchema, new Object[]{1, "Aa"}, new Object[]{2, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(rightSchema, new Object[]{2, "Aa"}, new Object[]{1, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); List joinClauses = new ArrayList<>(); List functionOperands = new ArrayList<>(); @@ -371,7 +371,7 @@ public void shouldHandleInequiJoinOnInt() { JoinNode node = new JoinNode(1, resultSchema, leftSchema, rightSchema, JoinRelType.INNER, getJoinKeys(new ArrayList<>(), new ArrayList<>()), joinClauses, Collections.emptyList()); HashJoinOperator join = - new HashJoinOperator(OperatorTestUtil.getDefaultContext(), _leftOperator, _rightOperator, leftSchema, node); + new HashJoinOperator(OperatorTestUtil.getTracingContext(), _leftOperator, _rightOperator, leftSchema, node); TransferableBlock result = join.nextBlock(); List resultRows = result.getContainer(); List expectedRows = Arrays.asList(new Object[]{1, "Aa", 2, "Aa"}, new Object[]{2, "BB", 1, "BB"}); @@ -391,10 +391,10 @@ public void shouldHandleRightJoin() { }); Mockito.when(_leftOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(leftSchema, new Object[]{1, "Aa"}, new Object[]{2, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(rightSchema, new Object[]{2, "Aa"}, new Object[]{2, "BB"}, new Object[]{3, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); List joinClauses = new ArrayList<>(); DataSchema resultSchema = new DataSchema(new String[]{"foo", "bar", "foo", "bar"}, new ColumnDataType[]{ @@ -403,7 +403,7 @@ public void shouldHandleRightJoin() { JoinNode node = new JoinNode(1, resultSchema, leftSchema, rightSchema, JoinRelType.RIGHT, getJoinKeys(Arrays.asList(0), Arrays.asList(0)), joinClauses, Collections.emptyList()); HashJoinOperator joinOnNum = - new HashJoinOperator(OperatorTestUtil.getDefaultContext(), _leftOperator, _rightOperator, leftSchema, node); + new HashJoinOperator(OperatorTestUtil.getTracingContext(), _leftOperator, _rightOperator, leftSchema, node); TransferableBlock result = joinOnNum.nextBlock(); List resultRows = result.getContainer(); List expectedRows = Arrays.asList(new Object[]{2, "BB", 2, "Aa"}, new Object[]{2, "BB", 2, "BB"}); @@ -418,7 +418,7 @@ public void shouldHandleRightJoin() { Assert.assertEquals(resultRows.get(0), expectedRows.get(0)); // Third block is EOS block. result = joinOnNum.nextBlock(); - Assert.assertTrue(result.isSuccessfulEndOfStreamBlock()); + TransferableBlockTestUtils.assertSuccessEos(result); } @Test @@ -431,10 +431,10 @@ public void shouldHandleSemiJoin() { }); Mockito.when(_leftOperator.nextBlock()).thenReturn( OperatorTestUtil.block(leftSchema, new Object[]{1, "Aa"}, new Object[]{2, "BB"}, new Object[]{4, "CC"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(rightSchema, new Object[]{2, "Aa"}, new Object[]{2, "BB"}, new Object[]{3, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); List joinClauses = new ArrayList<>(); DataSchema resultSchema = new DataSchema(new String[]{"foo", "bar", "foo", "bar"}, new ColumnDataType[]{ @@ -443,7 +443,7 @@ public void shouldHandleSemiJoin() { JoinNode node = new JoinNode(1, resultSchema, leftSchema, rightSchema, JoinRelType.SEMI, getJoinKeys(Arrays.asList(1), Arrays.asList(1)), joinClauses, Collections.emptyList()); HashJoinOperator join = - new HashJoinOperator(OperatorTestUtil.getDefaultContext(), _leftOperator, _rightOperator, leftSchema, node); + new HashJoinOperator(OperatorTestUtil.getTracingContext(), _leftOperator, _rightOperator, leftSchema, node); TransferableBlock result = join.nextBlock(); List resultRows = result.getContainer(); List expectedRows = @@ -465,10 +465,10 @@ public void shouldHandleFullJoin() { }); Mockito.when(_leftOperator.nextBlock()).thenReturn( OperatorTestUtil.block(leftSchema, new Object[]{1, "Aa"}, new Object[]{2, "BB"}, new Object[]{4, "CC"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(rightSchema, new Object[]{2, "Aa"}, new Object[]{2, "BB"}, new Object[]{3, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); List joinClauses = new ArrayList<>(); DataSchema resultSchema = new DataSchema(new String[]{"foo", "bar", "foo", "bar"}, new ColumnDataType[]{ ColumnDataType.INT, ColumnDataType.STRING, ColumnDataType.INT, ColumnDataType.STRING @@ -476,7 +476,7 @@ public void shouldHandleFullJoin() { JoinNode node = new JoinNode(1, resultSchema, leftSchema, rightSchema, JoinRelType.FULL, getJoinKeys(Arrays.asList(0), Arrays.asList(0)), joinClauses, Collections.emptyList()); HashJoinOperator join = - new HashJoinOperator(OperatorTestUtil.getDefaultContext(), _leftOperator, _rightOperator, leftSchema, node); + new HashJoinOperator(OperatorTestUtil.getTracingContext(), _leftOperator, _rightOperator, leftSchema, node); TransferableBlock result = join.nextBlock(); List resultRows = result.getContainer(); List expectedRows = ImmutableList.of(new Object[]{1, "Aa", null, null}, new Object[]{2, "BB", 2, "Aa"}, @@ -490,11 +490,12 @@ public void shouldHandleFullJoin() { result = join.nextBlock(); resultRows = result.getContainer(); expectedRows = ImmutableList.of(new Object[]{null, null, 3, "BB"}); + TransferableBlockTestUtils.assertDataBlock(result); Assert.assertEquals(resultRows.size(), expectedRows.size()); Assert.assertEquals(resultRows.get(0), expectedRows.get(0)); // Third block is EOS block. result = join.nextBlock(); - Assert.assertTrue(result.isSuccessfulEndOfStreamBlock()); + TransferableBlockTestUtils.assertSuccessEos(result); } @Test @@ -507,10 +508,10 @@ public void shouldHandleAntiJoin() { }); Mockito.when(_leftOperator.nextBlock()).thenReturn( OperatorTestUtil.block(leftSchema, new Object[]{1, "Aa"}, new Object[]{2, "BB"}, new Object[]{4, "CC"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(rightSchema, new Object[]{2, "Aa"}, new Object[]{2, "BB"}, new Object[]{3, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); List joinClauses = new ArrayList<>(); DataSchema resultSchema = new DataSchema(new String[]{"foo", "bar", "foo", "bar"}, new ColumnDataType[]{ @@ -519,7 +520,7 @@ public void shouldHandleAntiJoin() { JoinNode node = new JoinNode(1, resultSchema, leftSchema, rightSchema, JoinRelType.ANTI, getJoinKeys(Arrays.asList(1), Arrays.asList(1)), joinClauses, Collections.emptyList()); HashJoinOperator join = - new HashJoinOperator(OperatorTestUtil.getDefaultContext(), _leftOperator, _rightOperator, leftSchema, node); + new HashJoinOperator(OperatorTestUtil.getTracingContext(), _leftOperator, _rightOperator, leftSchema, node); TransferableBlock result = join.nextBlock(); List resultRows = result.getContainer(); List expectedRows = ImmutableList.of(new Object[]{4, "CC", null, null}); @@ -539,7 +540,7 @@ public void shouldPropagateRightTableError() { }); Mockito.when(_leftOperator.nextBlock()).thenReturn( OperatorTestUtil.block(rightSchema, new Object[]{1, "BB"}, new Object[]{1, "CC"}, new Object[]{3, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()) .thenReturn(TransferableBlockUtils.getErrorTransferableBlock(new Exception("testInnerJoinRightError"))); @@ -551,7 +552,7 @@ public void shouldPropagateRightTableError() { JoinNode node = new JoinNode(1, resultSchema, leftSchema, rightSchema, JoinRelType.INNER, getJoinKeys(Arrays.asList(0), Arrays.asList(0)), joinClauses, Collections.emptyList()); HashJoinOperator join = - new HashJoinOperator(OperatorTestUtil.getDefaultContext(), _leftOperator, _rightOperator, leftSchema, node); + new HashJoinOperator(OperatorTestUtil.getTracingContext(), _leftOperator, _rightOperator, leftSchema, node); TransferableBlock result = join.nextBlock(); Assert.assertTrue(result.isErrorBlock()); @@ -569,7 +570,7 @@ public void shouldPropagateLeftTableError() { }); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(rightSchema, new Object[]{1, "BB"}, new Object[]{1, "CC"}, new Object[]{3, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_leftOperator.nextBlock()) .thenReturn(TransferableBlockUtils.getErrorTransferableBlock(new Exception("testInnerJoinLeftError"))); @@ -581,7 +582,7 @@ public void shouldPropagateLeftTableError() { JoinNode node = new JoinNode(1, resultSchema, leftSchema, rightSchema, JoinRelType.INNER, getJoinKeys(Arrays.asList(0), Arrays.asList(0)), joinClauses, Collections.emptyList()); HashJoinOperator join = - new HashJoinOperator(OperatorTestUtil.getDefaultContext(), _leftOperator, _rightOperator, leftSchema, node); + new HashJoinOperator(OperatorTestUtil.getTracingContext(), _leftOperator, _rightOperator, leftSchema, node); TransferableBlock result = join.nextBlock(); Assert.assertTrue(result.isErrorBlock()); @@ -598,10 +599,10 @@ public void shouldPropagateJoinLimitError() { }); Mockito.when(_leftOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(leftSchema, new Object[]{1, "Aa"}, new Object[]{2, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(rightSchema, new Object[]{2, "Aa"}, new Object[]{2, "BB"}, new Object[]{3, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); List joinClauses = new ArrayList<>(); DataSchema resultSchema = @@ -613,7 +614,7 @@ public void shouldPropagateJoinLimitError() { JoinNode node = new JoinNode(1, resultSchema, leftSchema, rightSchema, JoinRelType.INNER, getJoinKeys(Arrays.asList(0), Arrays.asList(0)), joinClauses, getJoinHints(hintsMap)); HashJoinOperator join = - new HashJoinOperator(OperatorTestUtil.getDefaultContext(), _leftOperator, _rightOperator, leftSchema, node); + new HashJoinOperator(OperatorTestUtil.getTracingContext(), _leftOperator, _rightOperator, leftSchema, node); TransferableBlock result = join.nextBlock(); Assert.assertTrue(result.isErrorBlock()); @@ -631,10 +632,10 @@ public void shouldHandleJoinWithPartialResultsWhenHitDataRowsLimit() { }); Mockito.when(_leftOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(leftSchema, new Object[]{1, "Aa"}, new Object[]{2, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(rightSchema, new Object[]{2, "Aa"}, new Object[]{2, "BB"}, new Object[]{3, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); List joinClauses = new ArrayList<>(); DataSchema resultSchema = @@ -646,19 +647,19 @@ public void shouldHandleJoinWithPartialResultsWhenHitDataRowsLimit() { JoinNode node = new JoinNode(1, resultSchema, leftSchema, rightSchema, JoinRelType.INNER, getJoinKeys(Arrays.asList(0), Arrays.asList(0)), joinClauses, getJoinHints(hintsMap)); - OpChainExecutionContext context = OperatorTestUtil.getDefaultContext(); + OpChainExecutionContext context = OperatorTestUtil.getTracingContext(); HashJoinOperator join = new HashJoinOperator(context, _leftOperator, _rightOperator, leftSchema, node); - TransferableBlock result = join.nextBlock(); + TransferableBlock firstBlock = join.nextBlock(); Mockito.verify(_rightOperator).earlyTerminate(); - Assert.assertFalse(result.isErrorBlock()); - Assert.assertEquals(result.getNumRows(), 1); - - String operatorId = - Joiner.on("_").join(HashJoinOperator.class.getSimpleName(), context.getStageId(), context.getServer()); - OperatorStats operatorStats = context.getStats().getOperatorStats(context, operatorId); - Assert.assertEquals( - operatorStats.getExecutionStats().get(DataTable.MetadataKey.MAX_ROWS_IN_JOIN_REACHED.getName()), "true"); + Assert.assertTrue(firstBlock.isDataBlock(), "First block should be a data block but is " + firstBlock.getClass()); + Assert.assertEquals(firstBlock.getNumRows(), 1); + + TransferableBlock secondBlock = join.nextBlock(); + StatMap joinStats = + OperatorTestUtil.getStatMap(HashJoinOperator.StatKey.class, secondBlock); + Assert.assertTrue(joinStats.getBoolean(HashJoinOperator.StatKey.MAX_ROWS_IN_JOIN_REACHED), + "Max rows in join should be reached"); } } // TODO: Add more inequi join tests. diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/IntersectOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/IntersectOperatorTest.java index 04c14caea1bd..51cc068f9013 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/IntersectOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/IntersectOperatorTest.java @@ -25,7 +25,7 @@ import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.query.routing.VirtualServerAddress; import org.apache.pinot.query.runtime.blocks.TransferableBlock; -import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; +import org.apache.pinot.query.runtime.blocks.TransferableBlockTestUtils; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.MockitoAnnotations; @@ -66,13 +66,13 @@ public void testIntersectOperator() { }); Mockito.when(_leftOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(schema, new Object[]{1, "AA"}, new Object[]{2, "BB"}, new Object[]{3, "CC"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(schema, new Object[]{1, "AA"}, new Object[]{2, "BB"}, new Object[]{4, "DD"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); IntersectOperator intersectOperator = - new IntersectOperator(OperatorTestUtil.getDefaultContext(), ImmutableList.of(_leftOperator, _rightOperator), + new IntersectOperator(OperatorTestUtil.getTracingContext(), ImmutableList.of(_leftOperator, _rightOperator), schema); TransferableBlock result = intersectOperator.nextBlock(); @@ -95,14 +95,14 @@ public void testDedup() { Mockito.when(_leftOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(schema, new Object[]{1, "AA"}, new Object[]{2, "BB"}, new Object[]{3, "CC"}, new Object[]{1, "AA"}, new Object[]{2, "BB"}, new Object[]{3, "CC"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(schema, new Object[]{1, "AA"}, new Object[]{2, "BB"}, new Object[]{4, "DD"}, new Object[]{1, "AA"}, new Object[]{2, "BB"}, new Object[]{4, "DD"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); IntersectOperator intersectOperator = - new IntersectOperator(OperatorTestUtil.getDefaultContext(), ImmutableList.of(_leftOperator, _rightOperator), + new IntersectOperator(OperatorTestUtil.getTracingContext(), ImmutableList.of(_leftOperator, _rightOperator), schema); TransferableBlock result = intersectOperator.nextBlock(); diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/LeafStageTransferableBlockOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/LeafStageTransferableBlockOperatorTest.java index b4365b8f3d76..2bca79dcf836 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/LeafStageTransferableBlockOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/LeafStageTransferableBlockOperatorTest.java @@ -109,7 +109,7 @@ public void shouldReturnDataBlockThenMetadataBlock() { InstanceResponseBlock metadataBlock = new InstanceResponseBlock(new MetadataResultsBlock()); QueryExecutor queryExecutor = mockQueryExecutor(dataBlocks, metadataBlock); LeafStageTransferableBlockOperator operator = - new LeafStageTransferableBlockOperator(OperatorTestUtil.getDefaultContext(), mockQueryRequests(1), schema, + new LeafStageTransferableBlockOperator(OperatorTestUtil.getTracingContext(), mockQueryRequests(1), schema, queryExecutor, _executorService); _operatorRef.set(operator); @@ -140,7 +140,7 @@ public void shouldHandleDesiredDataSchemaConversionCorrectly() { InstanceResponseBlock metadataBlock = new InstanceResponseBlock(new MetadataResultsBlock()); QueryExecutor queryExecutor = mockQueryExecutor(dataBlocks, metadataBlock); LeafStageTransferableBlockOperator operator = - new LeafStageTransferableBlockOperator(OperatorTestUtil.getDefaultContext(), mockQueryRequests(1), + new LeafStageTransferableBlockOperator(OperatorTestUtil.getTracingContext(), mockQueryRequests(1), desiredSchema, queryExecutor, _executorService); _operatorRef.set(operator); @@ -167,7 +167,7 @@ public void shouldReturnMultipleDataBlockThenMetadataBlock() { InstanceResponseBlock metadataBlock = new InstanceResponseBlock(new MetadataResultsBlock()); QueryExecutor queryExecutor = mockQueryExecutor(dataBlocks, metadataBlock); LeafStageTransferableBlockOperator operator = - new LeafStageTransferableBlockOperator(OperatorTestUtil.getDefaultContext(), mockQueryRequests(1), schema, + new LeafStageTransferableBlockOperator(OperatorTestUtil.getTracingContext(), mockQueryRequests(1), schema, queryExecutor, _executorService); _operatorRef.set(operator); @@ -198,7 +198,7 @@ public void shouldHandleMultipleRequests() { InstanceResponseBlock metadataBlock = new InstanceResponseBlock(new MetadataResultsBlock()); QueryExecutor queryExecutor = mockQueryExecutor(dataBlocks, metadataBlock); LeafStageTransferableBlockOperator operator = - new LeafStageTransferableBlockOperator(OperatorTestUtil.getDefaultContext(), mockQueryRequests(2), schema, + new LeafStageTransferableBlockOperator(OperatorTestUtil.getTracingContext(), mockQueryRequests(2), schema, queryExecutor, _executorService); _operatorRef.set(operator); @@ -224,7 +224,7 @@ public void shouldGetErrorBlockWhenInstanceResponseContainsError() { errorBlock.addException(QueryException.QUERY_EXECUTION_ERROR.getErrorCode(), "foobar"); QueryExecutor queryExecutor = mockQueryExecutor(dataBlocks, errorBlock); LeafStageTransferableBlockOperator operator = - new LeafStageTransferableBlockOperator(OperatorTestUtil.getDefaultContext(), mockQueryRequests(1), schema, + new LeafStageTransferableBlockOperator(OperatorTestUtil.getTracingContext(), mockQueryRequests(1), schema, queryExecutor, _executorService); _operatorRef.set(operator); @@ -252,7 +252,7 @@ public void shouldNotErrorOutWhenIncorrectDataSchemaProvidedWithEmptyRowsSelecti new InstanceResponseBlock(new SelectionResultsBlock(resultSchema, Collections.emptyList(), queryContext)); QueryExecutor queryExecutor = mockQueryExecutor(dataBlocks, emptySelectionResponseBlock); LeafStageTransferableBlockOperator operator = - new LeafStageTransferableBlockOperator(OperatorTestUtil.getDefaultContext(), mockQueryRequests(1), + new LeafStageTransferableBlockOperator(OperatorTestUtil.getTracingContext(), mockQueryRequests(1), desiredSchema, queryExecutor, _executorService); _operatorRef.set(operator); diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/LiteralValueOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/LiteralValueOperatorTest.java index 959038c67716..bce3ba4e47e9 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/LiteralValueOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/LiteralValueOperatorTest.java @@ -63,7 +63,7 @@ public void shouldReturnLiteralBlock() { new RexExpression.Literal(ColumnDataType.INT, 1)), ImmutableList.of(new RexExpression.Literal(ColumnDataType.STRING, ""), new RexExpression.Literal(ColumnDataType.INT, 2))); - LiteralValueOperator operator = new LiteralValueOperator(OperatorTestUtil.getDefaultContext(), schema, literals); + LiteralValueOperator operator = new LiteralValueOperator(OperatorTestUtil.getTracingContext(), schema, literals); // When: TransferableBlock transferableBlock = operator.nextBlock(); @@ -79,7 +79,7 @@ public void shouldHandleEmptyLiteralRows() { // Given: DataSchema schema = new DataSchema(new String[]{}, new ColumnDataType[]{}); List> literals = ImmutableList.of(ImmutableList.of()); - LiteralValueOperator operator = new LiteralValueOperator(OperatorTestUtil.getDefaultContext(), schema, literals); + LiteralValueOperator operator = new LiteralValueOperator(OperatorTestUtil.getTracingContext(), schema, literals); // When: TransferableBlock transferableBlock = operator.nextBlock(); diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperatorTest.java index 2868f6526cb2..060ef3de7ed4 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperatorTest.java @@ -35,6 +35,7 @@ import org.apache.pinot.query.routing.StageMetadata; import org.apache.pinot.query.routing.WorkerMetadata; import org.apache.pinot.query.runtime.blocks.TransferableBlock; +import org.apache.pinot.query.runtime.blocks.TransferableBlockTestUtils; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.mockito.Mock; @@ -120,7 +121,7 @@ public void shouldTimeout() @Test public void shouldReceiveEosDirectlyFromSender() { when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_1))).thenReturn(_mailbox1); - when(_mailbox1.poll()).thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + when(_mailbox1.poll()).thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); OpChainExecutionContext context = OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadata1); @@ -134,7 +135,7 @@ public void shouldReceiveSingletonMailbox() { when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_1))).thenReturn(_mailbox1); Object[] row = new Object[]{1, 1}; when(_mailbox1.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row), - TransferableBlockUtils.getEndOfStreamTransferableBlock()); + TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); OpChainExecutionContext context = OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadata1); @@ -165,11 +166,11 @@ public void shouldReceiveSingletonErrorMailbox() { @Test public void shouldReceiveMailboxFromTwoServersOneNull() { when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_1))).thenReturn(_mailbox1); - when(_mailbox1.poll()).thenReturn(null, TransferableBlockUtils.getEndOfStreamTransferableBlock()); + when(_mailbox1.poll()).thenReturn(null, TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_2))).thenReturn(_mailbox2); Object[] row = new Object[]{1, 1}; when(_mailbox2.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row), - TransferableBlockUtils.getEndOfStreamTransferableBlock()); + TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); OpChainExecutionContext context = OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadataBoth); @@ -189,10 +190,10 @@ public void shouldReceiveMailboxFromTwoServers() { Object[] row3 = new Object[]{3, 3}; when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_1))).thenReturn(_mailbox1); when(_mailbox1.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row1), - OperatorTestUtil.block(DATA_SCHEMA, row3), TransferableBlockUtils.getEndOfStreamTransferableBlock()); + OperatorTestUtil.block(DATA_SCHEMA, row3), TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_2))).thenReturn(_mailbox2); when(_mailbox2.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row2), - TransferableBlockUtils.getEndOfStreamTransferableBlock()); + TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); OpChainExecutionContext context = OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadataBoth); @@ -217,7 +218,7 @@ public void shouldGetReceptionReceiveErrorMailbox() { when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_2))).thenReturn(_mailbox2); Object[] row = new Object[]{3, 3}; when(_mailbox2.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row), - TransferableBlockUtils.getEndOfStreamTransferableBlock()); + TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); OpChainExecutionContext context = OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadataBoth); @@ -236,10 +237,10 @@ public void shouldEarlyTerminateMailboxesWhenIndicated() { Object[] row3 = new Object[]{3, 3}; when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_1))).thenReturn(_mailbox1); when(_mailbox1.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row1), - OperatorTestUtil.block(DATA_SCHEMA, row3), TransferableBlockUtils.getEndOfStreamTransferableBlock()); + OperatorTestUtil.block(DATA_SCHEMA, row3), TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_2))).thenReturn(_mailbox2); when(_mailbox2.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row2), - TransferableBlockUtils.getEndOfStreamTransferableBlock()); + TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); OpChainExecutionContext context = OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadataBoth); diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxSendOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxSendOperatorTest.java index 92f7b2ce7ab3..e28d1dc477ce 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxSendOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxSendOperatorTest.java @@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import java.util.List; -import java.util.Map; import java.util.concurrent.TimeoutException; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.query.mailbox.MailboxService; @@ -30,6 +29,7 @@ import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.operator.exchange.BlockExchange; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.mockito.ArgumentCaptor; import org.mockito.Mock; @@ -41,7 +41,7 @@ import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.*; import static org.mockito.MockitoAnnotations.openMocks; -import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertSame; import static org.testng.Assert.assertTrue; @@ -124,7 +124,8 @@ public void shouldNotSendErrorBlockWhenTimedOut() public void shouldSendEosBlock() throws Exception { // Given: - TransferableBlock eosBlock = TransferableBlockUtils.getEndOfStreamTransferableBlock(); + TransferableBlock eosBlock = TransferableBlockUtils.getEndOfStreamTransferableBlock( + MultiStageQueryStats.emptyStats(SENDER_STAGE_ID)); when(_sourceOperator.nextBlock()).thenReturn(eosBlock); // When: @@ -145,7 +146,8 @@ public void shouldSendDataBlock() OperatorTestUtil.block(new DataSchema(new String[]{}, new DataSchema.ColumnDataType[]{})); TransferableBlock dataBlock2 = OperatorTestUtil.block(new DataSchema(new String[]{}, new DataSchema.ColumnDataType[]{})); - TransferableBlock eosBlock = TransferableBlockUtils.getEndOfStreamTransferableBlock(); + TransferableBlock eosBlock = TransferableBlockUtils.getEndOfStreamTransferableBlock( + MultiStageQueryStats.emptyStats(SENDER_STAGE_ID)); when(_sourceOperator.nextBlock()).thenReturn(dataBlock1, dataBlock2, eosBlock); // When: @@ -172,9 +174,10 @@ public void shouldSendDataBlock() assertTrue(blocks.get(2).isSuccessfulEndOfStreamBlock(), "expected to send EOS block to exchange on third call"); // EOS block should contain statistics - Map resultMetadata = blocks.get(2).getResultMetadata(); - assertEquals(resultMetadata.size(), 1); - assertTrue(resultMetadata.containsKey(mailboxSendOperator.getOperatorId())); + MultiStageQueryStats resultMetadata = blocks.get(2).getQueryStats(); + MultiStageQueryStats.StageStats stageStats = resultMetadata.getCurrentStats(); + assertNotNull(stageStats, "expected to have stats for sender stage"); + assertNotNull(stageStats.getOperatorStats(0)); } @Test @@ -200,6 +203,6 @@ private MailboxSendOperator getMailboxSendOperator() { OpChainExecutionContext context = new OpChainExecutionContext(_mailboxService, 123L, Long.MAX_VALUE, ImmutableMap.of(), stageMetadata, workerMetadata, null); - return new MailboxSendOperator(context, _sourceOperator, _exchange, null, null, false); + return new MailboxSendOperator(context, _sourceOperator, statMap -> _exchange, null, null, false); } } diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MinusOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MinusOperatorTest.java index 53e81612ad66..81f4b136993f 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MinusOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MinusOperatorTest.java @@ -25,7 +25,7 @@ import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.query.routing.VirtualServerAddress; import org.apache.pinot.query.runtime.blocks.TransferableBlock; -import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; +import org.apache.pinot.query.runtime.blocks.TransferableBlockTestUtils; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.MockitoAnnotations; @@ -67,13 +67,13 @@ public void testExceptOperator() { Mockito.when(_leftOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(schema, new Object[]{1, "AA"}, new Object[]{2, "BB"}, new Object[]{3, "CC"}, new Object[]{4, "DD"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(schema, new Object[]{1, "AA"}, new Object[]{2, "BB"}, new Object[]{5, "EE"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); MinusOperator minusOperator = - new MinusOperator(OperatorTestUtil.getDefaultContext(), ImmutableList.of(_leftOperator, _rightOperator), + new MinusOperator(OperatorTestUtil.getTracingContext(), ImmutableList.of(_leftOperator, _rightOperator), schema); TransferableBlock result = minusOperator.nextBlock(); @@ -97,14 +97,14 @@ public void testDedup() { .thenReturn(OperatorTestUtil.block(schema, new Object[]{1, "AA"}, new Object[]{2, "BB"}, new Object[]{3, "CC"}, new Object[]{4, "DD"}, new Object[]{1, "AA"}, new Object[]{2, "BB"}, new Object[]{3, "CC"}, new Object[]{4, "DD"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(schema, new Object[]{1, "AA"}, new Object[]{2, "BB"}, new Object[]{5, "EE"}, new Object[]{1, "AA"}, new Object[]{2, "BB"}, new Object[]{5, "EE"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); MinusOperator minusOperator = - new MinusOperator(OperatorTestUtil.getDefaultContext(), ImmutableList.of(_leftOperator, _rightOperator), + new MinusOperator(OperatorTestUtil.getTracingContext(), ImmutableList.of(_leftOperator, _rightOperator), schema); TransferableBlock result = minusOperator.nextBlock(); diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OpChainTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OpChainTest.java index f4cd38a42ea9..beb78740d805 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OpChainTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OpChainTest.java @@ -22,15 +22,14 @@ import com.google.common.collect.ImmutableMap; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.Stack; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicReference; import javax.annotation.Nullable; -import org.apache.calcite.rel.RelDistribution; -import org.apache.pinot.common.datatable.DataTable; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; import org.apache.pinot.core.operator.blocks.InstanceResponseBlock; @@ -50,12 +49,15 @@ import org.apache.pinot.query.routing.StageMetadata; import org.apache.pinot.query.routing.WorkerMetadata; import org.apache.pinot.query.runtime.blocks.TransferableBlock; +import org.apache.pinot.query.runtime.blocks.TransferableBlockTestUtils; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.operator.exchange.BlockExchange; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; -import org.apache.pinot.spi.utils.CommonConstants; import org.mockito.Mock; import org.mockito.MockitoAnnotations; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.testng.annotations.AfterClass; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -65,9 +67,7 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertTrue; -import static org.testng.Assert.fail; +import static org.testng.Assert.*; public class OpChainTest { @@ -111,7 +111,7 @@ public void setUpMethod() { }).when(_exchange).send(any(TransferableBlock.class)); when(_mailbox2.poll()).then(x -> { if (_blockList.isEmpty()) { - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + return TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0); } return _blockList.remove(0); }); @@ -132,134 +132,42 @@ public void tearDown() { _executor.shutdown(); } - @Test - public void testExecutionTimerStats() { - when(_sourceOperator.nextBlock()).then(x -> { - Thread.sleep(100); - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); - }); - OpChain opChain = new OpChain(OperatorTestUtil.getDefaultContext(), _sourceOperator); - opChain.getStats().executing(); - opChain.getRoot().nextBlock(); - opChain.getStats().queued(); - assertTrue(opChain.getStats().getExecutionTime() >= 100); - - when(_sourceOperator.nextBlock()).then(x -> { - Thread.sleep(20); - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); - }); - opChain = new OpChain(OperatorTestUtil.getDefaultContext(), _sourceOperator); - opChain.getStats().executing(); - opChain.getRoot().nextBlock(); - opChain.getStats().queued(); - assertTrue(opChain.getStats().getExecutionTime() >= 20); - assertTrue(opChain.getStats().getExecutionTime() < 100); - } - @Test public void testStatsCollectionTracingEnabled() { - OpChainExecutionContext context = OperatorTestUtil.getDefaultContext(); + OpChainExecutionContext context = OperatorTestUtil.getTracingContext(); DummyMultiStageOperator dummyMultiStageOperator = new DummyMultiStageOperator(context); OpChain opChain = new OpChain(context, dummyMultiStageOperator); - opChain.getStats().executing(); - opChain.getRoot().nextBlock(); - opChain.getStats().queued(); - - assertTrue(opChain.getStats().getExecutionTime() >= 1000); - assertEquals(opChain.getStats().getOperatorStatsMap().size(), 1); - assertTrue(opChain.getStats().getOperatorStatsMap().containsKey(dummyMultiStageOperator.getOperatorId())); + TransferableBlock eosBlock = drainOpChain(opChain); - Map executionStats = - opChain.getStats().getOperatorStatsMap().get(dummyMultiStageOperator.getOperatorId()).getExecutionStats(); + assertTrue(eosBlock.isSuccessfulEndOfStreamBlock(), "Expected end of stream block to be successful"); + MultiStageQueryStats queryStats = eosBlock.getQueryStats(); + assertNotNull(queryStats, "Expected query stats to be non-null"); - long time = Long.parseLong(executionStats.get(DataTable.MetadataKey.OPERATOR_EXECUTION_TIME_MS.getName())); - assertTrue(time >= 1000 && time <= 2000, - "Expected " + DataTable.MetadataKey.OPERATOR_EXECUTION_TIME_MS + " to be in [1000, 2000] but found " + time); + @SuppressWarnings("unchecked") + StatMap lastOperatorStats = + (StatMap) queryStats.getCurrentStats().getLastOperatorStats(); + assertNotEquals(lastOperatorStats.getLong(LiteralValueOperator.StatKey.EXECUTION_TIME_MS), 0L, + "Expected execution time to be non-zero"); } @Test public void testStatsCollectionTracingDisabled() { - OpChainExecutionContext context = OperatorTestUtil.getDefaultContextWithTracingDisabled(); + OpChainExecutionContext context = OperatorTestUtil.getNoTracingContext(); DummyMultiStageOperator dummyMultiStageOperator = new DummyMultiStageOperator(context); OpChain opChain = new OpChain(context, dummyMultiStageOperator); - opChain.getStats().executing(); - opChain.getRoot().nextBlock(); - opChain.getStats().queued(); - - assertTrue(opChain.getStats().getExecutionTime() >= 1000); - assertEquals(opChain.getStats().getOperatorStatsMap().size(), 0); - } - - @Test - public void testStatsCollectionTracingEnabledMultipleOperators() { - long dummyOperatorWaitTime = 1000L; - - OpChainExecutionContext context = new OpChainExecutionContext(_mailboxService1, 123L, Long.MAX_VALUE, - ImmutableMap.of(CommonConstants.Broker.Request.TRACE, "true"), _stageMetadata, _workerMetadata, null); - Stack operators = getFullOpChain(context, dummyOperatorWaitTime); - - OpChain opChain = new OpChain(context, operators.peek()); - opChain.getStats().executing(); - while (!opChain.getRoot().nextBlock().isEndOfStreamBlock()) { - // Drain the opchain - } - opChain.getStats().queued(); - - OpChainExecutionContext secondStageContext = new OpChainExecutionContext(_mailboxService2, 123L, Long.MAX_VALUE, - ImmutableMap.of(CommonConstants.Broker.Request.TRACE, "true"), _stageMetadata, _workerMetadata, null); - MailboxReceiveOperator secondStageReceiveOp = - new MailboxReceiveOperator(secondStageContext, RelDistribution.Type.BROADCAST_DISTRIBUTED, 1); - - assertTrue(opChain.getStats().getExecutionTime() >= dummyOperatorWaitTime); - int numOperators = operators.size(); - assertEquals(opChain.getStats().getOperatorStatsMap().size(), numOperators); - while (!operators.isEmpty()) { - assertTrue(opChain.getStats().getOperatorStatsMap().containsKey(operators.pop().getOperatorId())); - } - - while (!secondStageReceiveOp.nextBlock().isEndOfStreamBlock()) { - // Drain the mailbox - } - assertEquals(secondStageContext.getStats().getOperatorStatsMap().size(), numOperators + 1); - } + TransferableBlock eosBlock = drainOpChain(opChain); - @Test - public void testStatsCollectionTracingDisableMultipleOperators() { - long dummyOperatorWaitTime = 1000L; - - OpChainExecutionContext context = - new OpChainExecutionContext(_mailboxService1, 123L, Long.MAX_VALUE, ImmutableMap.of(), _stageMetadata, - _workerMetadata, null); - Stack operators = getFullOpChain(context, dummyOperatorWaitTime); - - OpChain opChain = new OpChain(context, operators.peek()); - opChain.getStats().executing(); - opChain.getRoot().nextBlock(); - opChain.getStats().queued(); - - OpChainExecutionContext secondStageContext = - new OpChainExecutionContext(_mailboxService2, 123L, Long.MAX_VALUE, ImmutableMap.of(), _stageMetadata, - _workerMetadata, null); - MailboxReceiveOperator secondStageReceiveOp = - new MailboxReceiveOperator(secondStageContext, RelDistribution.Type.BROADCAST_DISTRIBUTED, 1); - - assertTrue(opChain.getStats().getExecutionTime() >= dummyOperatorWaitTime); - assertEquals(opChain.getStats().getOperatorStatsMap().size(), 2); - assertTrue(opChain.getStats().getOperatorStatsMap().containsKey(operators.pop().getOperatorId())); - - while (!secondStageReceiveOp.nextBlock().isEndOfStreamBlock()) { - // Drain the mailbox - } + assertTrue(eosBlock.isSuccessfulEndOfStreamBlock(), "Expected end of stream block to be successful"); + MultiStageQueryStats queryStats = eosBlock.getQueryStats(); + assertNotNull(queryStats, "Expected query stats to be non-null"); - while (!operators.isEmpty()) { - MultiStageOperator operator = operators.pop(); - if (operator.toExplainString().contains("SEND") || operator.toExplainString().contains("LEAF")) { - assertTrue(opChain.getStats().getOperatorStatsMap().containsKey(operator.getOperatorId())); - } - } - assertEquals(secondStageContext.getStats().getOperatorStatsMap().size(), 2); + @SuppressWarnings("unchecked") + StatMap lastOperatorStats = + (StatMap) queryStats.getCurrentStats().getLastOperatorStats(); + assertNotEquals(lastOperatorStats.getLong(LiteralValueOperator.StatKey.EXECUTION_TIME_MS), 0L, + "Expected execution time to be collected"); } private Stack getFullOpChain(OpChainExecutionContext context, long waitTimeInMillis) { @@ -268,7 +176,7 @@ private Stack getFullOpChain(OpChainExecutionContext context //Mailbox Receive Operator try { when(_mailbox1.poll()).thenReturn(OperatorTestUtil.block(upStreamSchema, new Object[]{1}), - TransferableBlockUtils.getEndOfStreamTransferableBlock()); + TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); } catch (Exception e) { fail("Exception while mocking mailbox receive: " + e.getMessage()); } @@ -297,7 +205,7 @@ private Stack getFullOpChain(OpChainExecutionContext context //Mailbox Send operator MailboxSendOperator sendOperator = - new MailboxSendOperator(context, dummyWaitOperator, _exchange, null, null, false); + new MailboxSendOperator(context, dummyWaitOperator, ignore -> _exchange, null, null, false); operators.push(leafOp); operators.push(transformOp); @@ -319,12 +227,33 @@ private QueryExecutor mockQueryExecutor(List dataBlocks, Insta return queryExecutor; } + private TransferableBlock drainOpChain(OpChain opChain) { + TransferableBlock resultBlock = opChain.getRoot().nextBlock(); + while (!resultBlock.isEndOfStreamBlock()) { + resultBlock = opChain.getRoot().nextBlock(); + } + return resultBlock; + } + static class DummyMultiStageOperator extends MultiStageOperator { + private static final Logger LOGGER = LoggerFactory.getLogger(DummyMultiStageOperator.class); + private final StatMap _statMap = new StatMap<>(LiteralValueOperator.StatKey.class); public DummyMultiStageOperator(OpChainExecutionContext context) { super(context); } + @Override + public void registerExecution(long time, int numRows) { + _statMap.merge(LiteralValueOperator.StatKey.EXECUTION_TIME_MS, time); + _statMap.merge(LiteralValueOperator.StatKey.EMITTED_ROWS, numRows); + } + + @Override + protected Logger logger() { + return LOGGER; + } + @Override protected TransferableBlock getNextBlock() { try { @@ -332,7 +261,17 @@ protected TransferableBlock getNextBlock() { } catch (InterruptedException e) { // IGNORE } - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + return TransferableBlockUtils.getEndOfStreamTransferableBlock(MultiStageQueryStats.createLiteral(0, _statMap)); + } + + @Override + public Type getOperatorType() { + return Type.LITERAL; + } + + @Override + public List getChildOperators() { + return Collections.emptyList(); } @Nullable @@ -343,8 +282,10 @@ public String toExplainString() { } static class DummyMultiStageCallableOperator extends MultiStageOperator { + private static final Logger LOGGER = LoggerFactory.getLogger(DummyMultiStageCallableOperator.class); private final MultiStageOperator _upstream; private final long _sleepTimeInMillis; + private final StatMap _statMap = new StatMap<>(TransformOperator.StatKey.class); public DummyMultiStageCallableOperator(OpChainExecutionContext context, MultiStageOperator upstream, long sleepTimeInMillis) { @@ -353,6 +294,17 @@ public DummyMultiStageCallableOperator(OpChainExecutionContext context, MultiSta _sleepTimeInMillis = sleepTimeInMillis; } + @Override + public void registerExecution(long time, int numRows) { + _statMap.merge(TransformOperator.StatKey.EXECUTION_TIME_MS, time); + _statMap.merge(TransformOperator.StatKey.EMITTED_ROWS, numRows); + } + + @Override + protected Logger logger() { + return LOGGER; + } + @Override public List getChildOperators() { return ImmutableList.of(_upstream); @@ -360,13 +312,26 @@ public List getChildOperators() { @Override protected TransferableBlock getNextBlock() { + TransferableBlock block; + try { Thread.sleep(_sleepTimeInMillis); - _upstream.nextBlock(); + do { + block = _upstream.nextBlock(); + } while (block.isEndOfStreamBlock()); + + MultiStageQueryStats queryStats = block.getQueryStats(); + assert queryStats != null; + queryStats.getCurrentStats().addLastOperator(getOperatorType(), _statMap); + return TransferableBlockUtils.getEndOfStreamTransferableBlock(queryStats); } catch (InterruptedException e) { - // IGNORE + throw new RuntimeException(e); } - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + } + + @Override + public Type getOperatorType() { + return Type.TRANSFORM; } @Override diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTestUtil.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTestUtil.java index 5280724541c4..da4537cb1970 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTestUtil.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTestUtil.java @@ -24,16 +24,19 @@ import java.util.List; import java.util.Map; import org.apache.pinot.common.datablock.DataBlock; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.query.mailbox.MailboxService; import org.apache.pinot.query.routing.StageMetadata; -import org.apache.pinot.query.routing.VirtualServerAddress; +import org.apache.pinot.query.routing.StagePlan; import org.apache.pinot.query.routing.WorkerMetadata; import org.apache.pinot.query.runtime.blocks.TransferableBlock; -import org.apache.pinot.query.runtime.operator.utils.OperatorUtils; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; +import org.apache.pinot.query.runtime.plan.server.ServerPlanRequestContext; import org.apache.pinot.query.testutils.MockDataBlockOperatorFactory; import org.apache.pinot.spi.utils.CommonConstants; +import org.testng.Assert; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -52,10 +55,8 @@ public class OperatorTestUtil { public static final String OP_1 = "op1"; public static final String OP_2 = "op2"; - public static Map getDummyStats(long requestId, int stageId, VirtualServerAddress serverAddress) { - OperatorStats operatorStats = new OperatorStats(requestId, stageId, serverAddress); - String statsId = new OpChainId(requestId, serverAddress.workerId(), stageId).toString(); - return OperatorUtils.getMetadataFromOperatorStats(ImmutableMap.of(statsId, operatorStats)); + public static MultiStageQueryStats getDummyStats(int stageId) { + return MultiStageQueryStats.createLeaf(stageId, new StatMap<>(LeafStageTransferableBlockOperator.StatKey.class)); } static { @@ -85,20 +86,44 @@ public static OpChainExecutionContext getOpChainContext(MailboxService mailboxSe stageMetadata.getWorkerMetadataList().get(0), null); } - public static OpChainExecutionContext getDefaultContext() { - return getDefaultContext(ImmutableMap.of(CommonConstants.Broker.Request.TRACE, "true")); + public static OpChainExecutionContext getTracingContext() { + return getTracingContext(ImmutableMap.of(CommonConstants.Broker.Request.TRACE, "true")); } - public static OpChainExecutionContext getDefaultContextWithTracingDisabled() { - return getDefaultContext(ImmutableMap.of()); + public static OpChainExecutionContext getNoTracingContext() { + return getTracingContext(ImmutableMap.of()); } - private static OpChainExecutionContext getDefaultContext(Map opChainMetadata) { + private static OpChainExecutionContext getTracingContext(Map opChainMetadata) { MailboxService mailboxService = mock(MailboxService.class); when(mailboxService.getHostname()).thenReturn("localhost"); when(mailboxService.getPort()).thenReturn(1234); WorkerMetadata workerMetadata = new WorkerMetadata(0, ImmutableMap.of(), ImmutableMap.of()); - return new OpChainExecutionContext(mailboxService, 123L, Long.MAX_VALUE, opChainMetadata, - new StageMetadata(0, ImmutableList.of(workerMetadata), ImmutableMap.of()), workerMetadata, null); + StageMetadata stageMetadata = new StageMetadata(0, ImmutableList.of(workerMetadata), ImmutableMap.of()); + OpChainExecutionContext opChainExecutionContext = new OpChainExecutionContext(mailboxService, 123L, Long.MAX_VALUE, + opChainMetadata, stageMetadata, workerMetadata, null); + + StagePlan stagePlan = new StagePlan(null, stageMetadata); + + opChainExecutionContext.setLeafStageContext( + new ServerPlanRequestContext(stagePlan, null, null, null)); + return opChainExecutionContext; + } + + /** + * Verifies that the given block is a successful end of stream block, verifies that its stats are of the same family + * as the given keyClass and returns the {@link StatMap} cast to the that key class. + */ + public static & StatMap.Key> StatMap getStatMap(Class keyClass, TransferableBlock block) { + Assert.assertTrue(block.isSuccessfulEndOfStreamBlock(), "Expected EOS block but found " + block.getClass()); + MultiStageQueryStats queryStats = block.getQueryStats(); + Assert.assertNotNull(queryStats, "Stats holder should not be null"); + MultiStageQueryStats.StageStats stageStats = queryStats.getCurrentStats(); + Assert.assertEquals(stageStats.getLastOperatorStats().getKeyClass(), keyClass, + "Key class should be " + keyClass.getName()); + + @SuppressWarnings("unchecked") + StatMap lastOperatorStats = (StatMap) stageStats.getLastOperatorStats(); + return lastOperatorStats; } } diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/SortOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/SortOperatorTest.java index 03fc1755ba99..7dd776368ec3 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/SortOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/SortOperatorTest.java @@ -30,6 +30,7 @@ import org.apache.pinot.query.planner.logical.RexExpression; import org.apache.pinot.query.routing.VirtualServerAddress; import org.apache.pinot.query.runtime.blocks.TransferableBlock; +import org.apache.pinot.query.runtime.blocks.TransferableBlockTestUtils; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.mockito.Mock; import org.mockito.Mockito; @@ -73,7 +74,7 @@ public void shouldHandleUpstreamErrorBlock() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 0, schema, false); Mockito.when(_input.nextBlock()) @@ -94,10 +95,10 @@ public void shouldCreateEmptyBlockOnUpstreamEOS() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 0, schema, false); - Mockito.when(_input.nextBlock()).thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + Mockito.when(_input.nextBlock()).thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); @@ -114,11 +115,11 @@ public void shouldConsumeAndSortInputOneBlockWithTwoRows() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 0, schema, false); Mockito.when(_input.nextBlock()).thenReturn(block(schema, new Object[]{2}, new Object[]{1})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -139,13 +140,13 @@ public void shouldConsumeAndSkipSortInputOneBlockWithTwoRowsInputSorted() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 0, schema, true); // Purposefully setting input as unsorted order for validation but 'isInputSorted' should only be true if actually // sorted Mockito.when(_input.nextBlock()).thenReturn(block(schema, new Object[]{2}, new Object[]{1})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -166,11 +167,11 @@ public void shouldConsumeAndSortOnNonZeroIdxCollation() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"ignored", "sort"}, new DataSchema.ColumnDataType[]{INT, INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 0, schema, false); Mockito.when(_input.nextBlock()).thenReturn(block(schema, new Object[]{1, 2}, new Object[]{2, 1})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -191,11 +192,11 @@ public void shouldConsumeAndSortInputOneBlockWithTwoRowsNonNumeric() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{STRING}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 0, schema, false); Mockito.when(_input.nextBlock()).thenReturn(block(schema, new Object[]{"b"}, new Object[]{"a"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -216,11 +217,11 @@ public void shouldConsumeAndSortDescending() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 0, schema, false); Mockito.when(_input.nextBlock()).thenReturn(block(schema, new Object[]{2}, new Object[]{1})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -241,11 +242,11 @@ public void shouldOffsetSortInputOneBlockWithThreeRows() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 1, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 1, schema, false); Mockito.when(_input.nextBlock()).thenReturn(block(schema, new Object[]{2}, new Object[]{1}, new Object[]{3})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -266,12 +267,12 @@ public void shouldOffsetSortInputOneBlockWithThreeRowsInputSorted() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 1, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 1, schema, true); // Set input rows as sorted since input is expected to be sorted Mockito.when(_input.nextBlock()).thenReturn(block(schema, new Object[]{1}, new Object[]{2}, new Object[]{3})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -292,11 +293,11 @@ public void shouldOffsetLimitSortInputOneBlockWithThreeRows() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 1, 1, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 1, 1, schema, false); Mockito.when(_input.nextBlock()).thenReturn(block(schema, new Object[]{2}, new Object[]{1}, new Object[]{3})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -316,12 +317,12 @@ public void shouldOffsetLimitSortInputOneBlockWithThreeRowsInputSorted() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 1, 1, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 1, 1, schema, true); // Set input rows as sorted since input is expected to be sorted Mockito.when(_input.nextBlock()).thenReturn(block(schema, new Object[]{1}, new Object[]{2}, new Object[]{3})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -341,11 +342,11 @@ public void shouldRespectDefaultLimit() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 0, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 0, 0, schema, false, 10, 1); Mockito.when(_input.nextBlock()).thenReturn(block(schema, new Object[]{2}, new Object[]{1}, new Object[]{3})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -365,11 +366,11 @@ public void shouldFetchAllWithNegativeFetch() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, -1, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, -1, 0, schema, false); Mockito.when(_input.nextBlock()).thenReturn(block(schema, new Object[]{2}, new Object[]{1}, new Object[]{3})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -388,12 +389,12 @@ public void shouldConsumeAndSortTwoInputBlocksWithOneRowEach() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 0, schema, false); Mockito.when(_input.nextBlock()).thenReturn(block(schema, new Object[]{2})) .thenReturn(block(schema, new Object[]{1})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -414,13 +415,13 @@ public void shouldConsumeAndSortTwoInputBlocksWithOneRowEachInputSorted() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 0, schema, true); // Set input rows as sorted since input is expected to be sorted Mockito.when(_input.nextBlock()).thenReturn(block(schema, new Object[]{1})) .thenReturn(block(schema, new Object[]{2})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -441,18 +442,19 @@ public void shouldBreakTiesUsingSecondCollationKey() { List nullDirections = ImmutableList.of(NullDirection.LAST, NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"first", "second"}, new DataSchema.ColumnDataType[]{INT, INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 0, schema, false); Mockito.when(_input.nextBlock()) .thenReturn(block(schema, new Object[]{1, 2}, new Object[]{1, 1}, new Object[]{1, 3})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct TransferableBlock block2 = op.nextBlock(); // eos // Then: + Assert.assertEquals(block.getType(), DataBlock.Type.ROW, "expected columnar block"); Assert.assertEquals(block.getNumRows(), 3); Assert.assertEquals(block.getContainer().get(0), new Object[]{1, 1}); Assert.assertEquals(block.getContainer().get(1), new Object[]{1, 2}); @@ -468,12 +470,12 @@ public void shouldBreakTiesUsingSecondCollationKeyWithDifferentDirection() { List nullDirections = ImmutableList.of(NullDirection.LAST, NullDirection.FIRST); DataSchema schema = new DataSchema(new String[]{"first", "second"}, new DataSchema.ColumnDataType[]{INT, INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 0, schema, false); Mockito.when(_input.nextBlock()) .thenReturn(block(schema, new Object[]{1, 2}, new Object[]{1, 1}, new Object[]{1, 3})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -495,12 +497,12 @@ public void shouldHandleNoOpUpstreamBlockWhileConstructing() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 0, schema, false); Mockito.when(_input.nextBlock()).thenReturn(block(schema, new Object[]{2})) .thenReturn(block(schema, new Object[]{1})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -521,13 +523,13 @@ public void shouldHandleNoOpUpstreamBlockWhileConstructingInputSorted() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 0, schema, true); // Set input rows as sorted since input is expected to be sorted Mockito.when(_input.nextBlock()).thenReturn(block(schema, new Object[]{1})) .thenReturn(block(schema, new Object[]{2})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -548,11 +550,11 @@ public void shouldHaveNullAtLast() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 0, schema, false); Mockito.when(_input.nextBlock()).thenReturn(block(schema, new Object[]{2}, new Object[]{1}, new Object[]{null})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -574,11 +576,11 @@ public void shouldHaveNullAtFirst() { List nullDirections = ImmutableList.of(NullDirection.FIRST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 0, schema, false); Mockito.when(_input.nextBlock()).thenReturn(block(schema, new Object[]{2}, new Object[]{1}, new Object[]{null})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -600,12 +602,12 @@ public void shouldHandleMultipleCollationKeysWithNulls() { List nullDirections = ImmutableList.of(NullDirection.FIRST, NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"first", "second"}, new DataSchema.ColumnDataType[]{INT, INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 0, schema, false); Mockito.when(_input.nextBlock()) .thenReturn(block(schema, new Object[]{1, 1}, new Object[]{1, null}, new Object[]{null, 1})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct @@ -627,13 +629,13 @@ public void shouldEarlyTerminateCorrectlyWithSignalingPropagateUpstream() { List nullDirections = ImmutableList.of(NullDirection.LAST); DataSchema schema = new DataSchema(new String[]{"sort"}, new DataSchema.ColumnDataType[]{INT}); SortOperator op = - new SortOperator(OperatorTestUtil.getDefaultContext(), _input, collation, directions, nullDirections, 10, 0, + new SortOperator(OperatorTestUtil.getTracingContext(), _input, collation, directions, nullDirections, 10, 0, schema, false); Mockito.when(_input.nextBlock()).thenReturn(block(schema, new Object[]{1}, new Object[]{2}, new Object[]{3}, new Object[]{4}, new Object[]{5}, new Object[]{6}, new Object[]{7}, new Object[]{8}, new Object[]{9}, new Object[]{10}, new Object[]{11}, new Object[]{12})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // When: TransferableBlock block = op.nextBlock(); // construct diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/SortedMailboxReceiveOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/SortedMailboxReceiveOperatorTest.java index 314081588f24..bee1ff07833b 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/SortedMailboxReceiveOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/SortedMailboxReceiveOperatorTest.java @@ -28,6 +28,7 @@ import org.apache.calcite.rel.RelDistribution; import org.apache.calcite.rel.RelFieldCollation.Direction; import org.apache.calcite.rel.RelFieldCollation.NullDirection; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.exception.QueryException; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.query.mailbox.MailboxService; @@ -40,6 +41,7 @@ import org.apache.pinot.query.routing.StageMetadata; import org.apache.pinot.query.routing.WorkerMetadata; import org.apache.pinot.query.runtime.blocks.TransferableBlock; +import org.apache.pinot.query.runtime.blocks.TransferableBlockTestUtils; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.mockito.Mock; @@ -94,6 +96,8 @@ public void setUpMethod() { _mocks = MockitoAnnotations.openMocks(this); when(_mailboxService.getHostname()).thenReturn("localhost"); when(_mailboxService.getPort()).thenReturn(1234); + when(_mailbox1.getStatMap()).thenReturn(new StatMap<>(ReceivingMailbox.StatKey.class)); + when(_mailbox2.getStatMap()).thenReturn(new StatMap<>(ReceivingMailbox.StatKey.class)); } @AfterMethod @@ -141,7 +145,7 @@ public void shouldTimeout() @Test public void shouldReceiveEosDirectlyFromSender() { when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_1))).thenReturn(_mailbox1); - when(_mailbox1.poll()).thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + when(_mailbox1.poll()).thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); OpChainExecutionContext context = OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadata1); try (SortedMailboxReceiveOperator receiveOp = new SortedMailboxReceiveOperator(context, @@ -156,7 +160,7 @@ public void shouldReceiveSingletonMailbox() { when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_1))).thenReturn(_mailbox1); Object[] row = new Object[]{1, 1}; when(_mailbox1.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row), - TransferableBlockUtils.getEndOfStreamTransferableBlock()); + TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); OpChainExecutionContext context = OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadata1); try (SortedMailboxReceiveOperator receiveOp = new SortedMailboxReceiveOperator(context, @@ -189,11 +193,11 @@ public void shouldReceiveSingletonErrorMailbox() { @Test public void shouldReceiveMailboxFromTwoServersOneNull() { when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_1))).thenReturn(_mailbox1); - when(_mailbox1.poll()).thenReturn(null, TransferableBlockUtils.getEndOfStreamTransferableBlock()); + when(_mailbox1.poll()).thenReturn(null, TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_2))).thenReturn(_mailbox2); Object[] row = new Object[]{1, 1}; when(_mailbox2.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row), - TransferableBlockUtils.getEndOfStreamTransferableBlock()); + TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); OpChainExecutionContext context = OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadataBoth); try (SortedMailboxReceiveOperator receiveOp = new SortedMailboxReceiveOperator(context, @@ -218,7 +222,7 @@ public void shouldGetReceptionReceiveErrorMailbox() { when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_2))).thenReturn(_mailbox2); Object[] row = new Object[]{3, 3}; when(_mailbox2.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row), - TransferableBlockUtils.getEndOfStreamTransferableBlock()); + TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); OpChainExecutionContext context = OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadataBoth); try (SortedMailboxReceiveOperator receiveOp = new SortedMailboxReceiveOperator(context, @@ -236,14 +240,14 @@ public void shouldReceiveMailboxFromTwoServersWithCollationKey() { Object[] row1 = new Object[]{3, 3}; Object[] row2 = new Object[]{1, 1}; when(_mailbox1.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row1), - OperatorTestUtil.block(DATA_SCHEMA, row2), TransferableBlockUtils.getEndOfStreamTransferableBlock()); + OperatorTestUtil.block(DATA_SCHEMA, row2), TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_2))).thenReturn(_mailbox2); Object[] row3 = new Object[]{4, 2}; Object[] row4 = new Object[]{2, 4}; Object[] row5 = new Object[]{-1, 95}; when(_mailbox2.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row3), OperatorTestUtil.block(DATA_SCHEMA, row4), OperatorTestUtil.block(DATA_SCHEMA, row5), - TransferableBlockUtils.getEndOfStreamTransferableBlock()); + TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); OpChainExecutionContext context = OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadataBoth); try (SortedMailboxReceiveOperator receiveOp = new SortedMailboxReceiveOperator(context, @@ -266,14 +270,14 @@ public void shouldReceiveMailboxFromTwoServersWithCollationKeyTwoColumns() { Object[] row1 = new Object[]{3, 3, "queen"}; Object[] row2 = new Object[]{1, 1, "pink floyd"}; when(_mailbox1.poll()).thenReturn(OperatorTestUtil.block(dataSchema, row1), - OperatorTestUtil.block(dataSchema, row2), TransferableBlockUtils.getEndOfStreamTransferableBlock()); + OperatorTestUtil.block(dataSchema, row2), TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_2))).thenReturn(_mailbox2); Object[] row3 = new Object[]{4, 2, "pink floyd"}; Object[] row4 = new Object[]{2, 4, "aerosmith"}; Object[] row5 = new Object[]{-1, 95, "foo fighters"}; when(_mailbox2.poll()).thenReturn(OperatorTestUtil.block(dataSchema, row3), OperatorTestUtil.block(dataSchema, row4), OperatorTestUtil.block(dataSchema, row5), - TransferableBlockUtils.getEndOfStreamTransferableBlock()); + TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); OpChainExecutionContext context = OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadataBoth); diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/TransformOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/TransformOperatorTest.java index 2c7883157029..123cda06a523 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/TransformOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/TransformOperatorTest.java @@ -69,7 +69,7 @@ public void shouldHandleRefTransform() { // Output column value RexExpression.InputRef ref0 = new RexExpression.InputRef(0); RexExpression.InputRef ref1 = new RexExpression.InputRef(1); - TransformOperator op = new TransformOperator(OperatorTestUtil.getDefaultContext(), _upstreamOp, resultSchema, + TransformOperator op = new TransformOperator(OperatorTestUtil.getTracingContext(), _upstreamOp, resultSchema, ImmutableList.of(ref0, ref1), upStreamSchema); TransferableBlock result = op.nextBlock(); @@ -93,7 +93,7 @@ public void shouldHandleLiteralTransform() { // Set up literal operands RexExpression.Literal boolLiteral = new RexExpression.Literal(ColumnDataType.BOOLEAN, 1); RexExpression.Literal strLiteral = new RexExpression.Literal(ColumnDataType.STRING, "str"); - TransformOperator op = new TransformOperator(OperatorTestUtil.getDefaultContext(), _upstreamOp, resultSchema, + TransformOperator op = new TransformOperator(OperatorTestUtil.getTracingContext(), _upstreamOp, resultSchema, ImmutableList.of(boolLiteral, strLiteral), upStreamSchema); TransferableBlock result = op.nextBlock(); // Literal operands should just output original literals. @@ -122,7 +122,7 @@ public void shouldHandlePlusMinusFuncTransform() { new RexExpression.FunctionCall(MINUS, ColumnDataType.DOUBLE, "minus", functionOperands); DataSchema resultSchema = new DataSchema(new String[]{"plusR", "minusR"}, new ColumnDataType[]{ColumnDataType.DOUBLE, ColumnDataType.DOUBLE}); - TransformOperator op = new TransformOperator(OperatorTestUtil.getDefaultContext(), _upstreamOp, resultSchema, + TransformOperator op = new TransformOperator(OperatorTestUtil.getTracingContext(), _upstreamOp, resultSchema, ImmutableList.of(plus01, minus01), upStreamSchema); TransferableBlock result = op.nextBlock(); Assert.assertTrue(!result.isErrorBlock()); @@ -150,7 +150,7 @@ public void shouldThrowOnTypeMismatchFuncTransform() { new RexExpression.FunctionCall(MINUS, ColumnDataType.DOUBLE, "minus", functionOperands); DataSchema resultSchema = new DataSchema(new String[]{"plusR", "minusR"}, new ColumnDataType[]{ColumnDataType.DOUBLE, ColumnDataType.DOUBLE}); - TransformOperator op = new TransformOperator(OperatorTestUtil.getDefaultContext(), _upstreamOp, resultSchema, + TransformOperator op = new TransformOperator(OperatorTestUtil.getTracingContext(), _upstreamOp, resultSchema, ImmutableList.of(plus01, minus01), upStreamSchema); TransferableBlock result = op.nextBlock(); @@ -169,7 +169,7 @@ public void shouldPropagateUpstreamError() { RexExpression.Literal strLiteral = new RexExpression.Literal(ColumnDataType.STRING, "str"); DataSchema resultSchema = new DataSchema(new String[]{"inCol", "strCol"}, new ColumnDataType[]{ColumnDataType.INT, ColumnDataType.STRING}); - TransformOperator op = new TransformOperator(OperatorTestUtil.getDefaultContext(), _upstreamOp, resultSchema, + TransformOperator op = new TransformOperator(OperatorTestUtil.getTracingContext(), _upstreamOp, resultSchema, ImmutableList.of(boolLiteral, strLiteral), upStreamSchema); TransferableBlock result = op.nextBlock(); Assert.assertTrue(result.isErrorBlock()); @@ -190,7 +190,7 @@ public void testNoopBlock() { RexExpression.Literal strLiteral = new RexExpression.Literal(ColumnDataType.STRING, "str"); DataSchema resultSchema = new DataSchema(new String[]{"boolCol", "strCol"}, new ColumnDataType[]{ColumnDataType.BOOLEAN, ColumnDataType.STRING}); - TransformOperator op = new TransformOperator(OperatorTestUtil.getDefaultContext(), _upstreamOp, resultSchema, + TransformOperator op = new TransformOperator(OperatorTestUtil.getTracingContext(), _upstreamOp, resultSchema, ImmutableList.of(boolLiteral, strLiteral), upStreamSchema); TransferableBlock result = op.nextBlock(); // First block has two rows @@ -219,7 +219,7 @@ public void testWrongNumTransform() { DataSchema upStreamSchema = new DataSchema(new String[]{"string1", "string2"}, new ColumnDataType[]{ ColumnDataType.STRING, ColumnDataType.STRING }); - new TransformOperator(OperatorTestUtil.getDefaultContext(), _upstreamOp, resultSchema, new ArrayList<>(), + new TransformOperator(OperatorTestUtil.getTracingContext(), _upstreamOp, resultSchema, new ArrayList<>(), upStreamSchema); } @@ -232,7 +232,7 @@ public void testMismatchedSchemaOperandSize() { ColumnDataType.STRING, ColumnDataType.STRING }); RexExpression.InputRef ref0 = new RexExpression.InputRef(0); - new TransformOperator(OperatorTestUtil.getDefaultContext(), _upstreamOp, resultSchema, ImmutableList.of(ref0), + new TransformOperator(OperatorTestUtil.getTracingContext(), _upstreamOp, resultSchema, ImmutableList.of(ref0), upStreamSchema); } }; diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/UnionOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/UnionOperatorTest.java index 16e8c4687484..6d805f81aff8 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/UnionOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/UnionOperatorTest.java @@ -25,7 +25,7 @@ import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.query.routing.VirtualServerAddress; import org.apache.pinot.query.runtime.blocks.TransferableBlock; -import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; +import org.apache.pinot.query.runtime.blocks.TransferableBlockTestUtils; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.MockitoAnnotations; @@ -66,13 +66,13 @@ public void testUnionOperator() { }); Mockito.when(_leftOperator.nextBlock()) .thenReturn(OperatorTestUtil.block(schema, new Object[]{1, "AA"}, new Object[]{2, "BB"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); Mockito.when(_rightOperator.nextBlock()).thenReturn( OperatorTestUtil.block(schema, new Object[]{3, "aa"}, new Object[]{4, "bb"}, new Object[]{5, "cc"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); UnionOperator unionOperator = - new UnionOperator(OperatorTestUtil.getDefaultContext(), ImmutableList.of(_leftOperator, _rightOperator), + new UnionOperator(OperatorTestUtil.getTracingContext(), ImmutableList.of(_leftOperator, _rightOperator), schema); List resultRows = new ArrayList<>(); TransferableBlock result = unionOperator.nextBlock(); diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperatorTest.java index 3af7d5b2cc83..2bfca7c14947 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperatorTest.java @@ -34,6 +34,7 @@ import org.apache.pinot.query.planner.plannode.WindowNode; import org.apache.pinot.query.routing.VirtualServerAddress; import org.apache.pinot.query.runtime.blocks.TransferableBlock; +import org.apache.pinot.query.runtime.blocks.TransferableBlockTestUtils; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.operator.utils.AggregationUtils; import org.mockito.Mock; @@ -84,7 +85,7 @@ public void testShouldHandleUpstreamErrorBlocks() { DataSchema inSchema = new DataSchema(new String[]{"group", "arg"}, new ColumnDataType[]{INT, INT}); DataSchema outSchema = new DataSchema(new String[]{"group", "arg", "sum"}, new ColumnDataType[]{INT, INT, DOUBLE}); WindowAggregateOperator operator = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), _input, group, Collections.emptyList(), + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), _input, group, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), calls, Integer.MIN_VALUE, Integer.MAX_VALUE, WindowNode.WindowFrameType.RANGE, Collections.emptyList(), outSchema, inSchema); @@ -102,12 +103,12 @@ public void testShouldHandleEndOfStreamBlockWithNoOtherInputs() { List calls = ImmutableList.of(getSum(new RexExpression.InputRef(1))); List group = ImmutableList.of(new RexExpression.InputRef(0)); - Mockito.when(_input.nextBlock()).thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + Mockito.when(_input.nextBlock()).thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema inSchema = new DataSchema(new String[]{"group", "arg"}, new ColumnDataType[]{INT, INT}); DataSchema outSchema = new DataSchema(new String[]{"group", "arg", "sum"}, new ColumnDataType[]{INT, INT, DOUBLE}); WindowAggregateOperator operator = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), _input, group, Collections.emptyList(), + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), _input, group, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), calls, Integer.MIN_VALUE, Integer.MAX_VALUE, WindowNode.WindowFrameType.RANGE, Collections.emptyList(), outSchema, inSchema); @@ -127,11 +128,11 @@ public void testShouldWindowAggregateOverSingleInputBlock() { DataSchema inSchema = new DataSchema(new String[]{"group", "arg"}, new ColumnDataType[]{INT, INT}); Mockito.when(_input.nextBlock()).thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, 1})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "arg", "sum"}, new ColumnDataType[]{INT, INT, DOUBLE}); WindowAggregateOperator operator = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), _input, group, Collections.emptyList(), + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), _input, group, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), calls, Integer.MIN_VALUE, Integer.MAX_VALUE, WindowNode.WindowFrameType.RANGE, Collections.emptyList(), outSchema, inSchema); @@ -155,11 +156,11 @@ public void testShouldWindowAggregateOverSingleInputBlockWithSameOrderByKeys() { DataSchema inSchema = new DataSchema(new String[]{"group", "arg"}, new ColumnDataType[]{INT, INT}); Mockito.when(_input.nextBlock()).thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, 1})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "arg", "sum"}, new ColumnDataType[]{INT, INT, DOUBLE}); WindowAggregateOperator operator = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), _input, group, order, + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), _input, group, order, Arrays.asList(RelFieldCollation.Direction.ASCENDING), Arrays.asList(RelFieldCollation.NullDirection.LAST), calls, Integer.MIN_VALUE, Integer.MAX_VALUE, WindowNode.WindowFrameType.RANGE, Collections.emptyList(), outSchema, inSchema); @@ -182,11 +183,11 @@ public void testShouldWindowAggregateOverSingleInputBlockWithoutPartitionByKeys( DataSchema inSchema = new DataSchema(new String[]{"group", "arg"}, new ColumnDataType[]{INT, INT}); Mockito.when(_input.nextBlock()).thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, 1})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "arg", "sum"}, new ColumnDataType[]{INT, INT, DOUBLE}); WindowAggregateOperator operator = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), _input, Collections.emptyList(), + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), _input, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), calls, Integer.MIN_VALUE, Integer.MAX_VALUE, WindowNode.WindowFrameType.RANGE, Collections.emptyList(), outSchema, inSchema); @@ -209,11 +210,11 @@ public void testShouldWindowAggregateOverSingleInputBlockWithLiteralInput() { DataSchema inSchema = new DataSchema(new String[]{"group", "arg"}, new ColumnDataType[]{INT, INT}); Mockito.when(_input.nextBlock()).thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, 3})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "arg", "sum"}, new ColumnDataType[]{INT, INT, DOUBLE}); WindowAggregateOperator operator = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), _input, group, Collections.emptyList(), + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), _input, group, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), calls, Integer.MIN_VALUE, Integer.MAX_VALUE, WindowNode.WindowFrameType.RANGE, Collections.emptyList(), outSchema, inSchema); @@ -239,14 +240,14 @@ public void testShouldCallMergerWhenWindowAggregatingMultipleRows() { Mockito.when(_input.nextBlock()) .thenReturn(OperatorTestUtil.block(inSchema, new Object[]{1, 1}, new Object[]{1, 2})) .thenReturn(OperatorTestUtil.block(inSchema, new Object[]{1, 3})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); AggregationUtils.Merger merger = Mockito.mock(AggregationUtils.Merger.class); Mockito.when(merger.merge(Mockito.any(), Mockito.any())).thenReturn(12d); Mockito.when(merger.init(Mockito.any(), Mockito.any())).thenReturn(1d); DataSchema outSchema = new DataSchema(new String[]{"group", "arg", "sum"}, new ColumnDataType[]{INT, INT, DOUBLE}); WindowAggregateOperator operator = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), _input, group, Collections.emptyList(), + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), _input, group, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), calls, Integer.MIN_VALUE, Integer.MAX_VALUE, WindowNode.WindowFrameType.RANGE, Collections.emptyList(), outSchema, inSchema, ImmutableMap.of("SUM", cdt -> merger)); @@ -277,7 +278,7 @@ public void testPartitionByWindowAggregateWithHashCollision() { DataSchema outSchema = new DataSchema(new String[]{"arg", "group", "sum"}, new ColumnDataType[]{INT, STRING, DOUBLE}); WindowAggregateOperator sum0PartitionBy1 = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), upstreamOperator, group, + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), upstreamOperator, group, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), calls, Integer.MIN_VALUE, Integer.MAX_VALUE, WindowNode.WindowFrameType.RANGE, Collections.emptyList(), outSchema, inSchema); @@ -303,7 +304,7 @@ public void testShouldThrowOnUnknownAggFunction() { // When: WindowAggregateOperator operator = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), _input, group, Collections.emptyList(), + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), _input, group, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), calls, Integer.MIN_VALUE, Integer.MAX_VALUE, WindowNode.WindowFrameType.RANGE, Collections.emptyList(), outSchema, inSchema); } @@ -321,7 +322,7 @@ public void testShouldThrowOnUnknownRankAggFunction() { // When: WindowAggregateOperator operator = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), _input, group, Collections.emptyList(), + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), _input, group, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), calls, Integer.MIN_VALUE, Integer.MAX_VALUE, WindowNode.WindowFrameType.RANGE, Collections.emptyList(), outSchema, inSchema); } @@ -342,14 +343,14 @@ public void testRankDenseRankRankingFunctions() { new Object[]{1, "foo"})).thenReturn( OperatorTestUtil.block(inSchema, new Object[]{1, "foo"}, new Object[]{2, "foo"}, new Object[]{1, "numb"}, new Object[]{2, "the"}, new Object[]{3, "true"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "arg", "rank", "dense_rank"}, new ColumnDataType[]{INT, STRING, LONG, LONG}); // When: WindowAggregateOperator operator = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), _input, group, order, Collections.emptyList(), + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), _input, group, order, Collections.emptyList(), Collections.emptyList(), calls, Integer.MIN_VALUE, 0, WindowNode.WindowFrameType.RANGE, Collections.emptyList(), outSchema, inSchema); @@ -399,14 +400,14 @@ public void testRowNumberRankingFunction() { OperatorTestUtil.block(inSchema, new Object[]{3, "and"}, new Object[]{2, "bar"}, new Object[]{2, "foo"})) .thenReturn( OperatorTestUtil.block(inSchema, new Object[]{1, "foo"}, new Object[]{2, "foo"}, new Object[]{2, "the"}, - new Object[]{3, "true"})).thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + new Object[]{3, "true"})).thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "arg", "row_number"}, new ColumnDataType[]{INT, STRING, LONG}); // When: WindowAggregateOperator operator = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), _input, group, order, Collections.emptyList(), + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), _input, group, order, Collections.emptyList(), Collections.emptyList(), calls, Integer.MIN_VALUE, 0, WindowNode.WindowFrameType.ROWS, Collections.emptyList(), outSchema, inSchema); @@ -454,12 +455,12 @@ public void testNonEmptyOrderByKeysNotMatchingPartitionByKeys() { OperatorTestUtil.block(inSchema, new Object[]{3, "and"}, new Object[]{2, "bar"}, new Object[]{2, "foo"})) .thenReturn( OperatorTestUtil.block(inSchema, new Object[]{1, "foo"}, new Object[]{2, "foo"}, new Object[]{3, "true"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "arg", "sum"}, new ColumnDataType[]{INT, STRING, DOUBLE}); WindowAggregateOperator operator = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), _input, group, order, + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), _input, group, order, Arrays.asList(RelFieldCollation.Direction.ASCENDING), Arrays.asList(RelFieldCollation.NullDirection.LAST), calls, Integer.MIN_VALUE, Integer.MAX_VALUE, WindowNode.WindowFrameType.RANGE, Collections.emptyList(), outSchema, inSchema); @@ -494,12 +495,12 @@ public void testNonEmptyOrderByKeysMatchingPartitionByKeysWithDifferentDirection Mockito.when(_input.nextBlock()).thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, "foo"})) .thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, "bar"})) .thenReturn(OperatorTestUtil.block(inSchema, new Object[]{3, "foo"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "arg", "sum"}, new ColumnDataType[]{INT, STRING, DOUBLE}); WindowAggregateOperator operator = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), _input, group, order, + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), _input, group, order, Arrays.asList(RelFieldCollation.Direction.DESCENDING), Arrays.asList(RelFieldCollation.NullDirection.LAST), calls, Integer.MIN_VALUE, Integer.MAX_VALUE, WindowNode.WindowFrameType.RANGE, Collections.emptyList(), outSchema, inSchema); @@ -526,12 +527,12 @@ public void testShouldThrowOnCustomFramesRows() { DataSchema inSchema = new DataSchema(new String[]{"group", "arg"}, new ColumnDataType[]{INT, STRING}); Mockito.when(_input.nextBlock()).thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, "foo"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "arg", "sum"}, new ColumnDataType[]{INT, STRING, DOUBLE}); WindowAggregateOperator operator = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), _input, group, Collections.emptyList(), + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), _input, group, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), calls, Integer.MIN_VALUE, Integer.MAX_VALUE, WindowNode.WindowFrameType.ROWS, Collections.emptyList(), outSchema, inSchema); } @@ -545,12 +546,12 @@ public void testShouldNotThrowCurrentRowPartitionByOrderByOnSameKey() { DataSchema inSchema = new DataSchema(new String[]{"group", "arg"}, new ColumnDataType[]{INT, STRING}); Mockito.when(_input.nextBlock()).thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, "foo"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "arg", "sum"}, new ColumnDataType[]{INT, STRING, DOUBLE}); WindowAggregateOperator operator = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), _input, group, order, + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), _input, group, order, Arrays.asList(RelFieldCollation.Direction.ASCENDING), Arrays.asList(RelFieldCollation.NullDirection.LAST), calls, Integer.MIN_VALUE, 0, WindowNode.WindowFrameType.RANGE, Collections.emptyList(), outSchema, inSchema); @@ -576,12 +577,12 @@ public void testShouldThrowOnCustomFramesCustomPreceding() { DataSchema inSchema = new DataSchema(new String[]{"group", "arg"}, new ColumnDataType[]{INT, STRING}); Mockito.when(_input.nextBlock()).thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, "foo"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "arg", "sum"}, new ColumnDataType[]{INT, STRING, DOUBLE}); WindowAggregateOperator operator = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), _input, group, Collections.emptyList(), + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), _input, group, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), calls, 5, Integer.MAX_VALUE, WindowNode.WindowFrameType.RANGE, Collections.emptyList(), outSchema, inSchema); } @@ -596,12 +597,12 @@ public void testShouldThrowOnCustomFramesCustomFollowing() { DataSchema inSchema = new DataSchema(new String[]{"group", "arg"}, new ColumnDataType[]{INT, STRING}); Mockito.when(_input.nextBlock()).thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, "foo"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "arg", "sum"}, new ColumnDataType[]{INT, STRING, DOUBLE}); WindowAggregateOperator operator = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), _input, group, Collections.emptyList(), + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), _input, group, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), calls, Integer.MIN_VALUE, 5, WindowNode.WindowFrameType.RANGE, Collections.emptyList(), outSchema, inSchema); } @@ -617,12 +618,12 @@ public void testShouldReturnErrorBlockOnUnexpectedInputType() { // (see the comment in WindowAggregate operator) Mockito.when(_input.nextBlock()).thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, "metallica"})) .thenReturn(OperatorTestUtil.block(inSchema, new Object[]{2, "pink floyd"})) - .thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + .thenReturn(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); DataSchema outSchema = new DataSchema(new String[]{"group", "arg", "sum"}, new ColumnDataType[]{INT, STRING, DOUBLE}); WindowAggregateOperator operator = - new WindowAggregateOperator(OperatorTestUtil.getDefaultContext(), _input, group, Collections.emptyList(), + new WindowAggregateOperator(OperatorTestUtil.getTracingContext(), _input, group, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), calls, Integer.MIN_VALUE, Integer.MAX_VALUE, WindowNode.WindowFrameType.RANGE, Collections.emptyList(), outSchema, inSchema); diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/exchange/BlockExchangeTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/exchange/BlockExchangeTest.java index 752d8ea4b39a..182b128798a8 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/exchange/BlockExchangeTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/exchange/BlockExchangeTest.java @@ -27,7 +27,7 @@ import org.apache.pinot.query.mailbox.SendingMailbox; import org.apache.pinot.query.runtime.blocks.BlockSplitter; import org.apache.pinot.query.runtime.blocks.TransferableBlock; -import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; +import org.apache.pinot.query.runtime.blocks.TransferableBlockTestUtils; import org.mockito.ArgumentCaptor; import org.mockito.Mock; import org.mockito.Mockito; @@ -67,7 +67,7 @@ public void shouldSendEosBlockToAllDestinations() BlockExchange exchange = new TestBlockExchange(destinations); // When: - exchange.send(TransferableBlockUtils.getEndOfStreamTransferableBlock()); + exchange.send(TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); // Then: ArgumentCaptor captor = ArgumentCaptor.forClass(TransferableBlock.class); diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/MultiStageQueryStatsTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/MultiStageQueryStatsTest.java new file mode 100644 index 000000000000..0135a429d0bc --- /dev/null +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/MultiStageQueryStatsTest.java @@ -0,0 +1,100 @@ +/** + * 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.pinot.query.runtime.plan; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import org.apache.pinot.common.datatable.StatMap; +import org.apache.pinot.query.runtime.operator.BaseMailboxReceiveOperator; +import org.apache.pinot.query.runtime.operator.LeafStageTransferableBlockOperator; +import org.apache.pinot.query.runtime.operator.MailboxSendOperator; +import org.apache.pinot.query.runtime.operator.MultiStageOperator; +import org.apache.pinot.query.runtime.operator.SortOperator; +import org.testng.Assert; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + + +public class MultiStageQueryStatsTest { + + /** + * A test that verifies calling {@link MultiStageQueryStats#mergeUpstream(MultiStageQueryStats)} is similar to call + * {@link MultiStageQueryStats#serialize()} and then {@link MultiStageQueryStats#mergeUpstream(List)}. + */ + @Test(dataProvider = "stats") + public void testMergeEquivalence(MultiStageQueryStats stats) + throws IOException { + + assert stats.getCurrentStageId() > 0 : "Stage id should be greater than 0 in order to run this test"; + + MultiStageQueryStats mergingHeap = MultiStageQueryStats.emptyStats(0); + mergingHeap.mergeUpstream(stats); + + List buffers = stats.serialize(); + MultiStageQueryStats rootStats = MultiStageQueryStats.emptyStats(0); + rootStats.mergeUpstream(buffers); + + Assert.assertEquals(mergingHeap, rootStats, "Merging objects should be equal to merging serialized buffers"); + } + + @DataProvider(name = "stats") + public static MultiStageQueryStats[] stats() { + return new MultiStageQueryStats[] { + stats1() + }; + } + + public static MultiStageQueryStats stats1() { + return new MultiStageQueryStats.Builder(1) + .customizeOpen(open -> + open.addLastOperator(MultiStageOperator.Type.MAILBOX_RECEIVE, + new StatMap<>(BaseMailboxReceiveOperator.StatKey.class) + .merge(BaseMailboxReceiveOperator.StatKey.EXECUTION_TIME_MS, 100) + .merge(BaseMailboxReceiveOperator.StatKey.EMITTED_ROWS, 10)) + .addLastOperator(MultiStageOperator.Type.SORT_OR_LIMIT, + new StatMap<>(SortOperator.StatKey.class) + .merge(SortOperator.StatKey.EXECUTION_TIME_MS, 10) + .merge(SortOperator.StatKey.EMITTED_ROWS, 10)) + .addLastOperator(MultiStageOperator.Type.MAILBOX_SEND, + new StatMap<>(MailboxSendOperator.StatKey.class) + .merge(MailboxSendOperator.StatKey.STAGE, 1) + .merge(MailboxSendOperator.StatKey.EXECUTION_TIME_MS, 100) + .merge(MailboxSendOperator.StatKey.EMITTED_ROWS, 10)) + ) + .addLast(stageStats -> + stageStats.addLastOperator(MultiStageOperator.Type.LEAF, + new StatMap<>(LeafStageTransferableBlockOperator.StatKey.class) + .merge(LeafStageTransferableBlockOperator.StatKey.NUM_SEGMENTS_QUERIED, 1) + .merge(LeafStageTransferableBlockOperator.StatKey.NUM_SEGMENTS_PROCESSED, 1) + .merge(LeafStageTransferableBlockOperator.StatKey.NUM_SEGMENTS_MATCHED, 1) + .merge(LeafStageTransferableBlockOperator.StatKey.NUM_DOCS_SCANNED, 10) + .merge(LeafStageTransferableBlockOperator.StatKey.NUM_ENTRIES_SCANNED_POST_FILTER, 5) + .merge(LeafStageTransferableBlockOperator.StatKey.TOTAL_DOCS, 5) + .merge(LeafStageTransferableBlockOperator.StatKey.EXECUTION_TIME_MS, 95) + .merge(LeafStageTransferableBlockOperator.StatKey.TABLE, "a")) + .addLastOperator(MultiStageOperator.Type.MAILBOX_SEND, + new StatMap<>(MailboxSendOperator.StatKey.class) + .merge(MailboxSendOperator.StatKey.STAGE, 2) + .merge(MailboxSendOperator.StatKey.EXECUTION_TIME_MS, 135) + .merge(MailboxSendOperator.StatKey.EMITTED_ROWS, 5)) + .close()) + .build(); + } +} diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutorTest.java index 58dcd2106ef7..6532e97a087d 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutorTest.java @@ -28,6 +28,7 @@ import org.apache.calcite.rel.RelDistribution; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.logical.PinotRelExchangeType; +import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.query.mailbox.MailboxService; import org.apache.pinot.query.mailbox.ReceivingMailbox; @@ -42,6 +43,7 @@ import org.apache.pinot.query.routing.VirtualServerAddress; import org.apache.pinot.query.routing.WorkerMetadata; import org.apache.pinot.query.runtime.blocks.TransferableBlock; +import org.apache.pinot.query.runtime.blocks.TransferableBlockTestUtils; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.executor.ExecutorServiceUtils; import org.apache.pinot.query.runtime.executor.OpChainSchedulerService; @@ -89,7 +91,9 @@ public void setUpMethod() { when(_mailboxService.getPort()).thenReturn(123); when(_mailbox1.getId()).thenReturn("mailbox1"); + when(_mailbox1.getStatMap()).thenReturn(new StatMap<>(ReceivingMailbox.StatKey.class)); when(_mailbox2.getId()).thenReturn("mailbox2"); + when(_mailbox2.getStatMap()).thenReturn(new StatMap<>(ReceivingMailbox.StatKey.class)); } @AfterMethod @@ -116,7 +120,7 @@ public void shouldReturnBlocksUponNormalOperation() { Object[] row2 = new Object[]{2, 3}; when(_mailbox1.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row1), OperatorTestUtil.block(DATA_SCHEMA, row2), - TransferableBlockUtils.getEndOfStreamTransferableBlock(OperatorTestUtil.getDummyStats(0, 1, _server))); + TransferableBlockUtils.getEndOfStreamTransferableBlock(OperatorTestUtil.getDummyStats(1))); PipelineBreakerResult pipelineBreakerResult = PipelineBreakerExecutor.executePipelineBreakers(_scheduler, _mailboxService, _workerMetadata, stagePlan, @@ -130,8 +134,9 @@ public void shouldReturnBlocksUponNormalOperation() { Assert.assertEquals(pipelineBreakerResult.getResultMap().values().iterator().next().size(), 2); // should collect stats from previous stage here - Assert.assertNotNull(pipelineBreakerResult.getOpChainStats()); - Assert.assertEquals(pipelineBreakerResult.getOpChainStats().getOperatorStatsMap().size(), 1); + Assert.assertNotNull(pipelineBreakerResult.getStageQueryStats()); + Assert.assertNotNull(pipelineBreakerResult.getStageQueryStats().getUpstreamStageStats(1), + "Stats for stage 1 should be sent"); } @Test @@ -154,9 +159,9 @@ public void shouldWorkWithMultiplePBNodeUponNormalOperation() { Object[] row1 = new Object[]{1, 1}; Object[] row2 = new Object[]{2, 3}; when(_mailbox1.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row1), - TransferableBlockUtils.getEndOfStreamTransferableBlock(OperatorTestUtil.getDummyStats(0, 1, _server))); + TransferableBlockUtils.getEndOfStreamTransferableBlock(OperatorTestUtil.getDummyStats(1))); when(_mailbox2.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row2), - TransferableBlockUtils.getEndOfStreamTransferableBlock(OperatorTestUtil.getDummyStats(0, 2, _server))); + TransferableBlockUtils.getEndOfStreamTransferableBlock(OperatorTestUtil.getDummyStats(2))); PipelineBreakerResult pipelineBreakerResult = PipelineBreakerExecutor.executePipelineBreakers(_scheduler, _mailboxService, _workerMetadata, stagePlan, @@ -173,8 +178,11 @@ public void shouldWorkWithMultiplePBNodeUponNormalOperation() { Assert.assertFalse(it.hasNext()); // should collect stats from previous stage here - Assert.assertNotNull(pipelineBreakerResult.getOpChainStats()); - Assert.assertEquals(pipelineBreakerResult.getOpChainStats().getOperatorStatsMap().size(), 2); + Assert.assertNotNull(pipelineBreakerResult.getStageQueryStats()); + Assert.assertNotNull(pipelineBreakerResult.getStageQueryStats().getUpstreamStageStats(1), + "Stats for stage 1 should be sent"); + Assert.assertNotNull(pipelineBreakerResult.getStageQueryStats().getUpstreamStageStats(2), + "Stats for stage 2 should be sent"); } @Test @@ -197,7 +205,7 @@ public void shouldReturnEmptyBlockWhenPBExecuteWithIncorrectMailboxNode() { List resultBlocks = pipelineBreakerResult.getResultMap().values().iterator().next(); Assert.assertEquals(resultBlocks.size(), 0); - Assert.assertNotNull(pipelineBreakerResult.getOpChainStats()); + Assert.assertNotNull(pipelineBreakerResult.getStageQueryStats()); } @Test @@ -212,7 +220,7 @@ public void shouldReturnErrorBlocksFailureWhenPBTimeout() { CountDownLatch latch = new CountDownLatch(1); when(_mailbox1.poll()).thenAnswer(invocation -> { latch.await(); - return TransferableBlockUtils.getEndOfStreamTransferableBlock(); + return TransferableBlockTestUtils.getEndOfStreamTransferableBlock(1); }); PipelineBreakerResult pipelineBreakerResult = @@ -249,9 +257,9 @@ public void shouldReturnWhenAnyPBReturnsEmpty() { Object[] row1 = new Object[]{1, 1}; Object[] row2 = new Object[]{2, 3}; when(_mailbox1.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row1), - TransferableBlockUtils.getEndOfStreamTransferableBlock()); + TransferableBlockTestUtils.getEndOfStreamTransferableBlock(1)); when(_mailbox2.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row2), - TransferableBlockUtils.getEndOfStreamTransferableBlock()); + TransferableBlockTestUtils.getEndOfStreamTransferableBlock(1)); PipelineBreakerResult pipelineBreakerResult = PipelineBreakerExecutor.executePipelineBreakers(_scheduler, _mailboxService, _workerMetadata, stagePlan, @@ -264,7 +272,7 @@ public void shouldReturnWhenAnyPBReturnsEmpty() { Assert.assertEquals(pipelineBreakerResult.getResultMap().get(0).size(), 1); Assert.assertEquals(pipelineBreakerResult.getResultMap().get(1).size(), 0); - Assert.assertNotNull(pipelineBreakerResult.getOpChainStats()); + Assert.assertNotNull(pipelineBreakerResult.getStageQueryStats()); } @Test @@ -289,7 +297,7 @@ public void shouldReturnErrorBlocksWhenReceivedErrorFromSender() { when(_mailbox1.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row1), TransferableBlockUtils.getErrorTransferableBlock(new RuntimeException("ERROR ON 1"))); when(_mailbox2.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row2), - TransferableBlockUtils.getEndOfStreamTransferableBlock()); + TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0)); PipelineBreakerResult pipelineBreakerResult = PipelineBreakerExecutor.executePipelineBreakers(_scheduler, _mailboxService, _workerMetadata, stagePlan, diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTest.java index 0aea318c12a4..266ddc15b63e 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTest.java @@ -155,7 +155,7 @@ public void tearDown() { */ @Test(dataProvider = "testDataWithSqlToFinalRowCount") public void testSqlWithFinalRowCountChecker(String sql, int expectedRows) { - ResultTable resultTable = queryRunner(sql, null); + ResultTable resultTable = queryRunner(sql, false).getResultTable(); Assert.assertEquals(resultTable.getRows().size(), expectedRows); } @@ -168,7 +168,7 @@ public void testSqlWithFinalRowCountChecker(String sql, int expectedRows) { @Test(dataProvider = "testSql") public void testSqlWithH2Checker(String sql) throws Exception { - ResultTable resultTable = queryRunner(sql, null); + ResultTable resultTable = queryRunner(sql, false).getResultTable(); // query H2 for data List expectedRows = queryH2(sql); compareRowEquals(resultTable, expectedRows); @@ -181,7 +181,7 @@ public void testSqlWithH2Checker(String sql) public void testSqlWithExceptionMsgChecker(String sql, String exceptionMsg) { try { // query pinot - ResultTable resultTable = queryRunner(sql, null); + ResultTable resultTable = queryRunner(sql, false).getResultTable(); Assert.fail("Expected error with message '" + exceptionMsg + "'. But instead rows were returned: " + JsonUtils.objectToPrettyString(resultTable)); } catch (Exception e) { diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTestBase.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTestBase.java index 298af4deaf8e..de6546672509 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTestBase.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTestBase.java @@ -50,7 +50,6 @@ import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; import org.apache.pinot.common.utils.config.QueryOptionsUtils; -import org.apache.pinot.core.query.reduce.ExecutionStatsAggregator; import org.apache.pinot.query.QueryEnvironment; import org.apache.pinot.query.QueryServerEnclosure; import org.apache.pinot.query.QueryTestSet; @@ -99,11 +98,17 @@ public abstract class QueryRunnerTestBase extends QueryTestSet { // QUERY UTILS // -------------------------------------------------------------------------- + protected QueryEnvironment.QueryPlannerResult planQuery(String sql) { + long requestId = REQUEST_ID_GEN.getAndIncrement(); + SqlNodeAndOptions sqlNodeAndOptions = CalciteSqlParser.compileToSqlNodeAndOptions(sql); + return _queryEnvironment.planQuery(sql, sqlNodeAndOptions, requestId); + } + /** * Dispatch query to each pinot-server. The logic should mimic QueryDispatcher.submit() but does not actually make * ser/de dispatches. */ - protected ResultTable queryRunner(String sql, Map executionStatsAggregatorMap) { + protected QueryDispatcher.QueryResult queryRunner(String sql, boolean trace) { long requestId = REQUEST_ID_GEN.getAndIncrement(); SqlNodeAndOptions sqlNodeAndOptions = CalciteSqlParser.compileToSqlNodeAndOptions(sql); QueryEnvironment.QueryPlannerResult queryPlannerResult = @@ -119,7 +124,7 @@ protected ResultTable queryRunner(String sql, Map> processDistributedStagePlans(DispatchableSubPlan dispatchableSubPlan, diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/ResourceBasedQueriesTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/ResourceBasedQueriesTest.java index bb6744084a2c..13e35af7ec31 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/ResourceBasedQueriesTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/ResourceBasedQueriesTest.java @@ -19,7 +19,9 @@ package org.apache.pinot.query.runtime.queries; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; import java.io.BufferedReader; import java.io.File; import java.io.InputStream; @@ -35,18 +37,22 @@ import java.util.TimeZone; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.Collectors; import javax.annotation.Nullable; import org.apache.commons.lang3.tuple.Pair; -import org.apache.pinot.common.datatable.DataTable; import org.apache.pinot.common.response.broker.BrokerResponseNativeV2; -import org.apache.pinot.common.response.broker.BrokerResponseStats; -import org.apache.pinot.common.response.broker.ResultTable; import org.apache.pinot.core.common.datatable.DataTableBuilderFactory; -import org.apache.pinot.core.query.reduce.ExecutionStatsAggregator; import org.apache.pinot.query.QueryEnvironmentTestBase; import org.apache.pinot.query.QueryServerEnclosure; import org.apache.pinot.query.mailbox.MailboxService; +import org.apache.pinot.query.planner.PlanFragment; +import org.apache.pinot.query.planner.physical.DispatchablePlanFragment; +import org.apache.pinot.query.planner.plannode.PlanNode; import org.apache.pinot.query.routing.QueryServerInstance; +import org.apache.pinot.query.runtime.MultiStageStatsTreeBuilder; +import org.apache.pinot.query.runtime.operator.LeafStageTransferableBlockOperator; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; +import org.apache.pinot.query.service.dispatch.QueryDispatcher; import org.apache.pinot.query.testutils.MockInstanceDataManagerFactory; import org.apache.pinot.query.testutils.QueryTestUtils; import org.apache.pinot.spi.config.table.TableType; @@ -256,9 +262,9 @@ public void testQueryTestCasesWithH2(String testCaseName, boolean isIgnored, Str boolean keepOutputRowOrder) throws Exception { // query pinot - runQuery(sql, expect, null).ifPresent(resultTable -> { + runQuery(sql, expect, false).ifPresent(queryResult -> { try { - compareRowEquals(resultTable, queryH2(h2Sql), keepOutputRowOrder); + compareRowEquals(queryResult.getResultTable(), queryH2(h2Sql), keepOutputRowOrder); } catch (Exception e) { Assert.fail(e.getMessage(), e); } @@ -269,74 +275,80 @@ public void testQueryTestCasesWithH2(String testCaseName, boolean isIgnored, Str public void testQueryTestCasesWithOutput(String testCaseName, boolean isIgnored, String sql, String h2Sql, List expectedRows, String expect, boolean keepOutputRowOrder) throws Exception { - runQuery(sql, expect, null).ifPresent( - resultTable -> compareRowEquals(resultTable, expectedRows, keepOutputRowOrder)); + runQuery(sql, expect, false).ifPresent( + queryResult -> compareRowEquals(queryResult.getResultTable(), expectedRows, keepOutputRowOrder)); + } + + private Map tableToStats(String sql, QueryDispatcher.QueryResult queryResult) { + + List planNodes = planQuery(sql).getQueryPlan().getQueryStageList().stream() + .map(DispatchablePlanFragment::getPlanFragment) + .map(PlanFragment::getFragmentRoot) + .collect(Collectors.toList()); + + MultiStageStatsTreeBuilder multiStageStatsTreeBuilder = + new MultiStageStatsTreeBuilder(planNodes, queryResult.getQueryStats()); + ObjectNode jsonNodes = multiStageStatsTreeBuilder.jsonStatsByStage(1); + + Map map = new HashMap<>(); + tableToStatsRec(map, jsonNodes); + return map; + } + + private void tableToStatsRec(Map map, ObjectNode node) { + JsonNode type = node.get("type"); + if (type == null || !type.equals("LEAF")) { + return; + } + String tableName = node.get("table").asText(); + JsonNode old = map.put(tableName, node); + if (old != null) { + throw new RuntimeException("Found at least two leaf stages for table " + tableName); + } + JsonNode children = node.get("children"); + if (children != null) { + for (JsonNode child : children) { + tableToStatsRec(map, (ObjectNode) child); + } + } } @Test(dataProvider = "testResourceQueryTestCaseProviderWithMetadata") public void testQueryTestCasesWithMetadata(String testCaseName, boolean isIgnored, String sql, String h2Sql, String expect, int numSegments) throws Exception { - Map executionStatsAggregatorMap = new HashMap<>(); - runQuery(sql, expect, executionStatsAggregatorMap).ifPresent(resultTable -> { + runQuery(sql, expect, true).ifPresent(queryResult -> { BrokerResponseNativeV2 brokerResponseNative = new BrokerResponseNativeV2(); - executionStatsAggregatorMap.get(0).setStats(brokerResponseNative); - Assert.assertFalse(executionStatsAggregatorMap.isEmpty()); - for (Integer stageId : executionStatsAggregatorMap.keySet()) { - if (stageId > 0) { - BrokerResponseStats brokerResponseStats = new BrokerResponseStats(); - executionStatsAggregatorMap.get(stageId).setStageLevelStats(null, brokerResponseStats, null); - brokerResponseNative.addStageStat(stageId, brokerResponseStats); - } + for (MultiStageQueryStats.StageStats.Closed stageStats : queryResult.getQueryStats()) { + stageStats.forEach((type, stats) -> type.mergeInto(brokerResponseNative, stats)); } Assert.assertEquals(brokerResponseNative.getNumSegmentsQueried(), numSegments); - Map stageIdStats = brokerResponseNative.getStageIdStats(); - int numTables = 0; - for (Integer stageId : stageIdStats.keySet()) { - // check stats only for leaf stage - BrokerResponseStats brokerResponseStats = stageIdStats.get(stageId); - - if (brokerResponseStats.getTableNames().isEmpty()) { - continue; - } - - String tableName = brokerResponseStats.getTableNames().get(0); - Assert.assertEquals(brokerResponseStats.getTableNames().size(), 1); - numTables++; + Map tableToStats = tableToStats(sql, queryResult); + for (Map.Entry entry : tableToStats.entrySet()) { + String tableName = entry.getKey(); TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableName); if (tableType == null) { tableName = TableNameBuilder.OFFLINE.tableNameWithType(tableName); } Assert.assertNotNull(_tableToSegmentMap.get(tableName)); - Assert.assertEquals(brokerResponseStats.getNumSegmentsQueried(), _tableToSegmentMap.get(tableName).size()); - - Assert.assertFalse(brokerResponseStats.getOperatorStats().isEmpty()); - Map> operatorStats = brokerResponseStats.getOperatorStats(); - for (Map.Entry> entry : operatorStats.entrySet()) { - if (entry.getKey().contains("LEAF_STAGE")) { - Assert.assertNotNull(entry.getValue().get(DataTable.MetadataKey.NUM_SEGMENTS_QUERIED.getName())); - } else { - Assert.assertNotNull(entry.getValue().get(DataTable.MetadataKey.NUM_BLOCKS.getName())); - } - } + String statName = LeafStageTransferableBlockOperator.StatKey.NUM_SEGMENTS_QUERIED.getStatName(); + int numSegmentsQueried = entry.getValue().get(statName).asInt(); + Assert.assertEquals(numSegmentsQueried, _tableToSegmentMap.get(tableName).size()); } - - Assert.assertTrue(numTables > 0); }); } - private Optional runQuery(String sql, final String except, - Map executionStatsAggregatorMap) + private Optional runQuery(String sql, final String except, boolean trace) throws Exception { try { // query pinot - ResultTable resultTable = queryRunner(sql, executionStatsAggregatorMap); + QueryDispatcher.QueryResult queryResult = queryRunner(sql, trace); Assert.assertNull(except, "Expected error with message '" + except + "'. But instead rows were returned: " - + JsonUtils.objectToPrettyString(resultTable)); - return Optional.of(resultTable); + + JsonUtils.objectToPrettyString(queryResult.getResultTable())); + return Optional.of(queryResult); } catch (Exception e) { if (except == null) { throw e; diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/QueryDispatcherTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/QueryDispatcherTest.java index 5af8f038c3a1..694fb3c08779 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/QueryDispatcherTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/QueryDispatcherTest.java @@ -118,7 +118,7 @@ public void testQueryDispatcherCancelWhenQueryServerCallsOnError() context.setRequestId(requestId); DispatchableSubPlan dispatchableSubPlan = _queryEnvironment.planQuery(sql); try { - _queryDispatcher.submitAndReduce(context, dispatchableSubPlan, 10_000L, Collections.emptyMap(), null); + _queryDispatcher.submitAndReduce(context, dispatchableSubPlan, 10_000L, Collections.emptyMap()); Assert.fail("Method call above should have failed"); } catch (Exception e) { Assert.assertTrue(e.getMessage().contains("Error dispatching query")); @@ -142,7 +142,7 @@ public void testQueryDispatcherCancelWhenQueryReducerThrowsError() DispatchableSubPlan dispatchableSubPlan = _queryEnvironment.planQuery(sql); try { // will throw b/c mailboxService is mocked - _queryDispatcher.submitAndReduce(context, dispatchableSubPlan, 10_000L, Collections.emptyMap(), null); + _queryDispatcher.submitAndReduce(context, dispatchableSubPlan, 10_000L, Collections.emptyMap()); Assert.fail("Method call above should have failed"); } catch (NullPointerException e) { // Expected diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/testutils/MockDataBlockOperatorFactory.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/testutils/MockDataBlockOperatorFactory.java index 94acd49fd8c6..abe5770ebc75 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/testutils/MockDataBlockOperatorFactory.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/testutils/MockDataBlockOperatorFactory.java @@ -25,7 +25,7 @@ import org.apache.pinot.common.datablock.DataBlock; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.query.runtime.blocks.TransferableBlock; -import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; +import org.apache.pinot.query.runtime.blocks.TransferableBlockTestUtils; import org.apache.pinot.query.runtime.operator.MultiStageOperator; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; @@ -60,7 +60,7 @@ public MultiStageOperator buildMockOperator(String operatorName) { private int _invocationCount = 0; public Object answer(InvocationOnMock invocation) { return _invocationCount >= _rowsMap.get(operatorName).size() - ? TransferableBlockUtils.getEndOfStreamTransferableBlock() + ? TransferableBlockTestUtils.getEndOfStreamTransferableBlock(0) : new TransferableBlock(_rowsMap.get(operatorName).get(_invocationCount++), _operatorSchemaMap.get(operatorName), DataBlock.Type.ROW); } diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/MultistageEngineQuickStart.java b/pinot-tools/src/main/java/org/apache/pinot/tools/MultistageEngineQuickStart.java index 9b6b714fe0c3..658aa694dc81 100644 --- a/pinot-tools/src/main/java/org/apache/pinot/tools/MultistageEngineQuickStart.java +++ b/pinot-tools/src/main/java/org/apache/pinot/tools/MultistageEngineQuickStart.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import org.apache.pinot.spi.utils.CommonConstants; @@ -293,6 +294,13 @@ public String[] getDefaultBatchTableDirectories() { return MULTI_STAGE_TABLE_DIRECTORIES; } + @Override + protected Map getConfigOverrides() { + Map configOverrides = new HashMap<>(); + configOverrides.put(CommonConstants.Server.CONFIG_OF_ENABLE_THREAD_CPU_TIME_MEASUREMENT, true); + return configOverrides; + } + @Override protected int getNumQuickstartRunnerServers() { return 3; diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/RealtimeQuickStart.java b/pinot-tools/src/main/java/org/apache/pinot/tools/RealtimeQuickStart.java index 67bea9c6bd09..89168d160b8d 100644 --- a/pinot-tools/src/main/java/org/apache/pinot/tools/RealtimeQuickStart.java +++ b/pinot-tools/src/main/java/org/apache/pinot/tools/RealtimeQuickStart.java @@ -22,8 +22,11 @@ import java.io.File; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; +import java.util.Map; import org.apache.commons.io.FileUtils; +import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.tools.Quickstart.Color; import org.apache.pinot.tools.admin.PinotAdministrator; import org.apache.pinot.tools.admin.command.QuickstartRunner; @@ -43,6 +46,13 @@ public static void main(String[] args) PinotAdministrator.main(arguments.toArray(new String[arguments.size()])); } + @Override + protected Map getConfigOverrides() { + Map configOverrides = new HashMap<>(); + configOverrides.put(CommonConstants.Server.CONFIG_OF_ENABLE_THREAD_CPU_TIME_MEASUREMENT, true); + return configOverrides; + } + @Override public void runSampleQueries(QuickstartRunner runner) throws Exception {