Jackie-Jiang commented on a change in pull request #6710: URL: https://github.com/apache/incubator-pinot/pull/6710#discussion_r606007420
########## File path: pinot-core/src/main/java/org/apache/pinot/core/common/datatable/BaseDataTable.java ########## @@ -0,0 +1,283 @@ +/** + * 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 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 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.spi.utils.ByteArray; +import org.apache.pinot.spi.utils.BytesUtils; + +import static org.apache.pinot.core.common.datatable.DataTableUtils.decodeString; Review comment: (Code style) Avoid using static import. Same for other non-test files ########## File path: pinot-common/src/main/java/org/apache/pinot/common/utils/DataTable.java ########## @@ -80,4 +82,76 @@ double[] getDoubleArray(int rowId, int colId); String[] getStringArray(int rowId, int colId); + + enum MetadataValueType { + INT, LONG, STRING + } + + /* The MetadataKey is used in V3, where we present metadata as Map<MetadataKey, String> + * ATTENTION: + * - Don't change existing keys. + * - Don't remove existing keys. + * - Always add new keys to the end. + * Otherwise, backward compatibility will be broken. + */ + enum MetadataKey { + UNKNOWN("unknown", MetadataValueType.STRING), + TABLE("table", MetadataValueType.STRING), // NOTE: this key is only used in PrioritySchedulerTest + NUM_DOCS_SCANNED("numDocsScanned", MetadataValueType.LONG), + NUM_ENTRIES_SCANNED_IN_FILTER("numEntriesScannedInFilter", MetadataValueType.LONG), + NUM_ENTRIES_SCANNED_POST_FILTER("numEntriesScannedPostFilter", MetadataValueType.LONG), + NUM_SEGMENTS_QUERIED("numSegmentsQueried", MetadataValueType.INT), + NUM_SEGMENTS_PROCESSED("numSegmentsProcessed", MetadataValueType.INT), + NUM_SEGMENTS_MATCHED("numSegmentsMatched", MetadataValueType.INT), + NUM_CONSUMING_SEGMENTS_PROCESSED("numConsumingSegmentsProcessed", MetadataValueType.INT), + MIN_CONSUMING_FRESHNESS_TIME_MS("minConsumingFreshnessTimeMs", MetadataValueType.LONG), + TOTAL_DOCS("totalDocs", MetadataValueType.LONG), + NUM_GROUPS_LIMIT_REACHED("numGroupsLimitReached", MetadataValueType.STRING), + TIME_USED_MS("timeUsedMs", MetadataValueType.LONG), + TRACE_INFO("traceInfo", MetadataValueType.STRING), + REQUEST_ID("requestId", MetadataValueType.LONG), + NUM_RESIZES("numResizes", MetadataValueType.INT), + RESIZE_TIME_MS("resizeTimeMs", MetadataValueType.LONG), + THREAD_CPU_TIME_NS("threadCpuTimeNs", MetadataValueType.LONG); + + private static final Map<String, MetadataKey> _nameToEnumKeyMap = new HashMap<>(); + private final String _name; + private final MetadataValueType _valueType; + + MetadataKey(String name, MetadataValueType valueType) { + this._name = name; + this._valueType = valueType; Review comment: (nit) ```suggestion _name = name; _valueType = valueType; ``` ########## File path: pinot-core/src/main/java/org/apache/pinot/core/common/datatable/BaseDataTable.java ########## @@ -0,0 +1,283 @@ +/** + * 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 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 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.spi.utils.ByteArray; +import org.apache.pinot.spi.utils.BytesUtils; + +import static org.apache.pinot.core.common.datatable.DataTableUtils.decodeString; + + +/** + * Base implementation of the DataTable interface. + */ +public abstract class BaseDataTable implements DataTable { + protected int _numRows; + protected int _numColumns; + protected DataSchema _dataSchema; + protected int[] _columnOffsets; + protected int _rowSizeInBytes; + protected Map<String, Map<Integer, String>> _dictionaryMap; + protected byte[] _fixedSizeDataBytes; + protected ByteBuffer _fixedSizeData; + protected byte[] _variableSizeDataBytes; + protected ByteBuffer _variableSizeData; + protected Map<String, String> _metadata; + + public BaseDataTable(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); + _metadata = new HashMap<>(); + } + + /** + * Construct empty data table. (Server side) + */ + public BaseDataTable() { + _numRows = 0; + _numColumns = 0; + _dataSchema = null; + _columnOffsets = null; + _rowSizeInBytes = 0; + _dictionaryMap = null; + _fixedSizeDataBytes = null; + _fixedSizeData = null; + _variableSizeDataBytes = null; + _variableSizeData = null; + _metadata = new HashMap<>(); + } + + /** + * Helper method to serialize dictionary map. + */ + protected byte[] serializeDictionaryMap(Map<String, Map<Integer, String>> dictionaryMap) + throws IOException { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); + + dataOutputStream.writeInt(dictionaryMap.size()); + for (Map.Entry<String, Map<Integer, String>> dictionaryMapEntry : dictionaryMap.entrySet()) { + String columnName = dictionaryMapEntry.getKey(); + Map<Integer, String> dictionary = dictionaryMapEntry.getValue(); + byte[] bytes = StringUtil.encodeUtf8(columnName); + dataOutputStream.writeInt(bytes.length); + dataOutputStream.write(bytes); + dataOutputStream.writeInt(dictionary.size()); + + for (Map.Entry<Integer, String> dictionaryEntry : dictionary.entrySet()) { + dataOutputStream.writeInt(dictionaryEntry.getKey()); + byte[] valueBytes = StringUtil.encodeUtf8(dictionaryEntry.getValue()); + dataOutputStream.writeInt(valueBytes.length); + dataOutputStream.write(valueBytes); + } + } + + return byteArrayOutputStream.toByteArray(); + } + + /** + * Helper method to deserialize dictionary map. + */ + protected Map<String, Map<Integer, String>> deserializeDictionaryMap(byte[] bytes) + throws IOException { + try (ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(bytes); + DataInputStream dataInputStream = new DataInputStream(byteArrayInputStream)) { + int numDictionaries = dataInputStream.readInt(); + Map<String, Map<Integer, String>> dictionaryMap = new HashMap<>(numDictionaries); + + for (int i = 0; i < numDictionaries; i++) { + String column = decodeString(dataInputStream); + int dictionarySize = dataInputStream.readInt(); + Map<Integer, String> dictionary = new HashMap<>(dictionarySize); + for (int j = 0; j < dictionarySize; j++) { + int key = dataInputStream.readInt(); + String value = decodeString(dataInputStream); + dictionary.put(key, value); + } + dictionaryMap.put(column, dictionary); + } + + return dictionaryMap; + } + } + + public Map<String, String> getMetadata() { Review comment: Put override annotation over these classes that implements the interface ########## File path: pinot-common/src/main/java/org/apache/pinot/common/utils/DataTable.java ########## @@ -80,4 +82,76 @@ double[] getDoubleArray(int rowId, int colId); String[] getStringArray(int rowId, int colId); + + enum MetadataValueType { + INT, LONG, STRING + } + + /* The MetadataKey is used in V3, where we present metadata as Map<MetadataKey, String> + * ATTENTION: + * - Don't change existing keys. + * - Don't remove existing keys. + * - Always add new keys to the end. + * Otherwise, backward compatibility will be broken. + */ + enum MetadataKey { + UNKNOWN("unknown", MetadataValueType.STRING), + TABLE("table", MetadataValueType.STRING), // NOTE: this key is only used in PrioritySchedulerTest + NUM_DOCS_SCANNED("numDocsScanned", MetadataValueType.LONG), + NUM_ENTRIES_SCANNED_IN_FILTER("numEntriesScannedInFilter", MetadataValueType.LONG), + NUM_ENTRIES_SCANNED_POST_FILTER("numEntriesScannedPostFilter", MetadataValueType.LONG), + NUM_SEGMENTS_QUERIED("numSegmentsQueried", MetadataValueType.INT), + NUM_SEGMENTS_PROCESSED("numSegmentsProcessed", MetadataValueType.INT), + NUM_SEGMENTS_MATCHED("numSegmentsMatched", MetadataValueType.INT), + NUM_CONSUMING_SEGMENTS_PROCESSED("numConsumingSegmentsProcessed", MetadataValueType.INT), + MIN_CONSUMING_FRESHNESS_TIME_MS("minConsumingFreshnessTimeMs", MetadataValueType.LONG), + TOTAL_DOCS("totalDocs", MetadataValueType.LONG), + NUM_GROUPS_LIMIT_REACHED("numGroupsLimitReached", MetadataValueType.STRING), + TIME_USED_MS("timeUsedMs", MetadataValueType.LONG), + TRACE_INFO("traceInfo", MetadataValueType.STRING), + REQUEST_ID("requestId", MetadataValueType.LONG), + NUM_RESIZES("numResizes", MetadataValueType.INT), + RESIZE_TIME_MS("resizeTimeMs", MetadataValueType.LONG), + THREAD_CPU_TIME_NS("threadCpuTimeNs", MetadataValueType.LONG); + + private static final Map<String, MetadataKey> _nameToEnumKeyMap = new HashMap<>(); + private final String _name; + private final MetadataValueType _valueType; + + MetadataKey(String name, MetadataValueType valueType) { + this._name = name; + this._valueType = valueType; + } + + // getByOrdinal returns an optional enum key for a given ordinal or null if the key does not exist. + public static MetadataKey getByOrdinal(int ordinal) { + if (ordinal >= MetadataKey.values().length) { + return null; + } + return MetadataKey.values()[ordinal]; + } + + // getByName returns an enum key for a given name or null if the key does not exist. + public static MetadataKey getByName(String name) { + return _nameToEnumKeyMap.getOrDefault(name, null); + } Review comment: ```suggestion @Nullable public static MetadataKey getByName(String name) { return _nameToEnumKeyMap.get(name); } ``` ########## File path: pinot-common/src/main/java/org/apache/pinot/common/utils/DataTable.java ########## @@ -80,4 +82,76 @@ double[] getDoubleArray(int rowId, int colId); String[] getStringArray(int rowId, int colId); + + enum MetadataValueType { + INT, LONG, STRING + } + + /* The MetadataKey is used in V3, where we present metadata as Map<MetadataKey, String> + * ATTENTION: + * - Don't change existing keys. + * - Don't remove existing keys. + * - Always add new keys to the end. + * Otherwise, backward compatibility will be broken. + */ + enum MetadataKey { + UNKNOWN("unknown", MetadataValueType.STRING), + TABLE("table", MetadataValueType.STRING), // NOTE: this key is only used in PrioritySchedulerTest + NUM_DOCS_SCANNED("numDocsScanned", MetadataValueType.LONG), + NUM_ENTRIES_SCANNED_IN_FILTER("numEntriesScannedInFilter", MetadataValueType.LONG), + NUM_ENTRIES_SCANNED_POST_FILTER("numEntriesScannedPostFilter", MetadataValueType.LONG), + NUM_SEGMENTS_QUERIED("numSegmentsQueried", MetadataValueType.INT), + NUM_SEGMENTS_PROCESSED("numSegmentsProcessed", MetadataValueType.INT), + NUM_SEGMENTS_MATCHED("numSegmentsMatched", MetadataValueType.INT), + NUM_CONSUMING_SEGMENTS_PROCESSED("numConsumingSegmentsProcessed", MetadataValueType.INT), + MIN_CONSUMING_FRESHNESS_TIME_MS("minConsumingFreshnessTimeMs", MetadataValueType.LONG), + TOTAL_DOCS("totalDocs", MetadataValueType.LONG), + NUM_GROUPS_LIMIT_REACHED("numGroupsLimitReached", MetadataValueType.STRING), + TIME_USED_MS("timeUsedMs", MetadataValueType.LONG), + TRACE_INFO("traceInfo", MetadataValueType.STRING), + REQUEST_ID("requestId", MetadataValueType.LONG), + NUM_RESIZES("numResizes", MetadataValueType.INT), + RESIZE_TIME_MS("resizeTimeMs", MetadataValueType.LONG), + THREAD_CPU_TIME_NS("threadCpuTimeNs", MetadataValueType.LONG); + + private static final Map<String, MetadataKey> _nameToEnumKeyMap = new HashMap<>(); + private final String _name; + private final MetadataValueType _valueType; + + MetadataKey(String name, MetadataValueType valueType) { + this._name = name; + this._valueType = valueType; + } + + // getByOrdinal returns an optional enum key for a given ordinal or null if the key does not exist. + public static MetadataKey getByOrdinal(int ordinal) { Review comment: ```suggestion @Nullable public static MetadataKey getByOrdinal(int ordinal) { ``` ########## File path: pinot-core/src/main/java/org/apache/pinot/core/common/datatable/BaseDataTable.java ########## @@ -0,0 +1,283 @@ +/** + * 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 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 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.spi.utils.ByteArray; +import org.apache.pinot.spi.utils.BytesUtils; + +import static org.apache.pinot.core.common.datatable.DataTableUtils.decodeString; + + +/** + * Base implementation of the DataTable interface. + */ +public abstract class BaseDataTable implements DataTable { + protected int _numRows; + protected int _numColumns; + protected DataSchema _dataSchema; + protected int[] _columnOffsets; + protected int _rowSizeInBytes; + protected Map<String, Map<Integer, String>> _dictionaryMap; + protected byte[] _fixedSizeDataBytes; + protected ByteBuffer _fixedSizeData; + protected byte[] _variableSizeDataBytes; + protected ByteBuffer _variableSizeData; + protected Map<String, String> _metadata; + + public BaseDataTable(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); + _metadata = new HashMap<>(); + } + + /** + * Construct empty data table. (Server side) + */ + public BaseDataTable() { + _numRows = 0; + _numColumns = 0; + _dataSchema = null; + _columnOffsets = null; + _rowSizeInBytes = 0; + _dictionaryMap = null; + _fixedSizeDataBytes = null; + _fixedSizeData = null; + _variableSizeDataBytes = null; + _variableSizeData = null; + _metadata = new HashMap<>(); + } + + /** + * Helper method to serialize dictionary map. + */ + protected byte[] serializeDictionaryMap(Map<String, Map<Integer, String>> dictionaryMap) Review comment: No need to have the argument. It always serializes the `_dictionaryMap` -- 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]
