siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465954429



##########
File path: 
pinot-controller/src/main/java/org/apache/pinot/controller/recommender/io/InputManager.java
##########
@@ -0,0 +1,519 @@
+/**
+ * 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.controller.recommender.io;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonSetter;
+import com.fasterxml.jackson.annotation.Nulls;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import 
org.apache.pinot.controller.recommender.io.exceptions.InvalidInputException;
+import org.apache.pinot.controller.recommender.io.metadata.ColumnMetaData;
+import org.apache.pinot.controller.recommender.io.metadata.SchemaWithMetaData;
+import org.apache.pinot.controller.recommender.rules.RulesToExecute;
+import 
org.apache.pinot.controller.recommender.rules.io.params.BloomFilterRuleParams;
+import 
org.apache.pinot.controller.recommender.rules.io.params.FlagQueryRuleParams;
+import 
org.apache.pinot.controller.recommender.rules.io.params.InvertedSortedIndexJointRuleParams;
+import 
org.apache.pinot.controller.recommender.rules.io.params.NoDictionaryOnHeapDictionaryJointRuleParams;
+import 
org.apache.pinot.controller.recommender.rules.io.params.PartitionRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.apache.pinot.spi.data.DimensionFieldSpec;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.MetricFieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.lang.Math.max;
+import static 
org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.*;
+
+
+@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE)
+public class InputManager {
+  private final Logger LOGGER = LoggerFactory.getLogger(InputManager.class);
+
+  /******************************Deserialized from input 
json*********************************/
+  // Basic input fields
+  public RulesToExecute _rulesToExecute = new RulesToExecute(); // dictates 
which rules to execute
+  public Schema _schema = new Schema();
+  public SchemaWithMetaData _schemaWithMetaData = new SchemaWithMetaData();
+
+  public String _queryType = SQL; // SQL or PQL
+  public long _qps = DEFAULT_QPS;
+  public Map<String, Double> _queryWeightMap = new HashMap<>(); // 
{"queryString":"queryWeight"}
+  public String _tableType = OFFLINE;
+  public long _numMessagesPerSec = DEFAULT_NUM_MSG_PER_SEC; // messages per 
sec for kafka to consume
+  public long _numRecordsPerPush = DEFAULT_NUM_RECORDS_PER_PUSH; // records 
per push for offline part of a table
+  public long _latencySLA = DEFAULT_LATENCY_SLA; // latency sla in ms
+  public int _numKafkaPartitions = DEFAULT_NUM_KAFKA_PARTITIONS;
+
+  // The parameters of rules
+  public PartitionRuleParams _partitionRuleParams = new PartitionRuleParams();
+  public InvertedSortedIndexJointRuleParams 
_invertedSortedIndexJointRuleParams =
+      new InvertedSortedIndexJointRuleParams();
+  public BloomFilterRuleParams _bloomFilterRuleParams = new 
BloomFilterRuleParams();
+  public NoDictionaryOnHeapDictionaryJointRuleParams 
_noDictionaryOnHeapDictionaryJointRuleParams =
+      new NoDictionaryOnHeapDictionaryJointRuleParams();
+  public FlagQueryRuleParams _flagQueryRuleParams = new FlagQueryRuleParams();
+
+  // For forward compatibility: 1. dev/sre to overwrite field(s) 2. 
incremental recommendation on existing/staging tables
+  public ConfigManager _overWrittenConfigs = new ConfigManager();
+
+  /******************************Ignored by 
deserializer****************************************/
+  public Map<String, ColumnMetaData> _metaDataMap = new HashMap<>(); // meta 
data per column, complement to schema
+  long _sizePerRecord = 0;
+  Map<String, FieldSpec.DataType> _colnameFieldTypeMap = new HashMap<>();
+  Set<String> _dimNames = null;
+  Set<String> _metricNames = null;
+  Set<String> _dateTimeNames = null;
+  Set<String> _dimNamesInveredSortedIndexApplicable = null;
+  Map<String, Integer> _colNameToIntMap = null;
+  String[] _intToColNameMap = null;
+  Map<FieldSpec.DataType, Integer> _dataTypeSizeMap = new 
HashMap<FieldSpec.DataType, Integer>() {{
+    put(FieldSpec.DataType.INT, DEFAULT_INT_SIZE);
+    put(FieldSpec.DataType.LONG, DEFAULT_LONG_SIZE);
+    put(FieldSpec.DataType.FLOAT, DEFAULT_FLOAT_SIZE);
+    put(FieldSpec.DataType.DOUBLE, DEFAULT_DOUBLE_SIZE);
+    put(FieldSpec.DataType.BYTES, DEFAULT_BYTE_SIZE);
+    put(FieldSpec.DataType.STRING, DEFAULT_CHAR_SIZE);
+    put(null, DEFAULT_NULL_SIZE);
+  }};
+
+  /**
+   * Process the dependencies incurred by overwritten configs.
+   * E.g. we will subtract the dimensions with overwritten indices from 
_dimNames to get _dimNamesIndexApplicable
+   * This ensures we do not recommend indices on those dimensions
+   */
+  public void init()
+      throws InvalidInputException {
+    LOGGER.info("Preprocessing Input:");
+    reorderDimsAndBuildMap();
+    registerColnameFieldType();
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setFlagQueryRuleParams(FlagQueryRuleParams flagQueryRuleParams) {
+    _flagQueryRuleParams = flagQueryRuleParams;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setNumKafkaPartitions(int numKafkaPartitions) {
+    _numKafkaPartitions = numKafkaPartitions;
+  }
+
+  @JsonSetter(value = "queriesWithWeights", nulls = Nulls.SKIP)
+  public void setQueryWeightMap(Map<String, Double> queryWeightMap) {
+    _queryWeightMap = queryWeightMap;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setNoDictionaryOnHeapDictionaryJointRuleParams(
+      NoDictionaryOnHeapDictionaryJointRuleParams 
noDictionaryOnHeapDictionaryJointRuleParams) {
+    _noDictionaryOnHeapDictionaryJointRuleParams = 
noDictionaryOnHeapDictionaryJointRuleParams;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setLatencySLA(int latencySLA) {
+    _latencySLA = latencySLA;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setQps(long qps) {
+    _qps = qps;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setBloomFilterRuleParams(BloomFilterRuleParams 
bloomFilterRuleParams) {
+    _bloomFilterRuleParams = bloomFilterRuleParams;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setPartitionRuleParams(PartitionRuleParams partitionRuleParams) {
+    _partitionRuleParams = partitionRuleParams;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setTableType(String tableType) {
+    _tableType = tableType;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setNumMessagesPerSec(long numMessagesPerSec) {
+    _numMessagesPerSec = numMessagesPerSec;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setNumRecordsPerPush(long numRecordsPerPush) {
+    _numRecordsPerPush = numRecordsPerPush;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setRulesToExecute(RulesToExecute rulesToExecute) {
+    _rulesToExecute = rulesToExecute;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setSchema(JsonNode jsonNode)
+      throws IOException {
+    ObjectReader reader = new ObjectMapper().readerFor(Schema.class);
+    this._schema=reader.readValue(jsonNode);
+    reader = new ObjectMapper().readerFor(SchemaWithMetaData.class);
+    this._schemaWithMetaData=reader.readValue(jsonNode);
+    _schemaWithMetaData.getDimensionFieldSpecs()
+        .forEach(columnMetaData -> 
{_metaDataMap.put(columnMetaData.getName(),columnMetaData);});
+    _schemaWithMetaData.getMetricFieldSpecs()
+        .forEach(columnMetaData -> 
{_metaDataMap.put(columnMetaData.getName(),columnMetaData);});
+    _schemaWithMetaData.getDateTimeFieldSpecs()
+        .forEach(columnMetaData -> 
{_metaDataMap.put(columnMetaData.getName(),columnMetaData);});
+    _metaDataMap.put(_schemaWithMetaData.getTimeFieldSpec().getName(), 
_schemaWithMetaData.getTimeFieldSpec());
+  }
+
+  @JsonIgnore
+  public void setMetaDataMap(Map<String, ColumnMetaData> metaDataMap) {
+    _metaDataMap = metaDataMap;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setQueryType(String queryType) {
+    _queryType = queryType;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setInvertedSortedIndexJointRuleParams(
+      InvertedSortedIndexJointRuleParams invertedSortedIndexJointRuleParams) {
+    _invertedSortedIndexJointRuleParams = invertedSortedIndexJointRuleParams;
+  }
+
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setOverWrittenConfigs(ConfigManager overWrittenConfigs) {
+    _overWrittenConfigs = overWrittenConfigs;
+  }
+
+
+  public FlagQueryRuleParams getFlagQueryRuleParams() {
+    return _flagQueryRuleParams;
+  }
+
+
+  public FieldSpec.DataType getFieldType(String colName){
+    return _colnameFieldTypeMap.getOrDefault(colName, null);
+  }
+
+  public Map<String, Integer> getColNameToIntMap() {
+    return _colNameToIntMap;
+  }
+
+  /**
+   * Get the number of dimensions we can apply indices on.
+   * @return total number of dimensions minus number of dimensions with 
overwritten indices
+   */
+  public int getNumDimsInvertedSortedApplicable() {
+    return _dimNamesInveredSortedIndexApplicable.size();
+  }
+
+  public NoDictionaryOnHeapDictionaryJointRuleParams 
getNoDictionaryOnHeapDictionaryJointRuleParams() {
+    return _noDictionaryOnHeapDictionaryJointRuleParams;
+  }
+
+  public int getNumDims() {
+    return _dimNames.size();
+  }
+
+  public int getNumCols() {
+    return _colNameToIntMap.size();
+  }
+
+  //TODO: Currently Pinot is using only ONE time column specified by 
TimeFieldSpec
+  //TODO: Change the implementation after the new schema with multiple 
_dateTimeNames is in use
+  public String getTimeCol() {
+    return _schema.getTimeFieldSpec().getName();
+  }
+
+  public Set<String> getColNamesNoDictionary() {
+    return _overWrittenConfigs.getIndexConfig().getNoDictionaryColumns();
+  }
+
+  public long getLatencySLA() {
+    return _latencySLA;
+  }
+
+  public long getQps() {
+    return _qps;
+  }
+
+  public BloomFilterRuleParams getBloomFilterRuleParams() {
+    return _bloomFilterRuleParams;
+  }
+
+  public PartitionRuleParams getPartitionRuleParams() {
+    return _partitionRuleParams;
+  }
+
+  public String getTableType() {
+    return _tableType;
+  }
+
+  public Map<String, Double> getQueryWeightMap() {
+    return _queryWeightMap;
+  }
+
+  public long getNumMessagesPerSec() {
+    return _numMessagesPerSec;
+  }
+
+  public long getNumRecordsPerPush() {
+    return _numRecordsPerPush;
+  }
+
+  public RulesToExecute getRulesToExecute() {
+    return _rulesToExecute;
+  }
+
+  public Schema getSchema() {
+    return _schema;
+  }
+
+  @JsonIgnore
+  public Map<String, ColumnMetaData> getMetaDataMap() {
+    return _metaDataMap;
+  }
+
+  public String getQueryType() {
+    return _queryType;
+  }
+
+  public InvertedSortedIndexJointRuleParams 
getInvertedSortedIndexJointRuleParams() {
+    return _invertedSortedIndexJointRuleParams;
+  }
+
+  public ConfigManager getOverWrittenConfigs() {
+    return _overWrittenConfigs;
+  }
+
+  public long getSizePerRecord() {
+    return _sizePerRecord;
+  }
+
+  public double getCardinality(String columnName) {
+    return max(_metaDataMap.getOrDefault(columnName, new 
ColumnMetaData()).getCardinality(), MIN_CARDINALITY);
+  }
+
+  public double getNumValuesPerEntry(String columnName) {
+    return _metaDataMap.getOrDefault(columnName, new 
ColumnMetaData()).getNumValuesPerEntry();
+  }
+
+  public int getAverageDataLen(String columnName) {
+    return _metaDataMap.getOrDefault(columnName, new 
ColumnMetaData()).getAverageLength();
+  }
+
+  public int getNumKafkaPartitions() {
+    return _numKafkaPartitions;
+  }
+
+  public boolean isIndexableDim(String colName) {
+    return _dimNamesInveredSortedIndexApplicable.contains(colName);
+  }
+
+  public boolean isSingleValueColumn(String colName){
+    ColumnMetaData columnMetaData = _metaDataMap.getOrDefault(colName, new 
ColumnMetaData());
+    return columnMetaData.isSingleValueField() && 
(columnMetaData.getNumValuesPerEntry() < DEFAULT_AVERAGE_NUM_VALUES_PER_ENTRY + 
EPSILON);
+  }
+
+  /**
+   * Map a index-applicable dimension name to an 
0<=integer<getNumDimsInvertedSortedApplicable,
+   * to be used with {@link FixedLenBitset}
+   * @param colName a dimension with no overwritten index
+   * @return a unique integer id
+   */
+  public int colNameToInt(String colName) {
+    return _colNameToIntMap.getOrDefault(colName, NO_SUCH_COL);
+  }
+
+  /**
+   * A reverse process of colNameToInt
+   * @param colID a unique integer id
+   * @return column name
+   */
+  public String intToColName(int colID) {
+    return _intToColNameMap[colID];
+  }
+
+  /**
+   * Test if colName is a valid dimension name
+   */
+  public boolean isDim(String colName) {
+    return _dimNames.contains(colName);
+  }
+
+  public boolean isDateTime(String colName) {
+    return _schema.getTimeFieldSpec().getName().equals(colName);
+  }
+
+  public void registerColnameFieldType() { // create a map from colname to 
data type
+    for (DimensionFieldSpec dimensionFieldSpec : 
_schema.getDimensionFieldSpecs()) {
+      _colnameFieldTypeMap.put(dimensionFieldSpec.getName(), 
dimensionFieldSpec.getDataType());
+    }
+    for (MetricFieldSpec metricFieldSpec : _schema.getMetricFieldSpecs()) {
+      _colnameFieldTypeMap.put(metricFieldSpec.getName(), 
metricFieldSpec.getDataType());
+    }
+    //TODO: add support for multiple getDateTimeFieldSpecs
+    _colnameFieldTypeMap.put(_schema.getTimeFieldSpec().getName(), 
_schema.getTimeFieldSpec().getDataType());
+  }
+
+  public void estimateSizePerRecord() {
+    for (String colName : _colnameFieldTypeMap.keySet()) {
+      _sizePerRecord += getColDataSizeWithDictionary(colName);
+      LOGGER.debug("{} {}",colName, getColDataSizeWithDictionary(colName));
+    }
+    LOGGER.info("*Estimated size per record {} bytes", _sizePerRecord);
+  }
+
+  public long getColDataSizeWithoutDictionary(String colName) {
+    //TODO: implement this after the complex is supported
+    FieldSpec.DataType dataType = getFieldType(colName);
+    if (dataType == FieldSpec.DataType.STRUCT || dataType == 
FieldSpec.DataType.MAP
+        || dataType == FieldSpec.DataType.LIST) {
+      return 0;
+    } else {
+      if (dataType == FieldSpec.DataType.BYTES || dataType == 
FieldSpec.DataType.STRING) {
+        return _dataTypeSizeMap.get(dataType) * getAverageDataLen(colName);
+      } else {
+        return _dataTypeSizeMap.get(dataType);
+      }
+    }
+  }
+
+  public long getColDataSizeWithDictionary(String colName) {
+    //TODO: implement this after the complex is supported
+    FieldSpec.DataType dataType = getFieldType(colName);
+    int numValuesPerEntry = (int) Math.ceil(getNumValuesPerEntry(colName));
+    LOGGER.trace("{} {}", colName, numValuesPerEntry);
+    if (dataType == FieldSpec.DataType.STRUCT || dataType == 
FieldSpec.DataType.MAP
+        || dataType == FieldSpec.DataType.LIST) {
+      return 0;
+    } else if 
(!_overWrittenConfigs.getIndexConfig().getNoDictionaryColumns().contains(colName))
 { // has dictionary
+      return getBitCompressedDataSize(colName) * numValuesPerEntry;
+    } else { // no dictionary
+      if (dataType == FieldSpec.DataType.BYTES || dataType == 
FieldSpec.DataType.STRING) {
+        return _dataTypeSizeMap.get(dataType) * numValuesPerEntry * 
getAverageDataLen(colName);
+      } else {
+        return _dataTypeSizeMap.get(dataType) * numValuesPerEntry;
+      }
+    }
+  }
+
+  public int getBitCompressedDataSize(String colName) {
+    return max((int) Math.ceil(Math.log(getCardinality(colName)) / (8 * 
Math.log(2))), 1);
+  }
+
+  //
+  public long getDictionarySize(String colName) {

Review comment:
       This function is only computing the size of dictionary right? We should 
not include the size of bit compressed forward index




----------------------------------------------------------------
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:
us...@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org

Reply via email to