You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2020/07/30 18:42:19 UTC

[GitHub] [incubator-pinot] jasperjiaguo opened a new pull request #5774: Config recommendation engine

jasperjiaguo opened a new pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774


   ## Description
   This is the continuation of the project Pinot-Tuna, issue 4461. In this PR we design an extensible engine that takes an input json with schema, column metadata, qps, etc., and try to recommend configurations as inverted/sorted indices, partitioning scheme, and dictionary.
   
   ## Documentation
   https://cwiki.apache.org/confluence/display/PINOT/Automated+Inverted+Index+Recommendation+for+Pinot
   


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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465930033



##########
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:
       The tuner package was the old code for index recommender. I did reuse some of the old code, so github shows I'm moving some files from the old package. But I'm not sure why this happens in InvalidInputException class as it is completely new.




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466012783



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/RecommenderConstants.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.io.params;
+
+public class RecommenderConstants {
+  public static class InvertedSortedIndexJointRule {
+    public static final double DEFAULT_PERCENT_SELECT_FOR_FUNCTION = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_TEXT_MATCH = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_RANGE = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_REGEX = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_ISNULL = 0.5d;
+    public static final double DEFAULT_THRESHOLD_MIN_AND_PREDICATE_INCREMENTAL_VOTE = 0.6d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_AND_PREDICATE_TOP_CANDIDATES = 0.8d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_GAIN_DIFF_BETWEEN_ITERATION = 0.05d;
+    public static final int DEFAULT_MAX_NUM_ITERATION_WITHOUT_GAIN = 3;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_NESI_FOR_TOP_CANDIDATES = 0.7d;
+  }
+
+  public static class RulesToExecute {
+    public static final boolean DEFAULT_RECOMMEND_FLAG_QUERY = true;
+    public static final boolean DEFAULT_RECOMMEND_VARIED_LENGTH_DICTIONARY = true;
+    public static final boolean DEFAULT_RECOMMEND_KAFKA_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_PINOT_TABLE_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_INVERTED_SORTED_INDEX_JOINT = true;
+    public static final boolean DEFAULT_RECOMMEND_BLOOM_FILTER = true;
+    public static final boolean DEFAULT_RECOMMEND_NO_DICTIONARY_ONHEAP_DICTIONARY_JOINT = true;
+  }
+
+  public static class PartitionRule {
+    public static final int DEFAULT_NUM_PARTITIONS = 0;
+
+    public static final long DEFAULT_THRESHOLD_MAX_SLA_PARTITION = 1000;
+    public static final long DEFAULT_THRESHOLD_MIN_QPS_PARTITION = 200;
+    public static final long DEFAULT_OPTIMAL_SIZE_PER_SEGMENT = 2000_000_000; //2GB
+    public static final long DEFAULT_KAFKA_NUM_MESSAGES_PER_SEC_PER_PARTITION = 250;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_DIMENSION_PARTITION_TOP_CANDIDATES = 0.8d;
+    public static final int DEFAULT_THRESHOLD_MAX_IN_LENGTH = 4;
+  }
+
+  public static class BloomFilterRule {
+    public static final long DEFAULT_THRESHOLD_MAX_CARDINALITY_BLOOMFILTER = 1000_000;
+    public static final double DEFAULT_THRESHOLD_MIN_PERCENT_EQ_BLOOMFILTER = 0.5d;
+  }
+
+  public static class NoDictionaryOnHeapDictionaryJointRule {
+    public static final double DEFAULT_THRESHOLD_MIN_FILTER_FREQ_DICTIONARY = 0d;
+    public static final double DEFAULT_THRESHOLD_MAX_SELECTION_FREQ_DICTIONARY = 0.3d;
+    public static final long DEFAULT_THRESHOLD_MIN_QPS_ON_HEAP = 10_000;
+    public static final long DEFAULT_THRESHOLD_MAX_DICTIONARY_SIZE_ON_HEAP = 1000_000L;
+    public static final double DEFAULT_THRESHOLD_MIN_FILTER_FREQ_ON_HEAP = 0.3d;
+    public static final double DEFAULT_THRESHOLD_MIN_PERCENT_DICTIONARY_STORAGE_SVAE = 0.95;
+

Review comment:
       Will revisit this later today




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466018547



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/RecommenderConstants.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.io.params;
+
+public class RecommenderConstants {
+  public static class InvertedSortedIndexJointRule {
+    public static final double DEFAULT_PERCENT_SELECT_FOR_FUNCTION = 0.5d;

Review comment:
       Will add the comments in the *Params class for parameters




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465932031



##########
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:
       Please see the above conversation.




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465941869



##########
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:
       done




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465930033



##########
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:
       The tuner package was the old code for index recommender. I did reuse some of the old code, so github shows I'm moving some files from the old package. But I'm not sure why this happens in InvalidInputException class as it is completely new. The old  index recommender is not used anywhere right now so I removed  the sources completely.




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465940323



##########
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:
       got it




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466637820



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/PartitionRuleParams.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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.io.params;
+
+import com.fasterxml.jackson.annotation.JsonSetter;
+import com.fasterxml.jackson.annotation.Nulls;
+
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.PartitionRule.*;
+
+
+public class PartitionRuleParams {

Review comment:
       done




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465811829



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/PartitionRuleParams.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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.io.params;
+
+import com.fasterxml.jackson.annotation.JsonSetter;
+import com.fasterxml.jackson.annotation.Nulls;
+
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.PartitionRule.*;
+
+
+public class PartitionRuleParams {

Review comment:
       Please add javadoc and a short one-line comment explaining the purpose of each configuration. Please try to do this for all param classes.

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/RecommenderConstants.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.io.params;
+
+public class RecommenderConstants {

Review comment:
       Please add javadoc 

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/RecommenderConstants.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.io.params;
+
+public class RecommenderConstants {
+  public static class InvertedSortedIndexJointRule {
+    public static final double DEFAULT_PERCENT_SELECT_FOR_FUNCTION = 0.5d;

Review comment:
       a very short one line comment above each configuration (at least for the ones that are not self explanatory and intuitive from the name)




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465807413



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/PinotTablePartitionRule.java
##########
@@ -0,0 +1,228 @@
+/**
+ * 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.impl;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Optional;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.InPredicate;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.PartitionRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.*;
+
+
+public class PinotTablePartitionRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(PinotTablePartitionRule.class);
+  PartitionRuleParams _params;
+
+  protected final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+
+  public PinotTablePartitionRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    this._params = inputManager.getPartitionRuleParams();
+  }
+
+  @Override
+  public void run() {
+    //**********Calculate size per record***************/
+    _inputManager.estimateSizePerRecord();
+    //**************************************************/
+
+    LOGGER.info("Recommending partition configurations");
+
+    if (_inputManager.getQps()
+        < _params.THRESHOLD_MIN_QPS_PARTITION) { //For a table whose QPS < Q (say 200 or 300) NO partitioning is needed.
+      LOGGER.info("*Input QPS {} < threshold {}, no partition needed", _inputManager.getQps(),
+          _params.THRESHOLD_MIN_QPS_PARTITION);
+      return;
+    }
+    if (_inputManager.getLatencySLA()
+        > _params.THRESHOLD_MAX_SLA_PARTITION) { //For a table whose latency SLA > L (say 1000ms) NO partitioning is needed.
+      LOGGER.info("*Input SLA {} > threshold {}, no partition needed", _inputManager.getLatencySLA(),
+          _params.THRESHOLD_MAX_SLA_PARTITION);
+      return;
+    }
+
+    LOGGER.info("*Recommending partition number");

Review comment:
       you can probably remove the "*". It just adds minor overhead to the logger




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465822093



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/RecommenderConstants.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.io.params;
+
+public class RecommenderConstants {
+  public static class InvertedSortedIndexJointRule {
+    public static final double DEFAULT_PERCENT_SELECT_FOR_FUNCTION = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_TEXT_MATCH = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_RANGE = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_REGEX = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_ISNULL = 0.5d;
+    public static final double DEFAULT_THRESHOLD_MIN_AND_PREDICATE_INCREMENTAL_VOTE = 0.6d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_AND_PREDICATE_TOP_CANDIDATES = 0.8d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_GAIN_DIFF_BETWEEN_ITERATION = 0.05d;
+    public static final int DEFAULT_MAX_NUM_ITERATION_WITHOUT_GAIN = 3;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_NESI_FOR_TOP_CANDIDATES = 0.7d;
+  }
+
+  public static class RulesToExecute {
+    public static final boolean DEFAULT_RECOMMEND_FLAG_QUERY = true;
+    public static final boolean DEFAULT_RECOMMEND_VARIED_LENGTH_DICTIONARY = true;
+    public static final boolean DEFAULT_RECOMMEND_KAFKA_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_PINOT_TABLE_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_INVERTED_SORTED_INDEX_JOINT = true;
+    public static final boolean DEFAULT_RECOMMEND_BLOOM_FILTER = true;
+    public static final boolean DEFAULT_RECOMMEND_NO_DICTIONARY_ONHEAP_DICTIONARY_JOINT = true;
+  }
+
+  public static class PartitionRule {
+    public static final int DEFAULT_NUM_PARTITIONS = 0;
+
+    public static final long DEFAULT_THRESHOLD_MAX_SLA_PARTITION = 1000;
+    public static final long DEFAULT_THRESHOLD_MIN_QPS_PARTITION = 200;
+    public static final long DEFAULT_OPTIMAL_SIZE_PER_SEGMENT = 2000_000_000; //2GB
+    public static final long DEFAULT_KAFKA_NUM_MESSAGES_PER_SEC_PER_PARTITION = 250;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_DIMENSION_PARTITION_TOP_CANDIDATES = 0.8d;
+    public static final int DEFAULT_THRESHOLD_MAX_IN_LENGTH = 4;
+  }
+
+  public static class BloomFilterRule {
+    public static final long DEFAULT_THRESHOLD_MAX_CARDINALITY_BLOOMFILTER = 1000_000;
+    public static final double DEFAULT_THRESHOLD_MIN_PERCENT_EQ_BLOOMFILTER = 0.5d;
+  }
+
+  public static class NoDictionaryOnHeapDictionaryJointRule {
+    public static final double DEFAULT_THRESHOLD_MIN_FILTER_FREQ_DICTIONARY = 0d;
+    public static final double DEFAULT_THRESHOLD_MAX_SELECTION_FREQ_DICTIONARY = 0.3d;
+    public static final long DEFAULT_THRESHOLD_MIN_QPS_ON_HEAP = 10_000;
+    public static final long DEFAULT_THRESHOLD_MAX_DICTIONARY_SIZE_ON_HEAP = 1000_000L;
+    public static final double DEFAULT_THRESHOLD_MIN_FILTER_FREQ_ON_HEAP = 0.3d;
+    public static final double DEFAULT_THRESHOLD_MIN_PERCENT_DICTIONARY_STORAGE_SVAE = 0.95;
+
+    public static final double DEFAULT_DICT_COEFF_A = 0.217769;
+    public static final double DEFAULT_DICT_COEFF_B = 89.0975;
+    public static final double DEFAULT_DICT_LOWER = 0;
+    public static final double DEFAULT_DICT_UPPER = 0;
+    public static final int DEFAUlT_NUM_PARTITIONS = 16;
+
+    public static final int DEFAULT_SEGMENT_FLUSH_TIME = 86400;
+  }
+
+  public static class FlagQueryRuleParams{
+    public static final long DEFAULT_THRESHOLD_MAX_LIMIT_SIZE = 100000;
+    public static final String WARNING_NO_FILTERING = "Warning: No filtering in ths query";
+    public static final String WARNING_NO_TIME_COL = "Warning: No time column used in ths query";
+    public static final String WARNING_TOO_LONG_LIMIT = "Warning: The size of LIMIT is longer than " + DEFAULT_THRESHOLD_MAX_LIMIT_SIZE;
+    public static final String ERROR_INVALID_QUERY = "Error: query not able to parse, skipped";
+  }
+
+  public static final String PQL = "pql";
+  public static final String SQL = "sql";
+  public static final String OFFLINE = "offline";
+  public static final String REALTIME = "realtime";
+  public static final String HYBRID = "hybrid";
+  public static final int NO_SUCH_COL = -1;
+  public static final double DEFAULT_CARDINALITY = 1;
+  public static final double MIN_CARDINALITY = 1;
+  public static final double DEFAULT_AVERAGE_NUM_VALUES_PER_ENTRY = 1d;
+  public static final int DEFAULT_QPS = 100;
+  public static final int DEFAULT_LATENCY_SLA = 1000;
+  public static final int DEFAULT_NUM_MSG_PER_SEC = 250;
+  public static final int DEFAULT_NUM_RECORDS_PER_PUSH = 10000;
+  public static final int DEFAULT_INT_SIZE = Integer.BYTES;
+  public static final int DEFAULT_NULL_SIZE = 0;

Review comment:
       I don't think you need to have DEFAULT_FLOAT/INT/LONG etc size. These are fixed width columns and will always be same as specified in their respective classes. -- Float.BYTES, Long.BYTES etc




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465834951



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/PinotTablePartitionRule.java
##########
@@ -0,0 +1,228 @@
+/**
+ * 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.impl;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Optional;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.InPredicate;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.PartitionRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.*;
+
+
+public class PinotTablePartitionRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(PinotTablePartitionRule.class);
+  PartitionRuleParams _params;
+
+  protected final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+
+  public PinotTablePartitionRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    this._params = inputManager.getPartitionRuleParams();
+  }
+
+  @Override
+  public void run() {
+    //**********Calculate size per record***************/
+    _inputManager.estimateSizePerRecord();
+    //**************************************************/
+
+    LOGGER.info("Recommending partition configurations");
+
+    if (_inputManager.getQps()
+        < _params.THRESHOLD_MIN_QPS_PARTITION) { //For a table whose QPS < Q (say 200 or 300) NO partitioning is needed.
+      LOGGER.info("*Input QPS {} < threshold {}, no partition needed", _inputManager.getQps(),
+          _params.THRESHOLD_MIN_QPS_PARTITION);
+      return;
+    }
+    if (_inputManager.getLatencySLA()
+        > _params.THRESHOLD_MAX_SLA_PARTITION) { //For a table whose latency SLA > L (say 1000ms) NO partitioning is needed.
+      LOGGER.info("*Input SLA {} > threshold {}, no partition needed", _inputManager.getLatencySLA(),
+          _params.THRESHOLD_MAX_SLA_PARTITION);
+      return;
+    }
+
+    LOGGER.info("*Recommending partition number");
+    if (_inputManager.getTableType().equalsIgnoreCase(
+        REALTIME)) { //real time partition num should be the same value as the number of kafka partitions
+      _outputManager.getPartitionConfig()
+          .setNumPartitionsRealtime(_outputManager.getPartitionConfig().getNumKafkaPartitions());
+    } else if (_inputManager.getTableType().equalsIgnoreCase(OFFLINE)) {

Review comment:
       It will also be very useful for reference to quote the study and analysis we had done for use case at Li. Please don't quote the table names. However, we should include the following general comments:
   
   - For realtime/hybrid, the number of partitions on realtime Pinot table side is same as number of kafka partitions. This is generally the case unless there is a reason for them to be different. We saw one outlier
   - For offline, the number of partitions on offline Pinot table side is dependent on the amount of data. For hybrid table, we have seen cases where this value = number of kafka partitions  = number of realtime table partitions. For hybrid table, we have also seen cases, where the value for offline is lower than realtime since the data generated on a given day is low volume and using a high count of number of partitions would lead to too many small sized segments since we typically have data from one partition in a segment.  




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466012311



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/PinotTablePartitionRule.java
##########
@@ -0,0 +1,228 @@
+/**
+ * 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.impl;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Optional;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.InPredicate;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.PartitionRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.*;
+
+
+public class PinotTablePartitionRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(PinotTablePartitionRule.class);
+  PartitionRuleParams _params;
+
+  protected final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+
+  public PinotTablePartitionRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    this._params = inputManager.getPartitionRuleParams();
+  }
+
+  @Override
+  public void run() {
+    //**********Calculate size per record***************/
+    _inputManager.estimateSizePerRecord();
+    //**************************************************/
+
+    LOGGER.info("Recommending partition configurations");
+
+    if (_inputManager.getQps()
+        < _params.THRESHOLD_MIN_QPS_PARTITION) { //For a table whose QPS < Q (say 200 or 300) NO partitioning is needed.
+      LOGGER.info("*Input QPS {} < threshold {}, no partition needed", _inputManager.getQps(),
+          _params.THRESHOLD_MIN_QPS_PARTITION);
+      return;
+    }
+    if (_inputManager.getLatencySLA()
+        > _params.THRESHOLD_MAX_SLA_PARTITION) { //For a table whose latency SLA > L (say 1000ms) NO partitioning is needed.
+      LOGGER.info("*Input SLA {} > threshold {}, no partition needed", _inputManager.getLatencySLA(),
+          _params.THRESHOLD_MAX_SLA_PARTITION);
+      return;
+    }
+
+    LOGGER.info("*Recommending partition number");
+    if (_inputManager.getTableType().equalsIgnoreCase(
+        REALTIME)) { //real time partition num should be the same value as the number of kafka partitions
+      _outputManager.getPartitionConfig()
+          .setNumPartitionsRealtime(_outputManager.getPartitionConfig().getNumKafkaPartitions());
+    } else if (_inputManager.getTableType().equalsIgnoreCase(OFFLINE)) {

Review comment:
       done




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466638540



##########
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:
       Added code 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 {

Review comment:
       done




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466637759



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/RecommenderConstants.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.io.params;
+
+public class RecommenderConstants {

Review comment:
       done




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465941468



##########
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:
       Got it. Thanks.




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



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


[GitHub] [incubator-pinot] codecov-commenter commented on pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#issuecomment-667309936


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/5774?src=pr&el=h1) Report
   > Merging [#5774](https://codecov.io/gh/apache/incubator-pinot/pull/5774?src=pr&el=desc) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1beaab59b73f26c4e35f3b9bc856b03806cddf5a&el=desc) will **decrease** coverage by `21.87%`.
   > The diff coverage is `56.01%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/5774/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/5774?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff             @@
   ##           master    #5774       +/-   ##
   ===========================================
   - Coverage   66.44%   44.57%   -21.88%     
   ===========================================
     Files        1075     1141       +66     
     Lines       54773    58755     +3982     
     Branches     8168     8911      +743     
   ===========================================
   - Hits        36396    26192    -10204     
   - Misses      15700    30228    +14528     
   + Partials     2677     2335      -342     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | #integrationtests | `44.57% <56.01%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/5774?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...quota/HelixExternalViewBasedQueryQuotaManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/5774/diff?src=pr&el=tree#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcXVlcnlxdW90YS9IZWxpeEV4dGVybmFsVmlld0Jhc2VkUXVlcnlRdW90YU1hbmFnZXIuamF2YQ==) | `40.00% <0.00%> (-27.88%)` | :arrow_down: |
   | [...org/apache/pinot/common/function/FunctionInfo.java](https://codecov.io/gh/apache/incubator-pinot/pull/5774/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25JbmZvLmphdmE=) | `73.33% <ø> (ø)` | |
   | [.../org/apache/pinot/common/lineage/LineageEntry.java](https://codecov.io/gh/apache/incubator-pinot/pull/5774/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbGluZWFnZS9MaW5lYWdlRW50cnkuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...apache/pinot/common/lineage/LineageEntryState.java](https://codecov.io/gh/apache/incubator-pinot/pull/5774/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbGluZWFnZS9MaW5lYWdlRW50cnlTdGF0ZS5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...rg/apache/pinot/common/lineage/SegmentLineage.java](https://codecov.io/gh/apache/incubator-pinot/pull/5774/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbGluZWFnZS9TZWdtZW50TGluZWFnZS5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...java/org/apache/pinot/common/segment/ReadMode.java](https://codecov.io/gh/apache/incubator-pinot/pull/5774/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vc2VnbWVudC9SZWFkTW9kZS5qYXZh) | `66.66% <ø> (ø)` | |
   | [...org/apache/pinot/common/utils/CommonConstants.java](https://codecov.io/gh/apache/incubator-pinot/pull/5774/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvQ29tbW9uQ29uc3RhbnRzLmphdmE=) | `38.09% <ø> (ø)` | |
   | [...e/pinot/common/utils/FileUploadDownloadClient.java](https://codecov.io/gh/apache/incubator-pinot/pull/5774/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvRmlsZVVwbG9hZERvd25sb2FkQ2xpZW50LmphdmE=) | `39.88% <ø> (-17.86%)` | :arrow_down: |
   | [...he/pinot/common/utils/config/TableConfigUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/5774/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvY29uZmlnL1RhYmxlQ29uZmlnVXRpbHMuamF2YQ==) | `72.52% <ø> (-18.00%)` | :arrow_down: |
   | [.../org/apache/pinot/pql/parsers/Pql2AstListener.java](https://codecov.io/gh/apache/incubator-pinot/pull/5774/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wcWwvcGFyc2Vycy9QcWwyQXN0TGlzdGVuZXIuamF2YQ==) | `79.01% <0.00%> (-13.02%)` | :arrow_down: |
   | ... and [1049 more](https://codecov.io/gh/apache/incubator-pinot/pull/5774/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/5774?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/5774?src=pr&el=footer). Last update [cb7de23...6278fb6](https://codecov.io/gh/apache/incubator-pinot/pull/5774?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465806243



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/NoDictionaryOnHeapDictionaryJointRule.java
##########
@@ -0,0 +1,245 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.NoDictionaryOnHeapDictionaryJointRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.NoDictionaryOnHeapDictionaryJointRule.*;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.REALTIME;
+
+
+public class NoDictionaryOnHeapDictionaryJointRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(NoDictionaryOnHeapDictionaryJointRule.class);
+  private final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+  private final NoDictionaryOnHeapDictionaryJointRuleParams _params;
+
+  public NoDictionaryOnHeapDictionaryJointRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    _params = inputManager.getNoDictionaryOnHeapDictionaryJointRuleParams();
+  }
+
+  @Override
+  public void run() {
+    LOGGER.info("Recommending no dictionary and on-heap dictionaries");
+
+    int numCols = _inputManager.getNumCols();
+    double[] filterGroupByWeights = new double[numCols];
+    double[] selectionWeights = new double[numCols];
+    AtomicDouble totalWeight = new AtomicDouble(0);
+
+    //**********No dictionary recommendation*******/
+    Set<String> noDictCols = new HashSet<>(_inputManager.getColNameToIntMap().keySet());
+
+    //Exclude cols with index
+    noDictCols.removeAll(_outputManager.getIndexConfig().getInvertedIndexColumns());
+    noDictCols.removeAll(_outputManager.getIndexConfig().getSortedColumn());
+    // TODO: Remove this after range index is implemented for no-dictionary
+    noDictCols.removeAll(_outputManager.getIndexConfig().getRangeIndexColumns());
+    LOGGER.debug("noDictCols {}", noDictCols);
+    //Find out columns used in filter&groupby and selection and corresponding frequencies
+    _inputManager.getQueryWeightMap().forEach((query, weight) -> {
+      parseQuery(query, weight, filterGroupByWeights, selectionWeights);
+      totalWeight.addAndGet(weight);
+    });
+
+    //Add dictionary on columns used in filter&groupby , with frequency > threshold
+    for (int i = 0; i < numCols; i++) {
+      double filterGroupByFreq = filterGroupByWeights[i] / totalWeight.get();
+      if (filterGroupByFreq > _params.THRESHOLD_MIN_FILTER_FREQ_DICTIONARY) {
+        noDictCols.remove(_inputManager.intToColName(i));
+      }
+    }
+
+    LOGGER.debug("filterGroupByWeights {}, selectionWeights{}, totalWeight{} ", filterGroupByWeights, selectionWeights,
+        totalWeight);
+    LOGGER.debug("noDictCols {}", noDictCols);
+
+    for (int i = 0; i < numCols; i++) {
+      // No dictionary on columns frequently used in selection
+      double selectionFreq = selectionWeights[i] / totalWeight.get();
+      if (selectionFreq > _params.THRESHOLD_MAX_SELECTION_FREQ_DICTIONARY) {
+        continue;
+      }
+
+      // Add dictionary on columns NOT frequently used in selection
+      // AND can save storage > threshold
+      String colName = _inputManager.intToColName(i);
+      double noDictSize;
+      double withDictSize;
+      long colDataSizeWithoutDictionary = _inputManager.getColDataSizeWithoutDictionary(colName);
+      double numValuesPerEntry = _inputManager.getNumValuesPerEntry(colName);
+      int bitCompressedDataSize = _inputManager.getBitCompressedDataSize(colName);
+      long dictionarySize = _inputManager.getDictionarySize(colName);
+      double cardinality = _inputManager.getCardinality(colName);
+      long numRecordsPerPush = _inputManager.getNumRecordsPerPush();
+      LOGGER.debug("colDataSizeWithoutDictionary {}", colDataSizeWithoutDictionary);
+      LOGGER.debug("bitCompressedDataSize {}", bitCompressedDataSize);
+      LOGGER.debug("dictionarySize {}", dictionarySize);
+      LOGGER.debug("numValuesPerEntry {}", numValuesPerEntry);
+
+      if (_inputManager.getTableType().equalsIgnoreCase(REALTIME)) {
+        //TODO: improve this estimation
+        noDictSize = // size of one segment flushed ith no dictionary
+            colDataSizeWithoutDictionary * numValuesPerEntry * _params.SEGMENT_FLUSH_TIME;
+        withDictSize = // size of one flushed segment with dictionary
+            dictionarySize + bitCompressedDataSize * numValuesPerEntry * _params.SEGMENT_FLUSH_TIME;
+      } else { // For hybrid or offline table, nodictionary follows the offline side
+        noDictSize = // size of all segments in one push  with no dictionary
+            colDataSizeWithoutDictionary * numValuesPerEntry * numRecordsPerPush;
+        withDictSize = // size of all segments in one push with dictionary
+            dictionarySize * dictionaryCoefficient(cardinality, numRecordsPerPush) * DEFAUlT_NUM_PARTITIONS

Review comment:
       Why is DEFAULT_NUM_PARTITIONS used here?




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r467300262



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java
##########
@@ -141,6 +142,18 @@ public SuccessResponse addTable(String tableConfigStr) {
     }
   }
 
+  @PUT
+  @Produces(MediaType.APPLICATION_JSON)

Review comment:
       I stayed with only `@Produces(MediaType.APPLICATION_JSON)` to follow the pattern of other APIs

##########
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:
       done




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465800278



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/NoDictionaryOnHeapDictionaryJointRule.java
##########
@@ -0,0 +1,245 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.NoDictionaryOnHeapDictionaryJointRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.NoDictionaryOnHeapDictionaryJointRule.*;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.REALTIME;
+
+
+public class NoDictionaryOnHeapDictionaryJointRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(NoDictionaryOnHeapDictionaryJointRule.class);
+  private final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+  private final NoDictionaryOnHeapDictionaryJointRuleParams _params;
+
+  public NoDictionaryOnHeapDictionaryJointRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    _params = inputManager.getNoDictionaryOnHeapDictionaryJointRuleParams();
+  }
+
+  @Override
+  public void run() {
+    LOGGER.info("Recommending no dictionary and on-heap dictionaries");
+
+    int numCols = _inputManager.getNumCols();
+    double[] filterGroupByWeights = new double[numCols];
+    double[] selectionWeights = new double[numCols];
+    AtomicDouble totalWeight = new AtomicDouble(0);
+
+    //**********No dictionary recommendation*******/
+    Set<String> noDictCols = new HashSet<>(_inputManager.getColNameToIntMap().keySet());
+
+    //Exclude cols with index
+    noDictCols.removeAll(_outputManager.getIndexConfig().getInvertedIndexColumns());
+    noDictCols.removeAll(_outputManager.getIndexConfig().getSortedColumn());
+    // TODO: Remove this after range index is implemented for no-dictionary
+    noDictCols.removeAll(_outputManager.getIndexConfig().getRangeIndexColumns());
+    LOGGER.debug("noDictCols {}", noDictCols);
+    //Find out columns used in filter&groupby and selection and corresponding frequencies
+    _inputManager.getQueryWeightMap().forEach((query, weight) -> {
+      parseQuery(query, weight, filterGroupByWeights, selectionWeights);
+      totalWeight.addAndGet(weight);
+    });
+
+    //Add dictionary on columns used in filter&groupby , with frequency > threshold
+    for (int i = 0; i < numCols; i++) {
+      double filterGroupByFreq = filterGroupByWeights[i] / totalWeight.get();
+      if (filterGroupByFreq > _params.THRESHOLD_MIN_FILTER_FREQ_DICTIONARY) {
+        noDictCols.remove(_inputManager.intToColName(i));
+      }
+    }
+
+    LOGGER.debug("filterGroupByWeights {}, selectionWeights{}, totalWeight{} ", filterGroupByWeights, selectionWeights,
+        totalWeight);
+    LOGGER.debug("noDictCols {}", noDictCols);
+
+    for (int i = 0; i < numCols; i++) {
+      // No dictionary on columns frequently used in selection
+      double selectionFreq = selectionWeights[i] / totalWeight.get();
+      if (selectionFreq > _params.THRESHOLD_MAX_SELECTION_FREQ_DICTIONARY) {
+        continue;
+      }
+
+      // Add dictionary on columns NOT frequently used in selection
+      // AND can save storage > threshold
+      String colName = _inputManager.intToColName(i);
+      double noDictSize;
+      double withDictSize;
+      long colDataSizeWithoutDictionary = _inputManager.getColDataSizeWithoutDictionary(colName);
+      double numValuesPerEntry = _inputManager.getNumValuesPerEntry(colName);
+      int bitCompressedDataSize = _inputManager.getBitCompressedDataSize(colName);
+      long dictionarySize = _inputManager.getDictionarySize(colName);
+      double cardinality = _inputManager.getCardinality(colName);
+      long numRecordsPerPush = _inputManager.getNumRecordsPerPush();
+      LOGGER.debug("colDataSizeWithoutDictionary {}", colDataSizeWithoutDictionary);
+      LOGGER.debug("bitCompressedDataSize {}", bitCompressedDataSize);
+      LOGGER.debug("dictionarySize {}", dictionarySize);
+      LOGGER.debug("numValuesPerEntry {}", numValuesPerEntry);
+
+      if (_inputManager.getTableType().equalsIgnoreCase(REALTIME)) {
+        //TODO: improve this estimation
+        noDictSize = // size of one segment flushed ith no dictionary
+            colDataSizeWithoutDictionary * numValuesPerEntry * _params.SEGMENT_FLUSH_TIME;

Review comment:
       We can simplify this by not bringing numValuesPerEntry into the equation. numValuesPerEntry is applicable to MV columns right? Pinot currently doesn't support raw MV columns so they are **always** dictionary encoded




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465823171



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/RecommenderConstants.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.io.params;
+
+public class RecommenderConstants {
+  public static class InvertedSortedIndexJointRule {
+    public static final double DEFAULT_PERCENT_SELECT_FOR_FUNCTION = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_TEXT_MATCH = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_RANGE = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_REGEX = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_ISNULL = 0.5d;
+    public static final double DEFAULT_THRESHOLD_MIN_AND_PREDICATE_INCREMENTAL_VOTE = 0.6d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_AND_PREDICATE_TOP_CANDIDATES = 0.8d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_GAIN_DIFF_BETWEEN_ITERATION = 0.05d;
+    public static final int DEFAULT_MAX_NUM_ITERATION_WITHOUT_GAIN = 3;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_NESI_FOR_TOP_CANDIDATES = 0.7d;
+  }
+
+  public static class RulesToExecute {
+    public static final boolean DEFAULT_RECOMMEND_FLAG_QUERY = true;
+    public static final boolean DEFAULT_RECOMMEND_VARIED_LENGTH_DICTIONARY = true;
+    public static final boolean DEFAULT_RECOMMEND_KAFKA_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_PINOT_TABLE_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_INVERTED_SORTED_INDEX_JOINT = true;
+    public static final boolean DEFAULT_RECOMMEND_BLOOM_FILTER = true;
+    public static final boolean DEFAULT_RECOMMEND_NO_DICTIONARY_ONHEAP_DICTIONARY_JOINT = true;
+  }
+
+  public static class PartitionRule {
+    public static final int DEFAULT_NUM_PARTITIONS = 0;
+
+    public static final long DEFAULT_THRESHOLD_MAX_SLA_PARTITION = 1000;
+    public static final long DEFAULT_THRESHOLD_MIN_QPS_PARTITION = 200;
+    public static final long DEFAULT_OPTIMAL_SIZE_PER_SEGMENT = 2000_000_000; //2GB
+    public static final long DEFAULT_KAFKA_NUM_MESSAGES_PER_SEC_PER_PARTITION = 250;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_DIMENSION_PARTITION_TOP_CANDIDATES = 0.8d;
+    public static final int DEFAULT_THRESHOLD_MAX_IN_LENGTH = 4;
+  }
+
+  public static class BloomFilterRule {
+    public static final long DEFAULT_THRESHOLD_MAX_CARDINALITY_BLOOMFILTER = 1000_000;
+    public static final double DEFAULT_THRESHOLD_MIN_PERCENT_EQ_BLOOMFILTER = 0.5d;
+  }
+
+  public static class NoDictionaryOnHeapDictionaryJointRule {
+    public static final double DEFAULT_THRESHOLD_MIN_FILTER_FREQ_DICTIONARY = 0d;
+    public static final double DEFAULT_THRESHOLD_MAX_SELECTION_FREQ_DICTIONARY = 0.3d;
+    public static final long DEFAULT_THRESHOLD_MIN_QPS_ON_HEAP = 10_000;
+    public static final long DEFAULT_THRESHOLD_MAX_DICTIONARY_SIZE_ON_HEAP = 1000_000L;
+    public static final double DEFAULT_THRESHOLD_MIN_FILTER_FREQ_ON_HEAP = 0.3d;
+    public static final double DEFAULT_THRESHOLD_MIN_PERCENT_DICTIONARY_STORAGE_SVAE = 0.95;
+

Review comment:
       How are these coefficients used?




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466007205



##########
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:
       Done! Thanks for pointing out this bug. 




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466638254



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/BloomFilterRule.java
##########
@@ -0,0 +1,142 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.BloomFilterRuleParams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class  BloomFilterRule extends AbstractRule {

Review comment:
       done

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/NoDictionaryOnHeapDictionaryJointRule.java
##########
@@ -0,0 +1,245 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.NoDictionaryOnHeapDictionaryJointRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.NoDictionaryOnHeapDictionaryJointRule.*;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.REALTIME;
+
+
+public class NoDictionaryOnHeapDictionaryJointRule extends AbstractRule {

Review comment:
       done




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465809195



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/PinotTablePartitionRule.java
##########
@@ -0,0 +1,228 @@
+/**
+ * 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.impl;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Optional;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.InPredicate;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.PartitionRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.*;
+
+
+public class PinotTablePartitionRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(PinotTablePartitionRule.class);
+  PartitionRuleParams _params;
+
+  protected final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+
+  public PinotTablePartitionRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    this._params = inputManager.getPartitionRuleParams();
+  }
+
+  @Override
+  public void run() {
+    //**********Calculate size per record***************/
+    _inputManager.estimateSizePerRecord();
+    //**************************************************/
+
+    LOGGER.info("Recommending partition configurations");
+
+    if (_inputManager.getQps()
+        < _params.THRESHOLD_MIN_QPS_PARTITION) { //For a table whose QPS < Q (say 200 or 300) NO partitioning is needed.
+      LOGGER.info("*Input QPS {} < threshold {}, no partition needed", _inputManager.getQps(),
+          _params.THRESHOLD_MIN_QPS_PARTITION);
+      return;
+    }
+    if (_inputManager.getLatencySLA()
+        > _params.THRESHOLD_MAX_SLA_PARTITION) { //For a table whose latency SLA > L (say 1000ms) NO partitioning is needed.
+      LOGGER.info("*Input SLA {} > threshold {}, no partition needed", _inputManager.getLatencySLA(),
+          _params.THRESHOLD_MAX_SLA_PARTITION);
+      return;
+    }
+
+    LOGGER.info("*Recommending partition number");
+    if (_inputManager.getTableType().equalsIgnoreCase(
+        REALTIME)) { //real time partition num should be the same value as the number of kafka partitions

Review comment:
       We can actually simplify the code here. Currently we are writing the code for offline and realtime.
   
   We can compute once for offline (if type is OFFLINE or HYBRID)
   We can compute once for realtime (if type if REALTIME or HYBRID)
   
   This will clenup the if-else block here




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466017179



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/RecommenderConstants.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.io.params;
+
+public class RecommenderConstants {
+  public static class InvertedSortedIndexJointRule {
+    public static final double DEFAULT_PERCENT_SELECT_FOR_FUNCTION = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_TEXT_MATCH = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_RANGE = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_REGEX = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_ISNULL = 0.5d;
+    public static final double DEFAULT_THRESHOLD_MIN_AND_PREDICATE_INCREMENTAL_VOTE = 0.6d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_AND_PREDICATE_TOP_CANDIDATES = 0.8d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_GAIN_DIFF_BETWEEN_ITERATION = 0.05d;
+    public static final int DEFAULT_MAX_NUM_ITERATION_WITHOUT_GAIN = 3;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_NESI_FOR_TOP_CANDIDATES = 0.7d;
+  }
+
+  public static class RulesToExecute {
+    public static final boolean DEFAULT_RECOMMEND_FLAG_QUERY = true;
+    public static final boolean DEFAULT_RECOMMEND_VARIED_LENGTH_DICTIONARY = true;
+    public static final boolean DEFAULT_RECOMMEND_KAFKA_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_PINOT_TABLE_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_INVERTED_SORTED_INDEX_JOINT = true;
+    public static final boolean DEFAULT_RECOMMEND_BLOOM_FILTER = true;
+    public static final boolean DEFAULT_RECOMMEND_NO_DICTIONARY_ONHEAP_DICTIONARY_JOINT = true;
+  }
+
+  public static class PartitionRule {
+    public static final int DEFAULT_NUM_PARTITIONS = 0;
+
+    public static final long DEFAULT_THRESHOLD_MAX_SLA_PARTITION = 1000;

Review comment:
       done




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465798271



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/NoDictionaryOnHeapDictionaryJointRule.java
##########
@@ -0,0 +1,245 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.NoDictionaryOnHeapDictionaryJointRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.NoDictionaryOnHeapDictionaryJointRule.*;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.REALTIME;
+
+
+public class NoDictionaryOnHeapDictionaryJointRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(NoDictionaryOnHeapDictionaryJointRule.class);
+  private final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+  private final NoDictionaryOnHeapDictionaryJointRuleParams _params;
+
+  public NoDictionaryOnHeapDictionaryJointRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    _params = inputManager.getNoDictionaryOnHeapDictionaryJointRuleParams();
+  }
+
+  @Override
+  public void run() {
+    LOGGER.info("Recommending no dictionary and on-heap dictionaries");
+
+    int numCols = _inputManager.getNumCols();
+    double[] filterGroupByWeights = new double[numCols];
+    double[] selectionWeights = new double[numCols];
+    AtomicDouble totalWeight = new AtomicDouble(0);
+
+    //**********No dictionary recommendation*******/
+    Set<String> noDictCols = new HashSet<>(_inputManager.getColNameToIntMap().keySet());
+
+    //Exclude cols with index
+    noDictCols.removeAll(_outputManager.getIndexConfig().getInvertedIndexColumns());
+    noDictCols.removeAll(_outputManager.getIndexConfig().getSortedColumn());
+    // TODO: Remove this after range index is implemented for no-dictionary
+    noDictCols.removeAll(_outputManager.getIndexConfig().getRangeIndexColumns());
+    LOGGER.debug("noDictCols {}", noDictCols);
+    //Find out columns used in filter&groupby and selection and corresponding frequencies
+    _inputManager.getQueryWeightMap().forEach((query, weight) -> {
+      parseQuery(query, weight, filterGroupByWeights, selectionWeights);
+      totalWeight.addAndGet(weight);
+    });
+
+    //Add dictionary on columns used in filter&groupby , with frequency > threshold
+    for (int i = 0; i < numCols; i++) {
+      double filterGroupByFreq = filterGroupByWeights[i] / totalWeight.get();
+      if (filterGroupByFreq > _params.THRESHOLD_MIN_FILTER_FREQ_DICTIONARY) {
+        noDictCols.remove(_inputManager.intToColName(i));
+      }
+    }
+
+    LOGGER.debug("filterGroupByWeights {}, selectionWeights{}, totalWeight{} ", filterGroupByWeights, selectionWeights,
+        totalWeight);
+    LOGGER.debug("noDictCols {}", noDictCols);
+
+    for (int i = 0; i < numCols; i++) {
+      // No dictionary on columns frequently used in selection
+      double selectionFreq = selectionWeights[i] / totalWeight.get();
+      if (selectionFreq > _params.THRESHOLD_MAX_SELECTION_FREQ_DICTIONARY) {
+        continue;
+      }
+
+      // Add dictionary on columns NOT frequently used in selection
+      // AND can save storage > threshold
+      String colName = _inputManager.intToColName(i);
+      double noDictSize;
+      double withDictSize;
+      long colDataSizeWithoutDictionary = _inputManager.getColDataSizeWithoutDictionary(colName);
+      double numValuesPerEntry = _inputManager.getNumValuesPerEntry(colName);
+      int bitCompressedDataSize = _inputManager.getBitCompressedDataSize(colName);
+      long dictionarySize = _inputManager.getDictionarySize(colName);
+      double cardinality = _inputManager.getCardinality(colName);
+      long numRecordsPerPush = _inputManager.getNumRecordsPerPush();
+      LOGGER.debug("colDataSizeWithoutDictionary {}", colDataSizeWithoutDictionary);
+      LOGGER.debug("bitCompressedDataSize {}", bitCompressedDataSize);
+      LOGGER.debug("dictionarySize {}", dictionarySize);
+      LOGGER.debug("numValuesPerEntry {}", numValuesPerEntry);
+
+      if (_inputManager.getTableType().equalsIgnoreCase(REALTIME)) {
+        //TODO: improve this estimation
+        noDictSize = // size of one segment flushed ith no dictionary
+            colDataSizeWithoutDictionary * numValuesPerEntry * _params.SEGMENT_FLUSH_TIME;
+        withDictSize = // size of one flushed segment with dictionary
+            dictionarySize + bitCompressedDataSize * numValuesPerEntry * _params.SEGMENT_FLUSH_TIME;
+      } else { // For hybrid or offline table, nodictionary follows the offline side
+        noDictSize = // size of all segments in one push  with no dictionary
+            colDataSizeWithoutDictionary * numValuesPerEntry * numRecordsPerPush;
+        withDictSize = // size of all segments in one push with dictionary
+            dictionarySize * dictionaryCoefficient(cardinality, numRecordsPerPush) * DEFAUlT_NUM_PARTITIONS
+                + bitCompressedDataSize * numValuesPerEntry * numRecordsPerPush;
+      }
+
+      double storageSaved = (double) (noDictSize - withDictSize) / noDictSize;
+      LOGGER.debug("colName {}, noDictSize {}, withDictSize{}, storageSaved{}", colName, noDictSize, withDictSize,
+          storageSaved);
+
+      if (storageSaved > _params.THRESHOLD_MIN_PERCENT_DICTIONARY_STORAGE_SVAE) {
+        noDictCols.remove(colName);
+      }
+    }
+
+    // Add the no dictionary cols to config
+    _outputManager.getIndexConfig().getNoDictionaryColumns().addAll(noDictCols);
+
+    //**********On heap dictionary recommendation*******/
+    if (_inputManager.getQps() > _params.THRESHOLD_MIN_QPS_ON_HEAP) { // QPS > THRESHOLD_MIN_QPS_ON_HEAP
+      for (String colName : _inputManager.getColNameToIntMap().keySet()) {
+        if (!_outputManager.getIndexConfig().getNoDictionaryColumns().contains(colName)) //exclude no dictionary column
+        {
+          long dictionarySize = _inputManager.getDictionarySize(colName);
+          int colId = _inputManager.colNameToInt(colName);
+          double filterGroupByFreq = filterGroupByWeights[colId] / totalWeight.get();
+          if (filterGroupByFreq > _params.THRESHOLD_MIN_FILTER_FREQ_ON_HEAP  //frequently used in filter/group by
+              && dictionarySize < _params.THRESHOLD_MAX_DICTIONARY_SIZE_ON_HEAP) { // memory foot print < threshold
+            _outputManager.getIndexConfig().getOnHeapDictionaryColumns().add(colName);
+          }
+        }
+      }
+    }
+  }
+
+  private double dictionaryCoefficient(double cardinality, long numRecordsPerPush) {
+    return 1 - min(max(DEFAULT_DICT_COEFF_A * Math.log(DEFAULT_DICT_COEFF_B * cardinality / numRecordsPerPush),
+        DEFAULT_DICT_LOWER), DEFAULT_DICT_UPPER);
+  }
+
+  public void parseQuery(String queryString, double weight, double[] filterGroupByWeights, double[] selectionWeights) {

Review comment:
       General question - Would it be possible to parse the query exactly once before the execution of first rule begins. Right now, it seems like as the rules are fired in order, each rule will parse the input query set? Even though the algorithm of each rule is different, is it possible to parse once and extract all the common info needed by all the rules?




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466078322



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/NoDictionaryOnHeapDictionaryJointRule.java
##########
@@ -0,0 +1,245 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.NoDictionaryOnHeapDictionaryJointRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.NoDictionaryOnHeapDictionaryJointRule.*;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.REALTIME;
+
+
+public class NoDictionaryOnHeapDictionaryJointRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(NoDictionaryOnHeapDictionaryJointRule.class);
+  private final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+  private final NoDictionaryOnHeapDictionaryJointRuleParams _params;
+
+  public NoDictionaryOnHeapDictionaryJointRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    _params = inputManager.getNoDictionaryOnHeapDictionaryJointRuleParams();
+  }
+
+  @Override
+  public void run() {
+    LOGGER.info("Recommending no dictionary and on-heap dictionaries");
+
+    int numCols = _inputManager.getNumCols();
+    double[] filterGroupByWeights = new double[numCols];
+    double[] selectionWeights = new double[numCols];
+    AtomicDouble totalWeight = new AtomicDouble(0);
+
+    //**********No dictionary recommendation*******/
+    Set<String> noDictCols = new HashSet<>(_inputManager.getColNameToIntMap().keySet());
+
+    //Exclude cols with index
+    noDictCols.removeAll(_outputManager.getIndexConfig().getInvertedIndexColumns());
+    noDictCols.removeAll(_outputManager.getIndexConfig().getSortedColumn());
+    // TODO: Remove this after range index is implemented for no-dictionary
+    noDictCols.removeAll(_outputManager.getIndexConfig().getRangeIndexColumns());
+    LOGGER.debug("noDictCols {}", noDictCols);
+    //Find out columns used in filter&groupby and selection and corresponding frequencies
+    _inputManager.getQueryWeightMap().forEach((query, weight) -> {
+      parseQuery(query, weight, filterGroupByWeights, selectionWeights);
+      totalWeight.addAndGet(weight);
+    });
+
+    //Add dictionary on columns used in filter&groupby , with frequency > threshold
+    for (int i = 0; i < numCols; i++) {
+      double filterGroupByFreq = filterGroupByWeights[i] / totalWeight.get();
+      if (filterGroupByFreq > _params.THRESHOLD_MIN_FILTER_FREQ_DICTIONARY) {
+        noDictCols.remove(_inputManager.intToColName(i));
+      }
+    }
+
+    LOGGER.debug("filterGroupByWeights {}, selectionWeights{}, totalWeight{} ", filterGroupByWeights, selectionWeights,
+        totalWeight);
+    LOGGER.debug("noDictCols {}", noDictCols);
+
+    for (int i = 0; i < numCols; i++) {
+      // No dictionary on columns frequently used in selection

Review comment:
       added




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465940006



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java
##########
@@ -141,6 +142,18 @@ public SuccessResponse addTable(String tableConfigStr) {
     }
   }
 
+  @PUT
+  @Produces(MediaType.APPLICATION_JSON)

Review comment:
       got it




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465794901



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/NoDictionaryOnHeapDictionaryJointRule.java
##########
@@ -0,0 +1,245 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.NoDictionaryOnHeapDictionaryJointRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.NoDictionaryOnHeapDictionaryJointRule.*;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.REALTIME;
+
+
+public class NoDictionaryOnHeapDictionaryJointRule extends AbstractRule {

Review comment:
       Please add javadoc and brief explanation of the algorithm (please do this for all rules)

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/NoDictionaryOnHeapDictionaryJointRule.java
##########
@@ -0,0 +1,245 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.NoDictionaryOnHeapDictionaryJointRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.NoDictionaryOnHeapDictionaryJointRule.*;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.REALTIME;
+
+
+public class NoDictionaryOnHeapDictionaryJointRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(NoDictionaryOnHeapDictionaryJointRule.class);
+  private final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+  private final NoDictionaryOnHeapDictionaryJointRuleParams _params;
+
+  public NoDictionaryOnHeapDictionaryJointRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    _params = inputManager.getNoDictionaryOnHeapDictionaryJointRuleParams();
+  }
+
+  @Override
+  public void run() {
+    LOGGER.info("Recommending no dictionary and on-heap dictionaries");
+
+    int numCols = _inputManager.getNumCols();
+    double[] filterGroupByWeights = new double[numCols];
+    double[] selectionWeights = new double[numCols];
+    AtomicDouble totalWeight = new AtomicDouble(0);
+
+    //**********No dictionary recommendation*******/
+    Set<String> noDictCols = new HashSet<>(_inputManager.getColNameToIntMap().keySet());
+
+    //Exclude cols with index
+    noDictCols.removeAll(_outputManager.getIndexConfig().getInvertedIndexColumns());
+    noDictCols.removeAll(_outputManager.getIndexConfig().getSortedColumn());
+    // TODO: Remove this after range index is implemented for no-dictionary
+    noDictCols.removeAll(_outputManager.getIndexConfig().getRangeIndexColumns());
+    LOGGER.debug("noDictCols {}", noDictCols);
+    //Find out columns used in filter&groupby and selection and corresponding frequencies
+    _inputManager.getQueryWeightMap().forEach((query, weight) -> {
+      parseQuery(query, weight, filterGroupByWeights, selectionWeights);
+      totalWeight.addAndGet(weight);
+    });
+
+    //Add dictionary on columns used in filter&groupby , with frequency > threshold
+    for (int i = 0; i < numCols; i++) {

Review comment:
       Not sure if I follow this. If the column is in filter and group by, why do we have to consider the frequency?




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466030098



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/RecommenderConstants.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.io.params;
+
+public class RecommenderConstants {
+  public static class InvertedSortedIndexJointRule {
+    public static final double DEFAULT_PERCENT_SELECT_FOR_FUNCTION = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_TEXT_MATCH = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_RANGE = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_REGEX = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_ISNULL = 0.5d;
+    public static final double DEFAULT_THRESHOLD_MIN_AND_PREDICATE_INCREMENTAL_VOTE = 0.6d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_AND_PREDICATE_TOP_CANDIDATES = 0.8d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_GAIN_DIFF_BETWEEN_ITERATION = 0.05d;
+    public static final int DEFAULT_MAX_NUM_ITERATION_WITHOUT_GAIN = 3;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_NESI_FOR_TOP_CANDIDATES = 0.7d;
+  }
+
+  public static class RulesToExecute {
+    public static final boolean DEFAULT_RECOMMEND_FLAG_QUERY = true;
+    public static final boolean DEFAULT_RECOMMEND_VARIED_LENGTH_DICTIONARY = true;
+    public static final boolean DEFAULT_RECOMMEND_KAFKA_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_PINOT_TABLE_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_INVERTED_SORTED_INDEX_JOINT = true;
+    public static final boolean DEFAULT_RECOMMEND_BLOOM_FILTER = true;
+    public static final boolean DEFAULT_RECOMMEND_NO_DICTIONARY_ONHEAP_DICTIONARY_JOINT = true;
+  }
+
+  public static class PartitionRule {
+    public static final int DEFAULT_NUM_PARTITIONS = 0;
+
+    public static final long DEFAULT_THRESHOLD_MAX_SLA_PARTITION = 1000;
+    public static final long DEFAULT_THRESHOLD_MIN_QPS_PARTITION = 200;
+    public static final long DEFAULT_OPTIMAL_SIZE_PER_SEGMENT = 2000_000_000; //2GB
+    public static final long DEFAULT_KAFKA_NUM_MESSAGES_PER_SEC_PER_PARTITION = 250;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_DIMENSION_PARTITION_TOP_CANDIDATES = 0.8d;
+    public static final int DEFAULT_THRESHOLD_MAX_IN_LENGTH = 4;
+  }
+
+  public static class BloomFilterRule {
+    public static final long DEFAULT_THRESHOLD_MAX_CARDINALITY_BLOOMFILTER = 1000_000;
+    public static final double DEFAULT_THRESHOLD_MIN_PERCENT_EQ_BLOOMFILTER = 0.5d;
+  }
+
+  public static class NoDictionaryOnHeapDictionaryJointRule {
+    public static final double DEFAULT_THRESHOLD_MIN_FILTER_FREQ_DICTIONARY = 0d;
+    public static final double DEFAULT_THRESHOLD_MAX_SELECTION_FREQ_DICTIONARY = 0.3d;
+    public static final long DEFAULT_THRESHOLD_MIN_QPS_ON_HEAP = 10_000;
+    public static final long DEFAULT_THRESHOLD_MAX_DICTIONARY_SIZE_ON_HEAP = 1000_000L;
+    public static final double DEFAULT_THRESHOLD_MIN_FILTER_FREQ_ON_HEAP = 0.3d;
+    public static final double DEFAULT_THRESHOLD_MIN_PERCENT_DICTIONARY_STORAGE_SVAE = 0.95;
+
+    public static final double DEFAULT_DICT_COEFF_A = 0.217769;
+    public static final double DEFAULT_DICT_COEFF_B = 89.0975;
+    public static final double DEFAULT_DICT_LOWER = 0;
+    public static final double DEFAULT_DICT_UPPER = 0;
+    public static final int DEFAUlT_NUM_PARTITIONS = 16;
+
+    public static final int DEFAULT_SEGMENT_FLUSH_TIME = 86400;
+  }
+
+  public static class FlagQueryRuleParams{
+    public static final long DEFAULT_THRESHOLD_MAX_LIMIT_SIZE = 100000;
+    public static final String WARNING_NO_FILTERING = "Warning: No filtering in ths query";
+    public static final String WARNING_NO_TIME_COL = "Warning: No time column used in ths query";
+    public static final String WARNING_TOO_LONG_LIMIT = "Warning: The size of LIMIT is longer than " + DEFAULT_THRESHOLD_MAX_LIMIT_SIZE;
+    public static final String ERROR_INVALID_QUERY = "Error: query not able to parse, skipped";
+  }
+
+  public static final String PQL = "pql";
+  public static final String SQL = "sql";
+  public static final String OFFLINE = "offline";
+  public static final String REALTIME = "realtime";
+  public static final String HYBRID = "hybrid";
+  public static final int NO_SUCH_COL = -1;
+  public static final double DEFAULT_CARDINALITY = 1;
+  public static final double MIN_CARDINALITY = 1;
+  public static final double DEFAULT_AVERAGE_NUM_VALUES_PER_ENTRY = 1d;

Review comment:
       done, I think on Dino side he can probably make these fields "required" 




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465998503



##########
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) {

Review comment:
       done




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466096424



##########
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:
       added comments




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465921444



##########
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:
       Yes I will add java docs...




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466017475



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/NoDictionaryOnHeapDictionaryJointRuleParams.java
##########
@@ -0,0 +1,98 @@
+/**
+ * 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.io.params;
+
+import com.fasterxml.jackson.annotation.JsonSetter;
+import com.fasterxml.jackson.annotation.Nulls;
+
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.NoDictionaryOnHeapDictionaryJointRule.*;
+
+
+public class NoDictionaryOnHeapDictionaryJointRuleParams {
+  public Double THRESHOLD_MIN_PERCENT_DICTIONARY_STORAGE_SVAE = DEFAULT_THRESHOLD_MIN_PERCENT_DICTIONARY_STORAGE_SVAE;

Review comment:
       fixed




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#issuecomment-666598254


   Fantastic work @jasperjiaguo . Really looking forward to using this. Please give me a day or two to finish the review. 
   This is amazing. Thanks


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



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


[GitHub] [incubator-pinot] siddharthteotia merged pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia merged pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774


   


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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465817604



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/NoDictionaryOnHeapDictionaryJointRuleParams.java
##########
@@ -0,0 +1,98 @@
+/**
+ * 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.io.params;
+
+import com.fasterxml.jackson.annotation.JsonSetter;
+import com.fasterxml.jackson.annotation.Nulls;
+
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.NoDictionaryOnHeapDictionaryJointRule.*;
+
+
+public class NoDictionaryOnHeapDictionaryJointRuleParams {
+  public Double THRESHOLD_MIN_PERCENT_DICTIONARY_STORAGE_SVAE = DEFAULT_THRESHOLD_MIN_PERCENT_DICTIONARY_STORAGE_SVAE;

Review comment:
       typo in name?




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465800915



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/NoDictionaryOnHeapDictionaryJointRule.java
##########
@@ -0,0 +1,245 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.NoDictionaryOnHeapDictionaryJointRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.NoDictionaryOnHeapDictionaryJointRule.*;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.REALTIME;
+
+
+public class NoDictionaryOnHeapDictionaryJointRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(NoDictionaryOnHeapDictionaryJointRule.class);
+  private final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+  private final NoDictionaryOnHeapDictionaryJointRuleParams _params;
+
+  public NoDictionaryOnHeapDictionaryJointRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    _params = inputManager.getNoDictionaryOnHeapDictionaryJointRuleParams();
+  }
+
+  @Override
+  public void run() {
+    LOGGER.info("Recommending no dictionary and on-heap dictionaries");
+
+    int numCols = _inputManager.getNumCols();
+    double[] filterGroupByWeights = new double[numCols];
+    double[] selectionWeights = new double[numCols];
+    AtomicDouble totalWeight = new AtomicDouble(0);
+
+    //**********No dictionary recommendation*******/
+    Set<String> noDictCols = new HashSet<>(_inputManager.getColNameToIntMap().keySet());
+
+    //Exclude cols with index
+    noDictCols.removeAll(_outputManager.getIndexConfig().getInvertedIndexColumns());
+    noDictCols.removeAll(_outputManager.getIndexConfig().getSortedColumn());
+    // TODO: Remove this after range index is implemented for no-dictionary
+    noDictCols.removeAll(_outputManager.getIndexConfig().getRangeIndexColumns());
+    LOGGER.debug("noDictCols {}", noDictCols);
+    //Find out columns used in filter&groupby and selection and corresponding frequencies
+    _inputManager.getQueryWeightMap().forEach((query, weight) -> {
+      parseQuery(query, weight, filterGroupByWeights, selectionWeights);
+      totalWeight.addAndGet(weight);
+    });
+
+    //Add dictionary on columns used in filter&groupby , with frequency > threshold
+    for (int i = 0; i < numCols; i++) {
+      double filterGroupByFreq = filterGroupByWeights[i] / totalWeight.get();
+      if (filterGroupByFreq > _params.THRESHOLD_MIN_FILTER_FREQ_DICTIONARY) {
+        noDictCols.remove(_inputManager.intToColName(i));
+      }
+    }
+
+    LOGGER.debug("filterGroupByWeights {}, selectionWeights{}, totalWeight{} ", filterGroupByWeights, selectionWeights,
+        totalWeight);
+    LOGGER.debug("noDictCols {}", noDictCols);
+
+    for (int i = 0; i < numCols; i++) {
+      // No dictionary on columns frequently used in selection
+      double selectionFreq = selectionWeights[i] / totalWeight.get();
+      if (selectionFreq > _params.THRESHOLD_MAX_SELECTION_FREQ_DICTIONARY) {
+        continue;
+      }
+
+      // Add dictionary on columns NOT frequently used in selection
+      // AND can save storage > threshold
+      String colName = _inputManager.intToColName(i);
+      double noDictSize;
+      double withDictSize;
+      long colDataSizeWithoutDictionary = _inputManager.getColDataSizeWithoutDictionary(colName);
+      double numValuesPerEntry = _inputManager.getNumValuesPerEntry(colName);
+      int bitCompressedDataSize = _inputManager.getBitCompressedDataSize(colName);
+      long dictionarySize = _inputManager.getDictionarySize(colName);
+      double cardinality = _inputManager.getCardinality(colName);
+      long numRecordsPerPush = _inputManager.getNumRecordsPerPush();
+      LOGGER.debug("colDataSizeWithoutDictionary {}", colDataSizeWithoutDictionary);
+      LOGGER.debug("bitCompressedDataSize {}", bitCompressedDataSize);
+      LOGGER.debug("dictionarySize {}", dictionarySize);
+      LOGGER.debug("numValuesPerEntry {}", numValuesPerEntry);
+
+      if (_inputManager.getTableType().equalsIgnoreCase(REALTIME)) {
+        //TODO: improve this estimation
+        noDictSize = // size of one segment flushed ith no dictionary
+            colDataSizeWithoutDictionary * numValuesPerEntry * _params.SEGMENT_FLUSH_TIME;

Review comment:
       We should add a TODO though -- stating to enhance this to consider MV columns as noDictionary in the future when Pinot supports that




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465941596



##########
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:
       Got it. Thanks




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465835848



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/RecommenderConstants.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.io.params;
+
+public class RecommenderConstants {
+  public static class InvertedSortedIndexJointRule {
+    public static final double DEFAULT_PERCENT_SELECT_FOR_FUNCTION = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_TEXT_MATCH = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_RANGE = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_REGEX = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_ISNULL = 0.5d;
+    public static final double DEFAULT_THRESHOLD_MIN_AND_PREDICATE_INCREMENTAL_VOTE = 0.6d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_AND_PREDICATE_TOP_CANDIDATES = 0.8d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_GAIN_DIFF_BETWEEN_ITERATION = 0.05d;
+    public static final int DEFAULT_MAX_NUM_ITERATION_WITHOUT_GAIN = 3;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_NESI_FOR_TOP_CANDIDATES = 0.7d;
+  }
+
+  public static class RulesToExecute {
+    public static final boolean DEFAULT_RECOMMEND_FLAG_QUERY = true;
+    public static final boolean DEFAULT_RECOMMEND_VARIED_LENGTH_DICTIONARY = true;
+    public static final boolean DEFAULT_RECOMMEND_KAFKA_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_PINOT_TABLE_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_INVERTED_SORTED_INDEX_JOINT = true;
+    public static final boolean DEFAULT_RECOMMEND_BLOOM_FILTER = true;
+    public static final boolean DEFAULT_RECOMMEND_NO_DICTIONARY_ONHEAP_DICTIONARY_JOINT = true;
+  }
+
+  public static class PartitionRule {
+    public static final int DEFAULT_NUM_PARTITIONS = 0;
+
+    public static final long DEFAULT_THRESHOLD_MAX_SLA_PARTITION = 1000;
+    public static final long DEFAULT_THRESHOLD_MIN_QPS_PARTITION = 200;
+    public static final long DEFAULT_OPTIMAL_SIZE_PER_SEGMENT = 2000_000_000; //2GB

Review comment:
       I think 2GB is on the high end. Let's just start with 1GB probably




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465794494



##########
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 {
+  public static class RuleFactory {
+    public static AbstractRule getRule(Rules rule, InputManager inputManager, ConfigManager outputManager) {
+      switch (rule) {
+        case FlagQueryRule:
+          return new FlagQueryRule(inputManager, outputManager);
+        case InvertedSortedIndexJointRule:
+          return new InvertedSortedIndexJointRule(inputManager, outputManager);
+        case KafkaPartitionRule:
+          return new KafkaPartitionRule(inputManager, outputManager);
+        case PinotTablePartitionRule:
+          return new PinotTablePartitionRule(inputManager, outputManager);
+        case BloomFilterRule:
+          return new BloomFilterRule(inputManager, outputManager);
+        case NoDictionaryOnHeapDictionaryJointRule:
+          return new NoDictionaryOnHeapDictionaryJointRule(inputManager, outputManager);
+        case VariedLengthDictionaryRule:
+          return new VariedLengthDictionaryRule(inputManager, outputManager);
+        default:
+          return null;
+      }
+    }
+  }
+  // All rules will execute by default unless explicitly specifying "recommendInvertedSortedIndexJoint" = "false"
+  boolean _recommendKafkaPartition = DEFAULT_RECOMMEND_KAFKA_PARTITION;
+  boolean _recommendPinotTablePartition = DEFAULT_RECOMMEND_PINOT_TABLE_PARTITION;
+  boolean _recommendInvertedSortedIndexJoint = DEFAULT_RECOMMEND_INVERTED_SORTED_INDEX_JOINT;
+  boolean _recommendBloomFilter = DEFAULT_RECOMMEND_BLOOM_FILTER;
+  boolean _recommendNoDictionaryOnHeapDictionaryJoint = DEFAULT_RECOMMEND_NO_DICTIONARY_ONHEAP_DICTIONARY_JOINT;
+  boolean _recommendVariedLengthDictionary = DEFAULT_RECOMMEND_VARIED_LENGTH_DICTIONARY;
+  boolean _recommendFlagQuery = DEFAULT_RECOMMEND_FLAG_QUERY;
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setRecommendVariedLengthDictionary(boolean recommendVariedLengthDictionary) {
+    _recommendVariedLengthDictionary = recommendVariedLengthDictionary;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setRecommendFlagQuery(boolean recommendFlagQuery) {
+    _recommendFlagQuery = recommendFlagQuery;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setRecommendNoDictionaryOnHeapDictionaryJoint(boolean recommendNoDictionaryOnHeapDictionaryJoint) {
+    _recommendNoDictionaryOnHeapDictionaryJoint = recommendNoDictionaryOnHeapDictionaryJoint;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setRecommendKafkaPartition(boolean recommendKafkaPartition) {
+    _recommendKafkaPartition = recommendKafkaPartition;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setRecommendInvertedSortedIndexJoint(boolean recommendInvertedSortedIndexJoint) {
+    _recommendInvertedSortedIndexJoint = recommendInvertedSortedIndexJoint;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setRecommendPinotTablePartition(boolean recommendPinotTablePartition) {
+    _recommendPinotTablePartition = recommendPinotTablePartition;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setRecommendBloomFilter(boolean recommendBloomFilter) {
+    _recommendBloomFilter = recommendBloomFilter;
+  }
+
+  public boolean isRecommendVariedLengthDictionary() {
+    return _recommendVariedLengthDictionary;
+  }
+
+  public boolean isRecommendFlagQuery() {
+    return _recommendFlagQuery;
+  }
+
+  public boolean isRecommendNoDictionaryOnHeapDictionaryJoint() {
+    return _recommendNoDictionaryOnHeapDictionaryJoint;
+  }
+
+  public boolean isRecommendKafkaPartition() {
+    return _recommendKafkaPartition;
+  }
+
+  public boolean isRecommendInvertedSortedIndexJoint() {
+    return _recommendInvertedSortedIndexJoint;
+  }
+
+  public boolean isRecommendPinotTablePartition() {
+    return _recommendPinotTablePartition;
+  }
+
+  public boolean isRecommendBloomFilter() {
+    return _recommendBloomFilter;
+  }
+
+  public enum Rules {

Review comment:
       The enum name shouldn't really have "plural". The enum although defines multiple constants, it represents only 1 of them at a give time. So, we should simply call it Rule?

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/BloomFilterRule.java
##########
@@ -0,0 +1,142 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.BloomFilterRuleParams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class  BloomFilterRule extends AbstractRule {

Review comment:
       Please add javadoc and brief explanation of the rule's algorithm. We already have that in the design, so just englishize it here :)

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/BloomFilterRule.java
##########
@@ -0,0 +1,142 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.BloomFilterRuleParams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class  BloomFilterRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(BloomFilterRule.class);
+  private final BloomFilterRuleParams _params;
+  protected final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+
+  public BloomFilterRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    _params = inputManager.getBloomFilterRuleParams();
+  }
+
+  @Override
+  public void run() {
+    int numDims = _inputManager.getNumDims();
+    double[] weights = new double[numDims];
+    AtomicDouble totalWeight = new AtomicDouble(0);
+
+    // For each query, find out the dimensions used in 'EQ'
+    // and accumulate the (weighted) frequencies
+    _inputManager.getQueryWeightMap().forEach((query,weight) -> {
+      totalWeight.addAndGet(weight);
+      FixedLenBitset fixedLenBitset = parseQuery(query);
+      LOGGER.debug("fixedLenBitset {}", fixedLenBitset);
+      for (Integer i : fixedLenBitset.getOffsets()) {
+        weights[i] += weight;
+      }
+    });
+    LOGGER.debug("Weight: {}, Total {}", weights, totalWeight);
+
+    for (int i = 0; i < numDims; i++) {
+      String dimName = _inputManager.intToColName(i);
+      if (((weights[i] / totalWeight.get()) > _params.THRESHOLD_MIN_PERCENT_EQ_BLOOMFILTER)
+          //The partitioned dimension should be frequently > P used
+          && (_inputManager.getCardinality(dimName)
+          < _params.THRESHOLD_MAX_CARDINALITY_BLOOMFILTER)) { //The Cardinality < C (1 million for 1MB size)
+        _outputManager.getIndexConfig().getBloomFilterColumns().add(dimName);
+      }
+    }
+  }
+
+  public FixedLenBitset parseQuery(String queryString) {
+    LOGGER.debug("Parsing query: {}", queryString);
+    if (queryString == null) {
+      return FixedLenBitset.IMMUTABLE_EMPTY_SET;
+    }
+
+    BrokerRequest brokerRequest;
+    AbstractCompiler parser = PinotQueryParserFactory.get(_inputManager.getQueryType());
+    try {
+      brokerRequest = parser.compileToBrokerRequest(queryString);
+    } catch (SqlCompilationException e) {
+      LOGGER.error("Error parsing query: {}, {}", queryString, e.getMessage());
+      return FixedLenBitset.IMMUTABLE_EMPTY_SET;
+    }
+    BrokerRequest optimizedRequest = _brokerRequestOptimizer.optimize(brokerRequest, _inputManager.getTimeCol());
+    QueryContext queryContext = BrokerRequestToQueryContextConverter.convert(optimizedRequest);
+
+    if (queryContext.getFilter() == null) {
+      return FixedLenBitset.IMMUTABLE_EMPTY_SET;
+    }
+
+    LOGGER.trace("Parsing Where Clause: {}", queryContext.getFilter().toString());
+    return parsePredicateList(queryContext.getFilter());
+  }
+
+  /**
+   * The partitioned dimension should used in the “=” (IN, NOT IN, != are not using bloom filter in Pinot for now) filter.
+   * @param filterContext filterContext

Review comment:
       Mark this is as a TODO since it can be easily spotted in the code if someone stumbles upon it in the future. 
   TODO: once Pinot starts supporting bloom filter based pruning for IN, !=, NOT IN, we should enhance the algorithm of this rule. 




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466058271



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/NoDictionaryOnHeapDictionaryJointRule.java
##########
@@ -0,0 +1,245 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.NoDictionaryOnHeapDictionaryJointRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.NoDictionaryOnHeapDictionaryJointRule.*;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.REALTIME;
+
+
+public class NoDictionaryOnHeapDictionaryJointRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(NoDictionaryOnHeapDictionaryJointRule.class);
+  private final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+  private final NoDictionaryOnHeapDictionaryJointRuleParams _params;
+
+  public NoDictionaryOnHeapDictionaryJointRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    _params = inputManager.getNoDictionaryOnHeapDictionaryJointRuleParams();
+  }
+
+  @Override
+  public void run() {
+    LOGGER.info("Recommending no dictionary and on-heap dictionaries");
+
+    int numCols = _inputManager.getNumCols();
+    double[] filterGroupByWeights = new double[numCols];
+    double[] selectionWeights = new double[numCols];
+    AtomicDouble totalWeight = new AtomicDouble(0);
+
+    //**********No dictionary recommendation*******/
+    Set<String> noDictCols = new HashSet<>(_inputManager.getColNameToIntMap().keySet());
+
+    //Exclude cols with index
+    noDictCols.removeAll(_outputManager.getIndexConfig().getInvertedIndexColumns());
+    noDictCols.removeAll(_outputManager.getIndexConfig().getSortedColumn());
+    // TODO: Remove this after range index is implemented for no-dictionary
+    noDictCols.removeAll(_outputManager.getIndexConfig().getRangeIndexColumns());
+    LOGGER.debug("noDictCols {}", noDictCols);
+    //Find out columns used in filter&groupby and selection and corresponding frequencies
+    _inputManager.getQueryWeightMap().forEach((query, weight) -> {
+      parseQuery(query, weight, filterGroupByWeights, selectionWeights);
+      totalWeight.addAndGet(weight);
+    });
+
+    //Add dictionary on columns used in filter&groupby , with frequency > threshold
+    for (int i = 0; i < numCols; i++) {
+      double filterGroupByFreq = filterGroupByWeights[i] / totalWeight.get();
+      if (filterGroupByFreq > _params.THRESHOLD_MIN_FILTER_FREQ_DICTIONARY) {
+        noDictCols.remove(_inputManager.intToColName(i));
+      }
+    }
+
+    LOGGER.debug("filterGroupByWeights {}, selectionWeights{}, totalWeight{} ", filterGroupByWeights, selectionWeights,
+        totalWeight);
+    LOGGER.debug("noDictCols {}", noDictCols);
+
+    for (int i = 0; i < numCols; i++) {
+      // No dictionary on columns frequently used in selection
+      double selectionFreq = selectionWeights[i] / totalWeight.get();
+      if (selectionFreq > _params.THRESHOLD_MAX_SELECTION_FREQ_DICTIONARY) {
+        continue;
+      }
+
+      // Add dictionary on columns NOT frequently used in selection
+      // AND can save storage > threshold
+      String colName = _inputManager.intToColName(i);
+      double noDictSize;
+      double withDictSize;
+      long colDataSizeWithoutDictionary = _inputManager.getColDataSizeWithoutDictionary(colName);
+      double numValuesPerEntry = _inputManager.getNumValuesPerEntry(colName);
+      int bitCompressedDataSize = _inputManager.getBitCompressedDataSize(colName);
+      long dictionarySize = _inputManager.getDictionarySize(colName);
+      double cardinality = _inputManager.getCardinality(colName);
+      long numRecordsPerPush = _inputManager.getNumRecordsPerPush();
+      LOGGER.debug("colDataSizeWithoutDictionary {}", colDataSizeWithoutDictionary);
+      LOGGER.debug("bitCompressedDataSize {}", bitCompressedDataSize);
+      LOGGER.debug("dictionarySize {}", dictionarySize);
+      LOGGER.debug("numValuesPerEntry {}", numValuesPerEntry);
+
+      if (_inputManager.getTableType().equalsIgnoreCase(REALTIME)) {
+        //TODO: improve this estimation
+        noDictSize = // size of one segment flushed ith no dictionary
+            colDataSizeWithoutDictionary * numValuesPerEntry * _params.SEGMENT_FLUSH_TIME;

Review comment:
       Done. Thanks for bring out this!




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465796631



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/NoDictionaryOnHeapDictionaryJointRule.java
##########
@@ -0,0 +1,245 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.NoDictionaryOnHeapDictionaryJointRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.NoDictionaryOnHeapDictionaryJointRule.*;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.REALTIME;
+
+
+public class NoDictionaryOnHeapDictionaryJointRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(NoDictionaryOnHeapDictionaryJointRule.class);
+  private final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+  private final NoDictionaryOnHeapDictionaryJointRuleParams _params;
+
+  public NoDictionaryOnHeapDictionaryJointRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    _params = inputManager.getNoDictionaryOnHeapDictionaryJointRuleParams();
+  }
+
+  @Override
+  public void run() {
+    LOGGER.info("Recommending no dictionary and on-heap dictionaries");
+
+    int numCols = _inputManager.getNumCols();
+    double[] filterGroupByWeights = new double[numCols];
+    double[] selectionWeights = new double[numCols];
+    AtomicDouble totalWeight = new AtomicDouble(0);
+
+    //**********No dictionary recommendation*******/
+    Set<String> noDictCols = new HashSet<>(_inputManager.getColNameToIntMap().keySet());
+
+    //Exclude cols with index
+    noDictCols.removeAll(_outputManager.getIndexConfig().getInvertedIndexColumns());
+    noDictCols.removeAll(_outputManager.getIndexConfig().getSortedColumn());
+    // TODO: Remove this after range index is implemented for no-dictionary
+    noDictCols.removeAll(_outputManager.getIndexConfig().getRangeIndexColumns());
+    LOGGER.debug("noDictCols {}", noDictCols);
+    //Find out columns used in filter&groupby and selection and corresponding frequencies
+    _inputManager.getQueryWeightMap().forEach((query, weight) -> {
+      parseQuery(query, weight, filterGroupByWeights, selectionWeights);
+      totalWeight.addAndGet(weight);
+    });
+
+    //Add dictionary on columns used in filter&groupby , with frequency > threshold
+    for (int i = 0; i < numCols; i++) {
+      double filterGroupByFreq = filterGroupByWeights[i] / totalWeight.get();
+      if (filterGroupByFreq > _params.THRESHOLD_MIN_FILTER_FREQ_DICTIONARY) {
+        noDictCols.remove(_inputManager.intToColName(i));
+      }
+    }
+
+    LOGGER.debug("filterGroupByWeights {}, selectionWeights{}, totalWeight{} ", filterGroupByWeights, selectionWeights,
+        totalWeight);
+    LOGGER.debug("noDictCols {}", noDictCols);
+
+    for (int i = 0; i < numCols; i++) {
+      // No dictionary on columns frequently used in selection

Review comment:
       Might want to add comment about the experiment done during the design phase -- for a column heavily used in selection only (not part of filter or group by), making it no dictionary reduces the latency by 20% (I guess) since we avoid the 2 lookups. 




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#issuecomment-669285729


   @jasperjiaguo , I have reviewed pretty much all of it. There are a few (2-3) classes that I would like to spend some more time eyeballing the code. Meanwhile, you may want to start going through the above comments


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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466076071



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/PinotTablePartitionRule.java
##########
@@ -0,0 +1,228 @@
+/**
+ * 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.impl;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Optional;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.InPredicate;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.PartitionRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.*;
+
+
+public class PinotTablePartitionRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(PinotTablePartitionRule.class);
+  PartitionRuleParams _params;
+
+  protected final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+
+  public PinotTablePartitionRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    this._params = inputManager.getPartitionRuleParams();
+  }
+
+  @Override
+  public void run() {
+    //**********Calculate size per record***************/
+    _inputManager.estimateSizePerRecord();
+    //**************************************************/
+
+    LOGGER.info("Recommending partition configurations");
+
+    if (_inputManager.getQps()
+        < _params.THRESHOLD_MIN_QPS_PARTITION) { //For a table whose QPS < Q (say 200 or 300) NO partitioning is needed.
+      LOGGER.info("*Input QPS {} < threshold {}, no partition needed", _inputManager.getQps(),
+          _params.THRESHOLD_MIN_QPS_PARTITION);
+      return;
+    }
+    if (_inputManager.getLatencySLA()
+        > _params.THRESHOLD_MAX_SLA_PARTITION) { //For a table whose latency SLA > L (say 1000ms) NO partitioning is needed.
+      LOGGER.info("*Input SLA {} > threshold {}, no partition needed", _inputManager.getLatencySLA(),
+          _params.THRESHOLD_MAX_SLA_PARTITION);
+      return;
+    }
+
+    LOGGER.info("*Recommending partition number");
+    if (_inputManager.getTableType().equalsIgnoreCase(
+        REALTIME)) { //real time partition num should be the same value as the number of kafka partitions

Review comment:
       done




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466079072



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/BloomFilterRule.java
##########
@@ -0,0 +1,142 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.BloomFilterRuleParams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class  BloomFilterRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(BloomFilterRule.class);
+  private final BloomFilterRuleParams _params;
+  protected final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+
+  public BloomFilterRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    _params = inputManager.getBloomFilterRuleParams();
+  }
+
+  @Override
+  public void run() {
+    int numDims = _inputManager.getNumDims();
+    double[] weights = new double[numDims];
+    AtomicDouble totalWeight = new AtomicDouble(0);
+
+    // For each query, find out the dimensions used in 'EQ'
+    // and accumulate the (weighted) frequencies
+    _inputManager.getQueryWeightMap().forEach((query,weight) -> {
+      totalWeight.addAndGet(weight);
+      FixedLenBitset fixedLenBitset = parseQuery(query);
+      LOGGER.debug("fixedLenBitset {}", fixedLenBitset);
+      for (Integer i : fixedLenBitset.getOffsets()) {
+        weights[i] += weight;
+      }
+    });
+    LOGGER.debug("Weight: {}, Total {}", weights, totalWeight);
+
+    for (int i = 0; i < numDims; i++) {
+      String dimName = _inputManager.intToColName(i);
+      if (((weights[i] / totalWeight.get()) > _params.THRESHOLD_MIN_PERCENT_EQ_BLOOMFILTER)
+          //The partitioned dimension should be frequently > P used
+          && (_inputManager.getCardinality(dimName)
+          < _params.THRESHOLD_MAX_CARDINALITY_BLOOMFILTER)) { //The Cardinality < C (1 million for 1MB size)
+        _outputManager.getIndexConfig().getBloomFilterColumns().add(dimName);
+      }
+    }
+  }
+
+  public FixedLenBitset parseQuery(String queryString) {
+    LOGGER.debug("Parsing query: {}", queryString);
+    if (queryString == null) {
+      return FixedLenBitset.IMMUTABLE_EMPTY_SET;
+    }
+
+    BrokerRequest brokerRequest;
+    AbstractCompiler parser = PinotQueryParserFactory.get(_inputManager.getQueryType());
+    try {
+      brokerRequest = parser.compileToBrokerRequest(queryString);
+    } catch (SqlCompilationException e) {
+      LOGGER.error("Error parsing query: {}, {}", queryString, e.getMessage());
+      return FixedLenBitset.IMMUTABLE_EMPTY_SET;
+    }
+    BrokerRequest optimizedRequest = _brokerRequestOptimizer.optimize(brokerRequest, _inputManager.getTimeCol());
+    QueryContext queryContext = BrokerRequestToQueryContextConverter.convert(optimizedRequest);
+
+    if (queryContext.getFilter() == null) {
+      return FixedLenBitset.IMMUTABLE_EMPTY_SET;
+    }
+
+    LOGGER.trace("Parsing Where Clause: {}", queryContext.getFilter().toString());
+    return parsePredicateList(queryContext.getFilter());
+  }
+
+  /**
+   * The partitioned dimension should used in the “=” (IN, NOT IN, != are not using bloom filter in Pinot for now) filter.
+   * @param filterContext filterContext

Review comment:
       done




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465941967



##########
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:
       Got it. They came from tuner




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466638116



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/NoDictionaryOnHeapDictionaryJointRule.java
##########
@@ -0,0 +1,245 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.NoDictionaryOnHeapDictionaryJointRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.NoDictionaryOnHeapDictionaryJointRule.*;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.REALTIME;
+
+
+public class NoDictionaryOnHeapDictionaryJointRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(NoDictionaryOnHeapDictionaryJointRule.class);
+  private final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+  private final NoDictionaryOnHeapDictionaryJointRuleParams _params;
+
+  public NoDictionaryOnHeapDictionaryJointRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    _params = inputManager.getNoDictionaryOnHeapDictionaryJointRuleParams();
+  }
+
+  @Override
+  public void run() {
+    LOGGER.info("Recommending no dictionary and on-heap dictionaries");
+
+    int numCols = _inputManager.getNumCols();
+    double[] filterGroupByWeights = new double[numCols];
+    double[] selectionWeights = new double[numCols];
+    AtomicDouble totalWeight = new AtomicDouble(0);
+
+    //**********No dictionary recommendation*******/
+    Set<String> noDictCols = new HashSet<>(_inputManager.getColNameToIntMap().keySet());
+
+    //Exclude cols with index
+    noDictCols.removeAll(_outputManager.getIndexConfig().getInvertedIndexColumns());
+    noDictCols.removeAll(_outputManager.getIndexConfig().getSortedColumn());
+    // TODO: Remove this after range index is implemented for no-dictionary
+    noDictCols.removeAll(_outputManager.getIndexConfig().getRangeIndexColumns());
+    LOGGER.debug("noDictCols {}", noDictCols);
+    //Find out columns used in filter&groupby and selection and corresponding frequencies
+    _inputManager.getQueryWeightMap().forEach((query, weight) -> {
+      parseQuery(query, weight, filterGroupByWeights, selectionWeights);
+      totalWeight.addAndGet(weight);
+    });
+
+    //Add dictionary on columns used in filter&groupby , with frequency > threshold
+    for (int i = 0; i < numCols; i++) {
+      double filterGroupByFreq = filterGroupByWeights[i] / totalWeight.get();
+      if (filterGroupByFreq > _params.THRESHOLD_MIN_FILTER_FREQ_DICTIONARY) {
+        noDictCols.remove(_inputManager.intToColName(i));
+      }
+    }
+
+    LOGGER.debug("filterGroupByWeights {}, selectionWeights{}, totalWeight{} ", filterGroupByWeights, selectionWeights,
+        totalWeight);
+    LOGGER.debug("noDictCols {}", noDictCols);
+
+    for (int i = 0; i < numCols; i++) {
+      // No dictionary on columns frequently used in selection
+      double selectionFreq = selectionWeights[i] / totalWeight.get();
+      if (selectionFreq > _params.THRESHOLD_MAX_SELECTION_FREQ_DICTIONARY) {
+        continue;
+      }
+
+      // Add dictionary on columns NOT frequently used in selection
+      // AND can save storage > threshold
+      String colName = _inputManager.intToColName(i);
+      double noDictSize;
+      double withDictSize;
+      long colDataSizeWithoutDictionary = _inputManager.getColDataSizeWithoutDictionary(colName);
+      double numValuesPerEntry = _inputManager.getNumValuesPerEntry(colName);
+      int bitCompressedDataSize = _inputManager.getBitCompressedDataSize(colName);
+      long dictionarySize = _inputManager.getDictionarySize(colName);
+      double cardinality = _inputManager.getCardinality(colName);
+      long numRecordsPerPush = _inputManager.getNumRecordsPerPush();
+      LOGGER.debug("colDataSizeWithoutDictionary {}", colDataSizeWithoutDictionary);
+      LOGGER.debug("bitCompressedDataSize {}", bitCompressedDataSize);
+      LOGGER.debug("dictionarySize {}", dictionarySize);
+      LOGGER.debug("numValuesPerEntry {}", numValuesPerEntry);
+
+      if (_inputManager.getTableType().equalsIgnoreCase(REALTIME)) {
+        //TODO: improve this estimation
+        noDictSize = // size of one segment flushed ith no dictionary
+            colDataSizeWithoutDictionary * numValuesPerEntry * _params.SEGMENT_FLUSH_TIME;
+        withDictSize = // size of one flushed segment with dictionary
+            dictionarySize + bitCompressedDataSize * numValuesPerEntry * _params.SEGMENT_FLUSH_TIME;
+      } else { // For hybrid or offline table, nodictionary follows the offline side
+        noDictSize = // size of all segments in one push  with no dictionary
+            colDataSizeWithoutDictionary * numValuesPerEntry * numRecordsPerPush;
+        withDictSize = // size of all segments in one push with dictionary
+            dictionarySize * dictionaryCoefficient(cardinality, numRecordsPerPush) * DEFAUlT_NUM_PARTITIONS
+                + bitCompressedDataSize * numValuesPerEntry * numRecordsPerPush;
+      }
+
+      double storageSaved = (double) (noDictSize - withDictSize) / noDictSize;
+      LOGGER.debug("colName {}, noDictSize {}, withDictSize{}, storageSaved{}", colName, noDictSize, withDictSize,
+          storageSaved);
+
+      if (storageSaved > _params.THRESHOLD_MIN_PERCENT_DICTIONARY_STORAGE_SVAE) {
+        noDictCols.remove(colName);
+      }
+    }
+
+    // Add the no dictionary cols to config
+    _outputManager.getIndexConfig().getNoDictionaryColumns().addAll(noDictCols);
+
+    //**********On heap dictionary recommendation*******/
+    if (_inputManager.getQps() > _params.THRESHOLD_MIN_QPS_ON_HEAP) { // QPS > THRESHOLD_MIN_QPS_ON_HEAP
+      for (String colName : _inputManager.getColNameToIntMap().keySet()) {
+        if (!_outputManager.getIndexConfig().getNoDictionaryColumns().contains(colName)) //exclude no dictionary column
+        {
+          long dictionarySize = _inputManager.getDictionarySize(colName);
+          int colId = _inputManager.colNameToInt(colName);
+          double filterGroupByFreq = filterGroupByWeights[colId] / totalWeight.get();
+          if (filterGroupByFreq > _params.THRESHOLD_MIN_FILTER_FREQ_ON_HEAP  //frequently used in filter/group by
+              && dictionarySize < _params.THRESHOLD_MAX_DICTIONARY_SIZE_ON_HEAP) { // memory foot print < threshold
+            _outputManager.getIndexConfig().getOnHeapDictionaryColumns().add(colName);
+          }
+        }
+      }
+    }
+  }
+
+  private double dictionaryCoefficient(double cardinality, long numRecordsPerPush) {
+    return 1 - min(max(DEFAULT_DICT_COEFF_A * Math.log(DEFAULT_DICT_COEFF_B * cardinality / numRecordsPerPush),
+        DEFAULT_DICT_LOWER), DEFAULT_DICT_UPPER);
+  }
+
+  public void parseQuery(String queryString, double weight, double[] filterGroupByWeights, double[] selectionWeights) {

Review comment:
       resolved - the code will parse only once




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465818633



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/RecommenderConstants.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.io.params;
+
+public class RecommenderConstants {
+  public static class InvertedSortedIndexJointRule {
+    public static final double DEFAULT_PERCENT_SELECT_FOR_FUNCTION = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_TEXT_MATCH = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_RANGE = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_REGEX = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_ISNULL = 0.5d;
+    public static final double DEFAULT_THRESHOLD_MIN_AND_PREDICATE_INCREMENTAL_VOTE = 0.6d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_AND_PREDICATE_TOP_CANDIDATES = 0.8d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_GAIN_DIFF_BETWEEN_ITERATION = 0.05d;
+    public static final int DEFAULT_MAX_NUM_ITERATION_WITHOUT_GAIN = 3;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_NESI_FOR_TOP_CANDIDATES = 0.7d;
+  }
+
+  public static class RulesToExecute {
+    public static final boolean DEFAULT_RECOMMEND_FLAG_QUERY = true;
+    public static final boolean DEFAULT_RECOMMEND_VARIED_LENGTH_DICTIONARY = true;
+    public static final boolean DEFAULT_RECOMMEND_KAFKA_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_PINOT_TABLE_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_INVERTED_SORTED_INDEX_JOINT = true;
+    public static final boolean DEFAULT_RECOMMEND_BLOOM_FILTER = true;
+    public static final boolean DEFAULT_RECOMMEND_NO_DICTIONARY_ONHEAP_DICTIONARY_JOINT = true;
+  }
+
+  public static class PartitionRule {
+    public static final int DEFAULT_NUM_PARTITIONS = 0;
+
+    public static final long DEFAULT_THRESHOLD_MAX_SLA_PARTITION = 1000;

Review comment:
       include latency in the name




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466007601



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/RecommenderConstants.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.io.params;
+
+public class RecommenderConstants {
+  public static class InvertedSortedIndexJointRule {
+    public static final double DEFAULT_PERCENT_SELECT_FOR_FUNCTION = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_TEXT_MATCH = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_RANGE = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_REGEX = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_ISNULL = 0.5d;
+    public static final double DEFAULT_THRESHOLD_MIN_AND_PREDICATE_INCREMENTAL_VOTE = 0.6d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_AND_PREDICATE_TOP_CANDIDATES = 0.8d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_GAIN_DIFF_BETWEEN_ITERATION = 0.05d;
+    public static final int DEFAULT_MAX_NUM_ITERATION_WITHOUT_GAIN = 3;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_NESI_FOR_TOP_CANDIDATES = 0.7d;
+  }
+
+  public static class RulesToExecute {
+    public static final boolean DEFAULT_RECOMMEND_FLAG_QUERY = true;
+    public static final boolean DEFAULT_RECOMMEND_VARIED_LENGTH_DICTIONARY = true;
+    public static final boolean DEFAULT_RECOMMEND_KAFKA_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_PINOT_TABLE_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_INVERTED_SORTED_INDEX_JOINT = true;
+    public static final boolean DEFAULT_RECOMMEND_BLOOM_FILTER = true;
+    public static final boolean DEFAULT_RECOMMEND_NO_DICTIONARY_ONHEAP_DICTIONARY_JOINT = true;
+  }
+
+  public static class PartitionRule {
+    public static final int DEFAULT_NUM_PARTITIONS = 0;
+
+    public static final long DEFAULT_THRESHOLD_MAX_SLA_PARTITION = 1000;
+    public static final long DEFAULT_THRESHOLD_MIN_QPS_PARTITION = 200;
+    public static final long DEFAULT_OPTIMAL_SIZE_PER_SEGMENT = 2000_000_000; //2GB

Review comment:
       changed to 1 GB




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465960789



##########
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) {

Review comment:
       This function doesn't actually return the total bit compressed size.
   What we need is the following:
   
   - get the number of bits per value
   - number of records
   
   multiply both
   
   number of bits per value  can be calculated by a function PinotDataBitSet




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465955937



##########
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) {

Review comment:
       You might want to rename it to getDictionaryEncodedForwardIndexSize




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465809956



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/PinotTablePartitionRule.java
##########
@@ -0,0 +1,228 @@
+/**
+ * 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.impl;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Optional;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.InPredicate;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.PartitionRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.*;
+
+
+public class PinotTablePartitionRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(PinotTablePartitionRule.class);
+  PartitionRuleParams _params;
+
+  protected final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+
+  public PinotTablePartitionRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    this._params = inputManager.getPartitionRuleParams();
+  }
+
+  @Override
+  public void run() {
+    //**********Calculate size per record***************/
+    _inputManager.estimateSizePerRecord();
+    //**************************************************/
+
+    LOGGER.info("Recommending partition configurations");
+
+    if (_inputManager.getQps()
+        < _params.THRESHOLD_MIN_QPS_PARTITION) { //For a table whose QPS < Q (say 200 or 300) NO partitioning is needed.
+      LOGGER.info("*Input QPS {} < threshold {}, no partition needed", _inputManager.getQps(),
+          _params.THRESHOLD_MIN_QPS_PARTITION);
+      return;
+    }
+    if (_inputManager.getLatencySLA()
+        > _params.THRESHOLD_MAX_SLA_PARTITION) { //For a table whose latency SLA > L (say 1000ms) NO partitioning is needed.
+      LOGGER.info("*Input SLA {} > threshold {}, no partition needed", _inputManager.getLatencySLA(),
+          _params.THRESHOLD_MAX_SLA_PARTITION);
+      return;
+    }
+
+    LOGGER.info("*Recommending partition number");

Review comment:
       (nit): suggest changing it to `"Recommending number of partitions"`




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465926766



##########
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:
       The fields following this line are pre-processed from the input and used as algorithm input only, so no need to serialize/deserialize them in any place




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465798271



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/NoDictionaryOnHeapDictionaryJointRule.java
##########
@@ -0,0 +1,245 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.NoDictionaryOnHeapDictionaryJointRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.NoDictionaryOnHeapDictionaryJointRule.*;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.REALTIME;
+
+
+public class NoDictionaryOnHeapDictionaryJointRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(NoDictionaryOnHeapDictionaryJointRule.class);
+  private final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+  private final NoDictionaryOnHeapDictionaryJointRuleParams _params;
+
+  public NoDictionaryOnHeapDictionaryJointRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    _params = inputManager.getNoDictionaryOnHeapDictionaryJointRuleParams();
+  }
+
+  @Override
+  public void run() {
+    LOGGER.info("Recommending no dictionary and on-heap dictionaries");
+
+    int numCols = _inputManager.getNumCols();
+    double[] filterGroupByWeights = new double[numCols];
+    double[] selectionWeights = new double[numCols];
+    AtomicDouble totalWeight = new AtomicDouble(0);
+
+    //**********No dictionary recommendation*******/
+    Set<String> noDictCols = new HashSet<>(_inputManager.getColNameToIntMap().keySet());
+
+    //Exclude cols with index
+    noDictCols.removeAll(_outputManager.getIndexConfig().getInvertedIndexColumns());
+    noDictCols.removeAll(_outputManager.getIndexConfig().getSortedColumn());
+    // TODO: Remove this after range index is implemented for no-dictionary
+    noDictCols.removeAll(_outputManager.getIndexConfig().getRangeIndexColumns());
+    LOGGER.debug("noDictCols {}", noDictCols);
+    //Find out columns used in filter&groupby and selection and corresponding frequencies
+    _inputManager.getQueryWeightMap().forEach((query, weight) -> {
+      parseQuery(query, weight, filterGroupByWeights, selectionWeights);
+      totalWeight.addAndGet(weight);
+    });
+
+    //Add dictionary on columns used in filter&groupby , with frequency > threshold
+    for (int i = 0; i < numCols; i++) {
+      double filterGroupByFreq = filterGroupByWeights[i] / totalWeight.get();
+      if (filterGroupByFreq > _params.THRESHOLD_MIN_FILTER_FREQ_DICTIONARY) {
+        noDictCols.remove(_inputManager.intToColName(i));
+      }
+    }
+
+    LOGGER.debug("filterGroupByWeights {}, selectionWeights{}, totalWeight{} ", filterGroupByWeights, selectionWeights,
+        totalWeight);
+    LOGGER.debug("noDictCols {}", noDictCols);
+
+    for (int i = 0; i < numCols; i++) {
+      // No dictionary on columns frequently used in selection
+      double selectionFreq = selectionWeights[i] / totalWeight.get();
+      if (selectionFreq > _params.THRESHOLD_MAX_SELECTION_FREQ_DICTIONARY) {
+        continue;
+      }
+
+      // Add dictionary on columns NOT frequently used in selection
+      // AND can save storage > threshold
+      String colName = _inputManager.intToColName(i);
+      double noDictSize;
+      double withDictSize;
+      long colDataSizeWithoutDictionary = _inputManager.getColDataSizeWithoutDictionary(colName);
+      double numValuesPerEntry = _inputManager.getNumValuesPerEntry(colName);
+      int bitCompressedDataSize = _inputManager.getBitCompressedDataSize(colName);
+      long dictionarySize = _inputManager.getDictionarySize(colName);
+      double cardinality = _inputManager.getCardinality(colName);
+      long numRecordsPerPush = _inputManager.getNumRecordsPerPush();
+      LOGGER.debug("colDataSizeWithoutDictionary {}", colDataSizeWithoutDictionary);
+      LOGGER.debug("bitCompressedDataSize {}", bitCompressedDataSize);
+      LOGGER.debug("dictionarySize {}", dictionarySize);
+      LOGGER.debug("numValuesPerEntry {}", numValuesPerEntry);
+
+      if (_inputManager.getTableType().equalsIgnoreCase(REALTIME)) {
+        //TODO: improve this estimation
+        noDictSize = // size of one segment flushed ith no dictionary
+            colDataSizeWithoutDictionary * numValuesPerEntry * _params.SEGMENT_FLUSH_TIME;
+        withDictSize = // size of one flushed segment with dictionary
+            dictionarySize + bitCompressedDataSize * numValuesPerEntry * _params.SEGMENT_FLUSH_TIME;
+      } else { // For hybrid or offline table, nodictionary follows the offline side
+        noDictSize = // size of all segments in one push  with no dictionary
+            colDataSizeWithoutDictionary * numValuesPerEntry * numRecordsPerPush;
+        withDictSize = // size of all segments in one push with dictionary
+            dictionarySize * dictionaryCoefficient(cardinality, numRecordsPerPush) * DEFAUlT_NUM_PARTITIONS
+                + bitCompressedDataSize * numValuesPerEntry * numRecordsPerPush;
+      }
+
+      double storageSaved = (double) (noDictSize - withDictSize) / noDictSize;
+      LOGGER.debug("colName {}, noDictSize {}, withDictSize{}, storageSaved{}", colName, noDictSize, withDictSize,
+          storageSaved);
+
+      if (storageSaved > _params.THRESHOLD_MIN_PERCENT_DICTIONARY_STORAGE_SVAE) {
+        noDictCols.remove(colName);
+      }
+    }
+
+    // Add the no dictionary cols to config
+    _outputManager.getIndexConfig().getNoDictionaryColumns().addAll(noDictCols);
+
+    //**********On heap dictionary recommendation*******/
+    if (_inputManager.getQps() > _params.THRESHOLD_MIN_QPS_ON_HEAP) { // QPS > THRESHOLD_MIN_QPS_ON_HEAP
+      for (String colName : _inputManager.getColNameToIntMap().keySet()) {
+        if (!_outputManager.getIndexConfig().getNoDictionaryColumns().contains(colName)) //exclude no dictionary column
+        {
+          long dictionarySize = _inputManager.getDictionarySize(colName);
+          int colId = _inputManager.colNameToInt(colName);
+          double filterGroupByFreq = filterGroupByWeights[colId] / totalWeight.get();
+          if (filterGroupByFreq > _params.THRESHOLD_MIN_FILTER_FREQ_ON_HEAP  //frequently used in filter/group by
+              && dictionarySize < _params.THRESHOLD_MAX_DICTIONARY_SIZE_ON_HEAP) { // memory foot print < threshold
+            _outputManager.getIndexConfig().getOnHeapDictionaryColumns().add(colName);
+          }
+        }
+      }
+    }
+  }
+
+  private double dictionaryCoefficient(double cardinality, long numRecordsPerPush) {
+    return 1 - min(max(DEFAULT_DICT_COEFF_A * Math.log(DEFAULT_DICT_COEFF_B * cardinality / numRecordsPerPush),
+        DEFAULT_DICT_LOWER), DEFAULT_DICT_UPPER);
+  }
+
+  public void parseQuery(String queryString, double weight, double[] filterGroupByWeights, double[] selectionWeights) {

Review comment:
       General question - Would it be possible to parse the query exactly once before the execution of first rule begins? Right now, it seems like as the rules are fired in order, each rule will parse the input query set? Even though the algorithm of each rule is different, is it possible to parse once and extract all the common info needed by all the rules?




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466099704



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/BloomFilterRule.java
##########
@@ -0,0 +1,142 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.BloomFilterRuleParams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class  BloomFilterRule extends AbstractRule {

Review comment:
       Done




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466078390



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/NoDictionaryOnHeapDictionaryJointRule.java
##########
@@ -0,0 +1,245 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.NoDictionaryOnHeapDictionaryJointRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.NoDictionaryOnHeapDictionaryJointRule.*;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.REALTIME;
+
+
+public class NoDictionaryOnHeapDictionaryJointRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(NoDictionaryOnHeapDictionaryJointRule.class);
+  private final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+  private final NoDictionaryOnHeapDictionaryJointRuleParams _params;
+
+  public NoDictionaryOnHeapDictionaryJointRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    _params = inputManager.getNoDictionaryOnHeapDictionaryJointRuleParams();
+  }
+
+  @Override
+  public void run() {
+    LOGGER.info("Recommending no dictionary and on-heap dictionaries");
+
+    int numCols = _inputManager.getNumCols();
+    double[] filterGroupByWeights = new double[numCols];
+    double[] selectionWeights = new double[numCols];
+    AtomicDouble totalWeight = new AtomicDouble(0);
+
+    //**********No dictionary recommendation*******/
+    Set<String> noDictCols = new HashSet<>(_inputManager.getColNameToIntMap().keySet());
+
+    //Exclude cols with index
+    noDictCols.removeAll(_outputManager.getIndexConfig().getInvertedIndexColumns());
+    noDictCols.removeAll(_outputManager.getIndexConfig().getSortedColumn());
+    // TODO: Remove this after range index is implemented for no-dictionary
+    noDictCols.removeAll(_outputManager.getIndexConfig().getRangeIndexColumns());
+    LOGGER.debug("noDictCols {}", noDictCols);
+    //Find out columns used in filter&groupby and selection and corresponding frequencies
+    _inputManager.getQueryWeightMap().forEach((query, weight) -> {
+      parseQuery(query, weight, filterGroupByWeights, selectionWeights);
+      totalWeight.addAndGet(weight);
+    });
+
+    //Add dictionary on columns used in filter&groupby , with frequency > threshold
+    for (int i = 0; i < numCols; i++) {

Review comment:
       resolved




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465793655



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java
##########
@@ -141,6 +142,18 @@ public SuccessResponse addTable(String tableConfigStr) {
     }
   }
 
+  @PUT
+  @Produces(MediaType.APPLICATION_JSON)

Review comment:
       We should also use @Consumes annotation right?




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466024885



##########
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 {
+  public static class RuleFactory {
+    public static AbstractRule getRule(Rules rule, InputManager inputManager, ConfigManager outputManager) {
+      switch (rule) {
+        case FlagQueryRule:
+          return new FlagQueryRule(inputManager, outputManager);
+        case InvertedSortedIndexJointRule:
+          return new InvertedSortedIndexJointRule(inputManager, outputManager);
+        case KafkaPartitionRule:
+          return new KafkaPartitionRule(inputManager, outputManager);
+        case PinotTablePartitionRule:
+          return new PinotTablePartitionRule(inputManager, outputManager);
+        case BloomFilterRule:
+          return new BloomFilterRule(inputManager, outputManager);
+        case NoDictionaryOnHeapDictionaryJointRule:
+          return new NoDictionaryOnHeapDictionaryJointRule(inputManager, outputManager);
+        case VariedLengthDictionaryRule:
+          return new VariedLengthDictionaryRule(inputManager, outputManager);
+        default:
+          return null;
+      }
+    }
+  }
+  // All rules will execute by default unless explicitly specifying "recommendInvertedSortedIndexJoint" = "false"
+  boolean _recommendKafkaPartition = DEFAULT_RECOMMEND_KAFKA_PARTITION;
+  boolean _recommendPinotTablePartition = DEFAULT_RECOMMEND_PINOT_TABLE_PARTITION;
+  boolean _recommendInvertedSortedIndexJoint = DEFAULT_RECOMMEND_INVERTED_SORTED_INDEX_JOINT;
+  boolean _recommendBloomFilter = DEFAULT_RECOMMEND_BLOOM_FILTER;
+  boolean _recommendNoDictionaryOnHeapDictionaryJoint = DEFAULT_RECOMMEND_NO_DICTIONARY_ONHEAP_DICTIONARY_JOINT;
+  boolean _recommendVariedLengthDictionary = DEFAULT_RECOMMEND_VARIED_LENGTH_DICTIONARY;
+  boolean _recommendFlagQuery = DEFAULT_RECOMMEND_FLAG_QUERY;
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setRecommendVariedLengthDictionary(boolean recommendVariedLengthDictionary) {
+    _recommendVariedLengthDictionary = recommendVariedLengthDictionary;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setRecommendFlagQuery(boolean recommendFlagQuery) {
+    _recommendFlagQuery = recommendFlagQuery;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setRecommendNoDictionaryOnHeapDictionaryJoint(boolean recommendNoDictionaryOnHeapDictionaryJoint) {
+    _recommendNoDictionaryOnHeapDictionaryJoint = recommendNoDictionaryOnHeapDictionaryJoint;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setRecommendKafkaPartition(boolean recommendKafkaPartition) {
+    _recommendKafkaPartition = recommendKafkaPartition;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setRecommendInvertedSortedIndexJoint(boolean recommendInvertedSortedIndexJoint) {
+    _recommendInvertedSortedIndexJoint = recommendInvertedSortedIndexJoint;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setRecommendPinotTablePartition(boolean recommendPinotTablePartition) {
+    _recommendPinotTablePartition = recommendPinotTablePartition;
+  }
+
+  @JsonSetter(nulls = Nulls.SKIP)
+  public void setRecommendBloomFilter(boolean recommendBloomFilter) {
+    _recommendBloomFilter = recommendBloomFilter;
+  }
+
+  public boolean isRecommendVariedLengthDictionary() {
+    return _recommendVariedLengthDictionary;
+  }
+
+  public boolean isRecommendFlagQuery() {
+    return _recommendFlagQuery;
+  }
+
+  public boolean isRecommendNoDictionaryOnHeapDictionaryJoint() {
+    return _recommendNoDictionaryOnHeapDictionaryJoint;
+  }
+
+  public boolean isRecommendKafkaPartition() {
+    return _recommendKafkaPartition;
+  }
+
+  public boolean isRecommendInvertedSortedIndexJoint() {
+    return _recommendInvertedSortedIndexJoint;
+  }
+
+  public boolean isRecommendPinotTablePartition() {
+    return _recommendPinotTablePartition;
+  }
+
+  public boolean isRecommendBloomFilter() {
+    return _recommendBloomFilter;
+  }
+
+  public enum Rules {

Review comment:
       done

##########
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:
       done




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465811109



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/PartitionRuleParams.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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.io.params;
+
+import com.fasterxml.jackson.annotation.JsonSetter;
+import com.fasterxml.jackson.annotation.Nulls;
+
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.PartitionRule.*;

Review comment:
       I think our check-style will complain for using * import. We should import specifically




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466015785



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/RecommenderConstants.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.io.params;
+
+public class RecommenderConstants {
+  public static class InvertedSortedIndexJointRule {
+    public static final double DEFAULT_PERCENT_SELECT_FOR_FUNCTION = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_TEXT_MATCH = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_RANGE = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_REGEX = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_ISNULL = 0.5d;
+    public static final double DEFAULT_THRESHOLD_MIN_AND_PREDICATE_INCREMENTAL_VOTE = 0.6d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_AND_PREDICATE_TOP_CANDIDATES = 0.8d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_GAIN_DIFF_BETWEEN_ITERATION = 0.05d;
+    public static final int DEFAULT_MAX_NUM_ITERATION_WITHOUT_GAIN = 3;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_NESI_FOR_TOP_CANDIDATES = 0.7d;
+  }
+
+  public static class RulesToExecute {
+    public static final boolean DEFAULT_RECOMMEND_FLAG_QUERY = true;
+    public static final boolean DEFAULT_RECOMMEND_VARIED_LENGTH_DICTIONARY = true;
+    public static final boolean DEFAULT_RECOMMEND_KAFKA_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_PINOT_TABLE_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_INVERTED_SORTED_INDEX_JOINT = true;
+    public static final boolean DEFAULT_RECOMMEND_BLOOM_FILTER = true;
+    public static final boolean DEFAULT_RECOMMEND_NO_DICTIONARY_ONHEAP_DICTIONARY_JOINT = true;
+  }
+
+  public static class PartitionRule {
+    public static final int DEFAULT_NUM_PARTITIONS = 0;
+
+    public static final long DEFAULT_THRESHOLD_MAX_SLA_PARTITION = 1000;
+    public static final long DEFAULT_THRESHOLD_MIN_QPS_PARTITION = 200;
+    public static final long DEFAULT_OPTIMAL_SIZE_PER_SEGMENT = 2000_000_000; //2GB
+    public static final long DEFAULT_KAFKA_NUM_MESSAGES_PER_SEC_PER_PARTITION = 250;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_DIMENSION_PARTITION_TOP_CANDIDATES = 0.8d;
+    public static final int DEFAULT_THRESHOLD_MAX_IN_LENGTH = 4;
+  }
+
+  public static class BloomFilterRule {
+    public static final long DEFAULT_THRESHOLD_MAX_CARDINALITY_BLOOMFILTER = 1000_000;
+    public static final double DEFAULT_THRESHOLD_MIN_PERCENT_EQ_BLOOMFILTER = 0.5d;
+  }
+
+  public static class NoDictionaryOnHeapDictionaryJointRule {
+    public static final double DEFAULT_THRESHOLD_MIN_FILTER_FREQ_DICTIONARY = 0d;
+    public static final double DEFAULT_THRESHOLD_MAX_SELECTION_FREQ_DICTIONARY = 0.3d;
+    public static final long DEFAULT_THRESHOLD_MIN_QPS_ON_HEAP = 10_000;
+    public static final long DEFAULT_THRESHOLD_MAX_DICTIONARY_SIZE_ON_HEAP = 1000_000L;
+    public static final double DEFAULT_THRESHOLD_MIN_FILTER_FREQ_ON_HEAP = 0.3d;
+    public static final double DEFAULT_THRESHOLD_MIN_PERCENT_DICTIONARY_STORAGE_SVAE = 0.95;
+
+    public static final double DEFAULT_DICT_COEFF_A = 0.217769;
+    public static final double DEFAULT_DICT_COEFF_B = 89.0975;
+    public static final double DEFAULT_DICT_LOWER = 0;
+    public static final double DEFAULT_DICT_UPPER = 0;
+    public static final int DEFAUlT_NUM_PARTITIONS = 16;
+
+    public static final int DEFAULT_SEGMENT_FLUSH_TIME = 86400;
+  }
+
+  public static class FlagQueryRuleParams{
+    public static final long DEFAULT_THRESHOLD_MAX_LIMIT_SIZE = 100000;
+    public static final String WARNING_NO_FILTERING = "Warning: No filtering in ths query";
+    public static final String WARNING_NO_TIME_COL = "Warning: No time column used in ths query";
+    public static final String WARNING_TOO_LONG_LIMIT = "Warning: The size of LIMIT is longer than " + DEFAULT_THRESHOLD_MAX_LIMIT_SIZE;
+    public static final String ERROR_INVALID_QUERY = "Error: query not able to parse, skipped";
+  }
+
+  public static final String PQL = "pql";
+  public static final String SQL = "sql";
+  public static final String OFFLINE = "offline";
+  public static final String REALTIME = "realtime";
+  public static final String HYBRID = "hybrid";
+  public static final int NO_SUCH_COL = -1;
+  public static final double DEFAULT_CARDINALITY = 1;
+  public static final double MIN_CARDINALITY = 1;
+  public static final double DEFAULT_AVERAGE_NUM_VALUES_PER_ENTRY = 1d;
+  public static final int DEFAULT_QPS = 100;
+  public static final int DEFAULT_LATENCY_SLA = 1000;
+  public static final int DEFAULT_NUM_MSG_PER_SEC = 250;
+  public static final int DEFAULT_NUM_RECORDS_PER_PUSH = 10000;
+  public static final int DEFAULT_INT_SIZE = Integer.BYTES;
+  public static final int DEFAULT_NULL_SIZE = 0;

Review comment:
       Done




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465947624



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/NoDictionaryOnHeapDictionaryJointRule.java
##########
@@ -0,0 +1,245 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.NoDictionaryOnHeapDictionaryJointRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.NoDictionaryOnHeapDictionaryJointRule.*;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.REALTIME;
+
+
+public class NoDictionaryOnHeapDictionaryJointRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(NoDictionaryOnHeapDictionaryJointRule.class);
+  private final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+  private final NoDictionaryOnHeapDictionaryJointRuleParams _params;
+
+  public NoDictionaryOnHeapDictionaryJointRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    _params = inputManager.getNoDictionaryOnHeapDictionaryJointRuleParams();
+  }
+
+  @Override
+  public void run() {
+    LOGGER.info("Recommending no dictionary and on-heap dictionaries");
+
+    int numCols = _inputManager.getNumCols();
+    double[] filterGroupByWeights = new double[numCols];
+    double[] selectionWeights = new double[numCols];
+    AtomicDouble totalWeight = new AtomicDouble(0);
+
+    //**********No dictionary recommendation*******/
+    Set<String> noDictCols = new HashSet<>(_inputManager.getColNameToIntMap().keySet());
+
+    //Exclude cols with index
+    noDictCols.removeAll(_outputManager.getIndexConfig().getInvertedIndexColumns());
+    noDictCols.removeAll(_outputManager.getIndexConfig().getSortedColumn());
+    // TODO: Remove this after range index is implemented for no-dictionary
+    noDictCols.removeAll(_outputManager.getIndexConfig().getRangeIndexColumns());
+    LOGGER.debug("noDictCols {}", noDictCols);
+    //Find out columns used in filter&groupby and selection and corresponding frequencies
+    _inputManager.getQueryWeightMap().forEach((query, weight) -> {
+      parseQuery(query, weight, filterGroupByWeights, selectionWeights);
+      totalWeight.addAndGet(weight);
+    });
+
+    //Add dictionary on columns used in filter&groupby , with frequency > threshold
+    for (int i = 0; i < numCols; i++) {
+      double filterGroupByFreq = filterGroupByWeights[i] / totalWeight.get();
+      if (filterGroupByFreq > _params.THRESHOLD_MIN_FILTER_FREQ_DICTIONARY) {
+        noDictCols.remove(_inputManager.intToColName(i));
+      }
+    }
+
+    LOGGER.debug("filterGroupByWeights {}, selectionWeights{}, totalWeight{} ", filterGroupByWeights, selectionWeights,
+        totalWeight);
+    LOGGER.debug("noDictCols {}", noDictCols);
+
+    for (int i = 0; i < numCols; i++) {
+      // No dictionary on columns frequently used in selection

Review comment:
       It looks like we can improve the performance of this loop starting at line 94 by going over the remaining columns




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465943616



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/impl/NoDictionaryOnHeapDictionaryJointRule.java
##########
@@ -0,0 +1,245 @@
+/**
+ * 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.impl;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.core.query.request.context.ExpressionContext;
+import org.apache.pinot.core.query.request.context.FilterContext;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.predicate.Predicate;
+import org.apache.pinot.core.query.request.context.utils.BrokerRequestToQueryContextConverter;
+import org.apache.pinot.core.requesthandler.BrokerRequestOptimizer;
+import org.apache.pinot.core.requesthandler.PinotQueryParserFactory;
+import org.apache.pinot.parsers.AbstractCompiler;
+import org.apache.pinot.sql.parsers.SqlCompilationException;
+import org.apache.pinot.controller.recommender.io.ConfigManager;
+import org.apache.pinot.controller.recommender.io.InputManager;
+import org.apache.pinot.controller.recommender.rules.AbstractRule;
+import org.apache.pinot.controller.recommender.rules.io.params.NoDictionaryOnHeapDictionaryJointRuleParams;
+import org.apache.pinot.controller.recommender.rules.utils.FixedLenBitset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.NoDictionaryOnHeapDictionaryJointRule.*;
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.REALTIME;
+
+
+public class NoDictionaryOnHeapDictionaryJointRule extends AbstractRule {
+  private final Logger LOGGER = LoggerFactory.getLogger(NoDictionaryOnHeapDictionaryJointRule.class);
+  private final BrokerRequestOptimizer _brokerRequestOptimizer = new BrokerRequestOptimizer();
+  private final NoDictionaryOnHeapDictionaryJointRuleParams _params;
+
+  public NoDictionaryOnHeapDictionaryJointRule(InputManager inputManager, ConfigManager outputManager) {
+    super(inputManager, outputManager);
+    _params = inputManager.getNoDictionaryOnHeapDictionaryJointRuleParams();
+  }
+
+  @Override
+  public void run() {
+    LOGGER.info("Recommending no dictionary and on-heap dictionaries");
+
+    int numCols = _inputManager.getNumCols();
+    double[] filterGroupByWeights = new double[numCols];
+    double[] selectionWeights = new double[numCols];
+    AtomicDouble totalWeight = new AtomicDouble(0);
+
+    //**********No dictionary recommendation*******/
+    Set<String> noDictCols = new HashSet<>(_inputManager.getColNameToIntMap().keySet());
+
+    //Exclude cols with index
+    noDictCols.removeAll(_outputManager.getIndexConfig().getInvertedIndexColumns());
+    noDictCols.removeAll(_outputManager.getIndexConfig().getSortedColumn());
+    // TODO: Remove this after range index is implemented for no-dictionary
+    noDictCols.removeAll(_outputManager.getIndexConfig().getRangeIndexColumns());
+    LOGGER.debug("noDictCols {}", noDictCols);
+    //Find out columns used in filter&groupby and selection and corresponding frequencies
+    _inputManager.getQueryWeightMap().forEach((query, weight) -> {
+      parseQuery(query, weight, filterGroupByWeights, selectionWeights);
+      totalWeight.addAndGet(weight);
+    });
+
+    //Add dictionary on columns used in filter&groupby , with frequency > threshold
+    for (int i = 0; i < numCols; i++) {

Review comment:
       Add a comment explaining the rationale




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465926766



##########
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:
       The fields following this line are pre-processed from the input and used as algorithm input only, so no need to serialize/deserialize them using jackson. 




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



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r465820552



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/RecommenderConstants.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.io.params;
+
+public class RecommenderConstants {
+  public static class InvertedSortedIndexJointRule {
+    public static final double DEFAULT_PERCENT_SELECT_FOR_FUNCTION = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_TEXT_MATCH = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_RANGE = 0.5d;
+    public static final double DEAFULT_PERCENT_SELECT_FOR_REGEX = 0.5d;
+    public static final double DEFAULT_PERCENT_SELECT_FOR_ISNULL = 0.5d;
+    public static final double DEFAULT_THRESHOLD_MIN_AND_PREDICATE_INCREMENTAL_VOTE = 0.6d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_AND_PREDICATE_TOP_CANDIDATES = 0.8d;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_GAIN_DIFF_BETWEEN_ITERATION = 0.05d;
+    public static final int DEFAULT_MAX_NUM_ITERATION_WITHOUT_GAIN = 3;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_NESI_FOR_TOP_CANDIDATES = 0.7d;
+  }
+
+  public static class RulesToExecute {
+    public static final boolean DEFAULT_RECOMMEND_FLAG_QUERY = true;
+    public static final boolean DEFAULT_RECOMMEND_VARIED_LENGTH_DICTIONARY = true;
+    public static final boolean DEFAULT_RECOMMEND_KAFKA_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_PINOT_TABLE_PARTITION = true;
+    public static final boolean DEFAULT_RECOMMEND_INVERTED_SORTED_INDEX_JOINT = true;
+    public static final boolean DEFAULT_RECOMMEND_BLOOM_FILTER = true;
+    public static final boolean DEFAULT_RECOMMEND_NO_DICTIONARY_ONHEAP_DICTIONARY_JOINT = true;
+  }
+
+  public static class PartitionRule {
+    public static final int DEFAULT_NUM_PARTITIONS = 0;
+
+    public static final long DEFAULT_THRESHOLD_MAX_SLA_PARTITION = 1000;
+    public static final long DEFAULT_THRESHOLD_MIN_QPS_PARTITION = 200;
+    public static final long DEFAULT_OPTIMAL_SIZE_PER_SEGMENT = 2000_000_000; //2GB
+    public static final long DEFAULT_KAFKA_NUM_MESSAGES_PER_SEC_PER_PARTITION = 250;
+    public static final double DEFAULT_THRESHOLD_RATIO_MIN_DIMENSION_PARTITION_TOP_CANDIDATES = 0.8d;
+    public static final int DEFAULT_THRESHOLD_MAX_IN_LENGTH = 4;
+  }
+
+  public static class BloomFilterRule {
+    public static final long DEFAULT_THRESHOLD_MAX_CARDINALITY_BLOOMFILTER = 1000_000;
+    public static final double DEFAULT_THRESHOLD_MIN_PERCENT_EQ_BLOOMFILTER = 0.5d;
+  }
+
+  public static class NoDictionaryOnHeapDictionaryJointRule {
+    public static final double DEFAULT_THRESHOLD_MIN_FILTER_FREQ_DICTIONARY = 0d;
+    public static final double DEFAULT_THRESHOLD_MAX_SELECTION_FREQ_DICTIONARY = 0.3d;
+    public static final long DEFAULT_THRESHOLD_MIN_QPS_ON_HEAP = 10_000;
+    public static final long DEFAULT_THRESHOLD_MAX_DICTIONARY_SIZE_ON_HEAP = 1000_000L;
+    public static final double DEFAULT_THRESHOLD_MIN_FILTER_FREQ_ON_HEAP = 0.3d;
+    public static final double DEFAULT_THRESHOLD_MIN_PERCENT_DICTIONARY_STORAGE_SVAE = 0.95;
+
+    public static final double DEFAULT_DICT_COEFF_A = 0.217769;
+    public static final double DEFAULT_DICT_COEFF_B = 89.0975;
+    public static final double DEFAULT_DICT_LOWER = 0;
+    public static final double DEFAULT_DICT_UPPER = 0;
+    public static final int DEFAUlT_NUM_PARTITIONS = 16;
+
+    public static final int DEFAULT_SEGMENT_FLUSH_TIME = 86400;
+  }
+
+  public static class FlagQueryRuleParams{
+    public static final long DEFAULT_THRESHOLD_MAX_LIMIT_SIZE = 100000;
+    public static final String WARNING_NO_FILTERING = "Warning: No filtering in ths query";
+    public static final String WARNING_NO_TIME_COL = "Warning: No time column used in ths query";
+    public static final String WARNING_TOO_LONG_LIMIT = "Warning: The size of LIMIT is longer than " + DEFAULT_THRESHOLD_MAX_LIMIT_SIZE;
+    public static final String ERROR_INVALID_QUERY = "Error: query not able to parse, skipped";
+  }
+
+  public static final String PQL = "pql";
+  public static final String SQL = "sql";
+  public static final String OFFLINE = "offline";
+  public static final String REALTIME = "realtime";
+  public static final String HYBRID = "hybrid";
+  public static final int NO_SUCH_COL = -1;
+  public static final double DEFAULT_CARDINALITY = 1;
+  public static final double MIN_CARDINALITY = 1;
+  public static final double DEFAULT_AVERAGE_NUM_VALUES_PER_ENTRY = 1d;

Review comment:
       I don't think we should accept input if QPS, latency, num messages per second in kafka topic and number of records pushed per day are not specified. It is something user (or whoever is getting the recommendation) needs to know. So, we should not have any defaults for these 4.




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



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


[GitHub] [incubator-pinot] jasperjiaguo commented on a change in pull request #5774: Config recommendation engine

Posted by GitBox <gi...@apache.org>.
jasperjiaguo commented on a change in pull request #5774:
URL: https://github.com/apache/incubator-pinot/pull/5774#discussion_r466730931



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/rules/io/params/PartitionRuleParams.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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.io.params;
+
+import com.fasterxml.jackson.annotation.JsonSetter;
+import com.fasterxml.jackson.annotation.Nulls;
+
+import static org.apache.pinot.controller.recommender.rules.io.params.RecommenderConstants.PartitionRule.*;

Review comment:
       `mvn checkstyle:check` success so we should be fine




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



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