siddharthteotia commented on a change in pull request #5774: URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465793738
########## File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/RecommenderDriver.java ########## @@ -0,0 +1,76 @@ +/** + * 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; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import org.apache.pinot.controller.recommender.io.ConfigManager; +import org.apache.pinot.controller.recommender.io.exceptions.InvalidInputException; +import org.apache.pinot.controller.recommender.rules.AbstractRule; +import org.apache.pinot.controller.recommender.rules.RulesToExecute; +import org.apache.pinot.controller.recommender.io.InputManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + + +public class RecommenderDriver { Review comment: A concise javadoc would be helpful explaining the purpose of each class. No need to explain the algorithm, but bullet-list items explaining the responsibility of the class ########## File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/io/ConfigManager.java ########## @@ -0,0 +1,54 @@ +/** + * 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.JsonSetter; +import com.fasterxml.jackson.annotation.Nulls; +import org.apache.pinot.controller.recommender.rules.io.FlaggedQueries; +import org.apache.pinot.controller.recommender.rules.io.configs.IndexConfig; +import org.apache.pinot.controller.recommender.rules.io.configs.PartitionConfig; + + +public class ConfigManager { Review comment: IIUC, this is the output side right? ########## 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 { Review comment: javadoc please ########## 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 Review comment: consider naming it numMessagesPerSecInKafKaTopic and add a comment stating this is applicable to realtime/hybrid table ########## 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(); Review comment: I think we need to make the purpose of overwrittenConfigs more clear. IIUC, this works as follows: I as a user/dev wants to use the rule engine to recommend configs. However, based on my experience or due to a special optimization for a use case, I know that it will help to have inverted index on a particular column. But I still want to run the engine to recommend inverted indexes on other columns (if applicable) and recommend other configs (sorted, bloom etc). The engine will do it's job of recommending by taking into account the overwritten config and honoring it. In other words, the recommended config is going to be a super-set of the overwritten config. Is this understanding correct? We should highlight the purpose clearly in comments ########## 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****************************************/ Review comment: I don't think I fully understand why this should be ignored by the deserializer ########## 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. Review comment: Please see my comment above w.r.t explaining the purpose and usage of overwritten configs. ########## File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/io/exceptions/InvalidInputException.java ########## @@ -16,14 +16,12 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pinot.tools.tuner.query.src.stats.wrapper; +package org.apache.pinot.controller.recommender.io.exceptions; Review comment: Why did we move this class to the recommender? Is it not being used elsewhere? ########## File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/io/metadata/ColumnMetaData.java ########## @@ -0,0 +1,71 @@ +/** + * 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.metadata; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonSetter; +import com.fasterxml.jackson.annotation.Nulls; +import org.apache.pinot.spi.data.FieldSpec; + +import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.DEFAULT_AVERAGE_NUM_VALUES_PER_ENTRY; +import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.DEFAULT_CARDINALITY; +import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.DEFAULT_DATA_LENGTH; + + +/** + * The metadata of a column + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class ColumnMetaData extends FieldSpec { Review comment: We have an existing class ColumnMetadata in pinot-core. Although, this is in a different package so there shouldn't be any conflict. But, just to avoid any confusion (intellij will display both files as the user starts typing the name in file search), please consider renaming it. ########## File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/AbstractRule.java ########## @@ -16,22 +16,19 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pinot.tools.tuner.query.src.parser; +package org.apache.pinot.controller.recommender.rules; -import javax.annotation.Nullable; -import org.apache.pinot.tools.tuner.query.src.stats.wrapper.AbstractQueryStats; +import org.apache.pinot.controller.recommender.io.ConfigManager; +import org.apache.pinot.controller.recommender.io.InputManager; -/** - * Parser interface for a query line - */ -public interface QueryParser { - /** - * parse the the complete log line to a parsed obj - * @param line the complete log line to be parsed, InputIterator should put broken lines together - * @return the parsed log line obj - */ - @Nullable - AbstractQueryStats parse(String line); -} +public abstract class AbstractRule { Review comment: This should be a new class/interface for the recommender right? Why are we moving an existing class? or is this a github issue? ########## File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/RulesToExecute.java ########## @@ -0,0 +1,140 @@ +/** + * 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.rules; + +import com.fasterxml.jackson.annotation.JsonSetter; +import com.fasterxml.jackson.annotation.Nulls; +import org.apache.pinot.controller.recommender.io.ConfigManager; +import org.apache.pinot.controller.recommender.io.InputManager; +import org.apache.pinot.controller.recommender.rules.impl.BloomFilterRule; +import org.apache.pinot.controller.recommender.rules.impl.FlagQueryRule; +import org.apache.pinot.controller.recommender.rules.impl.InvertedSortedIndexJointRule; +import org.apache.pinot.controller.recommender.rules.impl.KafkaPartitionRule; +import org.apache.pinot.controller.recommender.rules.impl.NoDictionaryOnHeapDictionaryJointRule; +import org.apache.pinot.controller.recommender.rules.impl.PinotTablePartitionRule; +import org.apache.pinot.controller.recommender.rules.impl.VariedLengthDictionaryRule; + +import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.RulesToExecute.*; + + +public class RulesToExecute { Review comment: Javadoc would be nice ---------------------------------------------------------------- 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