siddharthteotia commented on a change in pull request #6710: URL: https://github.com/apache/incubator-pinot/pull/6710#discussion_r603691048
########## File path: pinot-core/src/main/java/org/apache/pinot/core/common/datatable/DataTableImplV3.java ########## @@ -0,0 +1,594 @@ +/** + * 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.core.common.datatable; + +import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import org.apache.pinot.common.response.ProcessingException; +import org.apache.pinot.common.utils.DataSchema; +import org.apache.pinot.common.utils.DataTable; +import org.apache.pinot.common.utils.StringUtil; +import org.apache.pinot.core.common.ObjectSerDeUtils; +import org.apache.pinot.core.query.request.context.ThreadTimer; +import org.apache.pinot.spi.utils.ByteArray; +import org.apache.pinot.spi.utils.BytesUtils; + +import static org.apache.pinot.common.utils.DataTable.MetadataKeys.EXECUTION_THREAD_CPU_TIME_NS; +import static org.apache.pinot.core.common.datatable.DataTableUtils.*; + + +/** + * Datatable V3 implementation. + * The layout of serialized V3 datatable looks like: + * +-----------------------------------------------+ + * | 13 bytes of header: | + * | VERSION | + * | NUM_ROWS | + * | NUM_COLUMNS | + * | EXCEPTIONS SECTION START OFFSET | + * | EXCEPTIONS SECTION LENGTH | + * | DICTIONARY_MAP SECTION START OFFSET | + * | DICTIONARY_MAP SECTION LENGTH | + * | DATA_SCHEMA SECTION START OFFSET | + * | DATA_SCHEMA SECTION LENGTH | + * | FIXED_SIZE_DATA SECTION START OFFSET | + * | FIXED_SIZE_DATA SECTION LENGTH | + * | VARIABLE_SIZE_DATA SECTION START OFFSET | + * | VARIABLE_SIZE_DATA SECTION LENGTH | + * +-----------------------------------------------+ + * | EXCEPTIONS SECTION | + * +-----------------------------------------------+ + * | DICTIONARY_MAP SECTION | + * +-----------------------------------------------+ + * | DATA_SCHEMA SECTION | + * +-----------------------------------------------+ + * | FIXED_SIZE_DATA SECTION | + * +-----------------------------------------------+ + * | VARIABLE_SIZE_DATA SECTION | + * +-----------------------------------------------+ + * | METADATA LENGTH | + * | METADATA SECTION | + * +-----------------------------------------------+ + */ +public class DataTableImplV3 implements DataTable { + private static final int VERSION = 3; + private static final int HEADER_SIZE = Integer.BYTES * 13; + + private final int _numRows; + private final int _numColumns; + private final DataSchema _dataSchema; + private final int[] _columnOffsets; + private final int _rowSizeInBytes; + private final Map<String, Map<Integer, String>> _dictionaryMap; + private final byte[] _fixedSizeDataBytes; + private final ByteBuffer _fixedSizeData; + private final byte[] _variableSizeDataBytes; + private final ByteBuffer _variableSizeData; + // _exceptions stores exceptions as a map of errorCode->errorMessage + private final Map<Integer, String> _exceptions; + private final Map<MetadataKeys, String> _metadata; + // _metadataV2 is just a V2 presentation of _metadata, we copy KV pairs between _metadata and _metadataV2 during + // serialization/deserialization. This is because V2 API of getMetadata returns a Map<String, String> and there are + // a lot of existing code using string as key to access metadata. + // TODO: remove this and change all metadata accessing code use MetadataKeys. + private final Map<String, String> _metadataV2; + + /** + * Construct data table with results. (Server side) + */ + public DataTableImplV3(int numRows, DataSchema dataSchema, Map<String, Map<Integer, String>> dictionaryMap, + byte[] fixedSizeDataBytes, byte[] variableSizeDataBytes) { + _numRows = numRows; + _numColumns = dataSchema.size(); + _dataSchema = dataSchema; + _columnOffsets = new int[_numColumns]; + _rowSizeInBytes = DataTableUtils.computeColumnOffsets(dataSchema, _columnOffsets); + _dictionaryMap = dictionaryMap; + _fixedSizeDataBytes = fixedSizeDataBytes; + _fixedSizeData = ByteBuffer.wrap(fixedSizeDataBytes); + _variableSizeDataBytes = variableSizeDataBytes; + _variableSizeData = ByteBuffer.wrap(variableSizeDataBytes); + _exceptions = new HashMap<>(); + _metadata = new HashMap<>(); + _metadataV2 = new HashMap<>(); + } + + /** + * Construct empty data table. (Server side) + */ + public DataTableImplV3() { + _numRows = 0; + _numColumns = 0; + _dataSchema = null; + _columnOffsets = null; + _rowSizeInBytes = 0; + _dictionaryMap = null; + _fixedSizeDataBytes = null; + _fixedSizeData = null; + _variableSizeDataBytes = null; + _variableSizeData = null; + _exceptions = new HashMap<>(); + _metadata = new HashMap<>(); + _metadataV2 = new HashMap<>(); + } + + /** + * Construct data table from byte array. (broker side) + */ + public DataTableImplV3(ByteBuffer byteBuffer) + throws IOException { + // Read header. + _numRows = byteBuffer.getInt(); + _numColumns = byteBuffer.getInt(); + int exceptionsStart = byteBuffer.getInt(); + int exceptionsLength = byteBuffer.getInt(); + int dictionaryMapStart = byteBuffer.getInt(); + int dictionaryMapLength = byteBuffer.getInt(); + int dataSchemaStart = byteBuffer.getInt(); + int dataSchemaLength = byteBuffer.getInt(); + int fixedSizeDataStart = byteBuffer.getInt(); + int fixedSizeDataLength = byteBuffer.getInt(); + int variableSizeDataStart = byteBuffer.getInt(); + int variableSizeDataLength = byteBuffer.getInt(); + + // Read exceptions. + if (exceptionsLength != 0) { + byte[] exceptionsBytes = new byte[exceptionsLength]; + byteBuffer.position(exceptionsStart); + byteBuffer.get(exceptionsBytes); + _exceptions = deserializeExceptions(exceptionsBytes); + } else { + _exceptions = new HashMap<>(); + } + + // Read dictionary. + if (dictionaryMapLength != 0) { + byte[] dictionaryMapBytes = new byte[dictionaryMapLength]; + byteBuffer.position(dictionaryMapStart); + byteBuffer.get(dictionaryMapBytes); + _dictionaryMap = deserializeDictionaryMap(dictionaryMapBytes); + } else { + _dictionaryMap = null; + } + + // Read data schema. + if (dataSchemaLength != 0) { + byte[] schemaBytes = new byte[dataSchemaLength]; + byteBuffer.position(dataSchemaStart); + byteBuffer.get(schemaBytes); + _dataSchema = DataSchema.fromBytes(schemaBytes); + _columnOffsets = new int[_dataSchema.size()]; + _rowSizeInBytes = DataTableUtils.computeColumnOffsets(_dataSchema, _columnOffsets); + } else { + _dataSchema = null; + _columnOffsets = null; + _rowSizeInBytes = 0; + } + + // Read fixed size data. + if (fixedSizeDataLength != 0) { + _fixedSizeDataBytes = new byte[fixedSizeDataLength]; + byteBuffer.position(fixedSizeDataStart); + byteBuffer.get(_fixedSizeDataBytes); + _fixedSizeData = ByteBuffer.wrap(_fixedSizeDataBytes); + } else { + _fixedSizeDataBytes = null; + _fixedSizeData = null; + } + + // Read variable size data. + if (variableSizeDataLength != 0) { + _variableSizeDataBytes = new byte[variableSizeDataLength]; + byteBuffer.position(variableSizeDataStart); + byteBuffer.get(_variableSizeDataBytes); + _variableSizeData = ByteBuffer.wrap(_variableSizeDataBytes); + } else { + _variableSizeDataBytes = null; + _variableSizeData = null; + } + + // Read metadata. + int metadataLength = byteBuffer.getInt(); + byte[] trailerBytes = new byte[metadataLength]; + byteBuffer.get(trailerBytes); + _metadata = deserializeMetadata(trailerBytes); + + _metadataV2 = new HashMap<>(); + for (MetadataKeys key : _metadata.keySet()) { + _metadataV2.put(key.getName(), _metadata.get(key)); + } + } + + @Override + public void addException(ProcessingException processingException) { + _exceptions.put(processingException.getErrorCode(), processingException.getMessage()); + } + + @Override + public Map<Integer, String> getExceptions() { + return _exceptions; + } + + @Override + public byte[] toBytes() + throws IOException { + ThreadTimer threadTimer = new ThreadTimer(); + threadTimer.start(); + + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); + dataOutputStream.writeInt(VERSION); + dataOutputStream.writeInt(_numRows); + dataOutputStream.writeInt(_numColumns); + int dataOffset = HEADER_SIZE; + + // Write exceptions section offset(START|SIZE). + dataOutputStream.writeInt(dataOffset); + byte[] exceptionsBytes; + exceptionsBytes = serializeExceptions(); + dataOutputStream.writeInt(exceptionsBytes.length); + dataOffset += exceptionsBytes.length; + + // Write dictionary map section offset(START|SIZE). + dataOutputStream.writeInt(dataOffset); + byte[] dictionaryMapBytes = null; + if (_dictionaryMap != null) { + dictionaryMapBytes = serializeDictionaryMap(_dictionaryMap); + dataOutputStream.writeInt(dictionaryMapBytes.length); + dataOffset += dictionaryMapBytes.length; + } else { + dataOutputStream.writeInt(0); + } + + // Write data schema section offset(START|SIZE). + dataOutputStream.writeInt(dataOffset); + byte[] dataSchemaBytes = null; + if (_dataSchema != null) { + dataSchemaBytes = _dataSchema.toBytes(); + dataOutputStream.writeInt(dataSchemaBytes.length); + dataOffset += dataSchemaBytes.length; + } else { + dataOutputStream.writeInt(0); + } + + // Write fixed size data section offset(START|SIZE). + dataOutputStream.writeInt(dataOffset); + if (_fixedSizeDataBytes != null) { + dataOutputStream.writeInt(_fixedSizeDataBytes.length); + dataOffset += _fixedSizeDataBytes.length; + } else { + dataOutputStream.writeInt(0); + } + + // Write variable size data section offset(START|SIZE). + dataOutputStream.writeInt(dataOffset); + if (_variableSizeDataBytes != null) { + dataOutputStream.writeInt(_variableSizeDataBytes.length); + dataOffset += _variableSizeDataBytes.length; + } else { + dataOutputStream.writeInt(0); + } + + // Write actual data. + // Write exceptions bytes. + dataOutputStream.write(exceptionsBytes); + // Write dictionary map bytes. + if (dictionaryMapBytes != null) { + dataOutputStream.write(dictionaryMapBytes); + } + // Write data schema bytes. + if (dataSchemaBytes != null) { + dataOutputStream.write(dataSchemaBytes); + } + // Write fixed size data bytes. + if (_fixedSizeDataBytes != null) { + dataOutputStream.write(_fixedSizeDataBytes); + } + // Write variable size data bytes. + if (_variableSizeDataBytes != null) { + dataOutputStream.write(_variableSizeDataBytes); + } + + // Update the value of "executionThreadCpuTimeNs" to account data table serialization time. + long responseSerializationCpuTimeNs = threadTimer.stopAndGetThreadTimeNs(); + long executionThreadCpuTimeNs = + Long.parseLong(getMetadata().getOrDefault(EXECUTION_THREAD_CPU_TIME_NS.getName(), "0")) + responseSerializationCpuTimeNs; + getMetadata().put(EXECUTION_THREAD_CPU_TIME_NS.getName(), String.valueOf(executionThreadCpuTimeNs)); + // Copy all KV pair in _metadataV2 into _metadata + for (String key : _metadataV2.keySet()) { + Optional<MetadataKeys> opt = MetadataKeys.getByName(key); + if (!opt.isPresent()) { + continue; + } + _metadata.put(opt.get(), _metadataV2.get(key)); + } + // Write metadata length and bytes. Review comment: See the comment above. Logic of lines 323 to 330 can be moved into serializeMetadata () itself and copy won't be needed -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
