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 2021/06/08 17:23:39 UTC

[GitHub] [incubator-pinot] GSharayu opened a new pull request #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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


   Typically the dimension columns are used in filter and group by. However, technically nothing prevents user from doing a filter on a non-dimension column. So, we need to update the query inverted index and sorted index recommender (QueryInvertedSortedIndexRecommender) and remove this limitation. This will allow rule engine to recommend inverted or sorted index on non-dimension columns as well
   This issue is followup for issue #6937


-- 
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] GSharayu commented on a change in pull request #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/io/InputManager.java
##########
@@ -237,11 +237,16 @@ private void reorderDimsAndBuildMap()
     _colNameToIntMap = new HashMap<>();
 
     _dimNamesInvertedSortedIndexApplicable = new HashSet<>(_dimNames);
+    _dimNamesInvertedSortedIndexApplicable.addAll(_metricNames);
+    _dimNamesInvertedSortedIndexApplicable.addAll(_dateTimeNames);
+
     _dimNamesInvertedSortedIndexApplicable.remove(sortedColumn);
     _dimNamesInvertedSortedIndexApplicable.removeAll(invertedIndexColumns);
     _dimNamesInvertedSortedIndexApplicable.removeAll(noDictionaryColumns);
 
     HashSet<String> dimNamesInveredSortedIndexNotApplicable = new HashSet<>(_dimNames);
+    dimNamesInveredSortedIndexNotApplicable.addAll(_metricNames);

Review comment:
       right, we can write it in clean way

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/io/InputManager.java
##########
@@ -259,14 +264,6 @@ private void reorderDimsAndBuildMap()
     LOGGER.debug("_dimNames{}", _dimNames);
     LOGGER.debug("_metricNames{}", _metricNames);
     LOGGER.debug("_dateTimeNames{}", _dateTimeNames);
-    _metricNames.forEach(name -> {

Review comment:
       yes




-- 
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] GSharayu commented on a change in pull request #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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



##########
File path: pinot-controller/src/test/java/org/apache/pinot/controller/recommender/TestConfigEngine.java
##########
@@ -125,6 +125,18 @@ void testInvertedSortedIndexJointRule()
     assertEquals(output.getIndexConfig().getSortedColumn(), "c");
   }
 
+  @Test

Review comment:
       covered in same test case




-- 
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] GSharayu commented on a change in pull request #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/io/InputManager.java
##########
@@ -237,11 +237,16 @@ private void reorderDimsAndBuildMap()
     _colNameToIntMap = new HashMap<>();
 
     _dimNamesInvertedSortedIndexApplicable = new HashSet<>(_dimNames);
+    _dimNamesInvertedSortedIndexApplicable.addAll(_metricNames);
+    _dimNamesInvertedSortedIndexApplicable.addAll(_dateTimeNames);
+
     _dimNamesInvertedSortedIndexApplicable.remove(sortedColumn);
     _dimNamesInvertedSortedIndexApplicable.removeAll(invertedIndexColumns);
     _dimNamesInvertedSortedIndexApplicable.removeAll(noDictionaryColumns);
 
     HashSet<String> dimNamesInveredSortedIndexNotApplicable = new HashSet<>(_dimNames);
+    dimNamesInveredSortedIndexNotApplicable.addAll(_metricNames);

Review comment:
       updated!




-- 
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 #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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


   


-- 
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 #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/io/InputManager.java
##########
@@ -237,11 +237,16 @@ private void reorderDimsAndBuildMap()
     _colNameToIntMap = new HashMap<>();
 
     _dimNamesInvertedSortedIndexApplicable = new HashSet<>(_dimNames);
+    _dimNamesInvertedSortedIndexApplicable.addAll(_metricNames);

Review comment:
       (nit) add a comment highlighting that all types of columns will be considered by the 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] GSharayu commented on a change in pull request #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/io/InputManager.java
##########
@@ -237,11 +237,16 @@ private void reorderDimsAndBuildMap()
     _colNameToIntMap = new HashMap<>();
 
     _dimNamesInvertedSortedIndexApplicable = new HashSet<>(_dimNames);
+    _dimNamesInvertedSortedIndexApplicable.addAll(_metricNames);

Review comment:
       refactored at other places as well!
   




-- 
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 #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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



##########
File path: pinot-controller/src/test/resources/recommenderInput/SortedInvertedIndexInputWithDateTimeColumn.json
##########
@@ -0,0 +1,87 @@
+{
+  "schema":{
+    "schemaName": "tableSchema",
+    "dimensionFieldSpecs": [
+      {
+        "name": "b",
+        "dataType": "DOUBLE",
+        "cardinality":6,
+        "singleValueField": false,
+        "numValuesPerEntry":1.5
+      },
+      {
+        "name": "c",
+        "dataType": "FLOAT",
+        "cardinality":7,
+        "numValuesPerEntry":1
+      },
+      {
+        "name": "d",
+        "dataType": "STRING",
+        "cardinality":41,
+        "singleValueField": false,
+        "numValuesPerEntry":2,
+        "averageLength" : 27
+      }
+    ],
+    "metricFieldSpecs": [
+      {
+        "name": "p",
+        "dataType": "DOUBLE",
+        "cardinality":10000,
+        "numValuesPerEntry":1
+      }
+    ],
+    "dateTimeFieldSpecs": [
+      {
+        "name": "x",
+        "dataType": "INT",
+        "format": "1:DAYS:EPOCH",
+        "granularity": "1:DAYS",
+        "cardinality": 3
+      }
+    ],
+    "timeFieldSpec": {
+      "incomingGranularitySpec": {
+        "dataType": "INT",
+        "name": "t",
+        "timeType": "DAYS",
+        "cardinality":10000,
+        "numValuesPerEntry":1
+      }
+    }
+  },
+  "queriesWithWeights":{
+    "select i from tableName where b in (2,4) and (c = 7) and t in ('#VALUES', 500)": 1,
+    "select j from tableName where (x=3)": 2,
+    "select f from tableName where (b=1) and t<3": 4,
+    "select f from tableName where (x=0 and b=1) or c=7 or (t = 7)": 2,
+    "select f from tableName where t between 1 and 1000": 2
+  },
+  "qps": 15000,
+  "numMessagesPerSecInKafkaTopic":1000,
+  "numRecordsPerPush":1000000000,
+  "tableType": "HYBRID",
+  "latencySLA": 500,
+
+  "rulesToExecute": {
+    "recommendInvertedSortedIndexJoint": true
+  },
+  "partitionRuleParams": {

Review comment:
       Remove all the params and just keep invertedSortedIndexJointRuleParams since that is what is used by test case

##########
File path: pinot-controller/src/test/resources/recommenderInput/SortedInvertedIndexInputWithDateTimeColumn.json
##########
@@ -0,0 +1,87 @@
+{
+  "schema":{
+    "schemaName": "tableSchema",
+    "dimensionFieldSpecs": [
+      {
+        "name": "b",
+        "dataType": "DOUBLE",
+        "cardinality":6,
+        "singleValueField": false,
+        "numValuesPerEntry":1.5
+      },
+      {
+        "name": "c",
+        "dataType": "FLOAT",
+        "cardinality":7,
+        "numValuesPerEntry":1
+      },
+      {
+        "name": "d",
+        "dataType": "STRING",
+        "cardinality":41,
+        "singleValueField": false,
+        "numValuesPerEntry":2,
+        "averageLength" : 27
+      }
+    ],
+    "metricFieldSpecs": [
+      {
+        "name": "p",
+        "dataType": "DOUBLE",
+        "cardinality":10000,
+        "numValuesPerEntry":1
+      }
+    ],
+    "dateTimeFieldSpecs": [
+      {
+        "name": "x",
+        "dataType": "INT",
+        "format": "1:DAYS:EPOCH",
+        "granularity": "1:DAYS",
+        "cardinality": 3
+      }
+    ],
+    "timeFieldSpec": {
+      "incomingGranularitySpec": {
+        "dataType": "INT",
+        "name": "t",
+        "timeType": "DAYS",
+        "cardinality":10000,
+        "numValuesPerEntry":1
+      }
+    }
+  },
+  "queriesWithWeights":{
+    "select i from tableName where b in (2,4) and (c = 7) and t in ('#VALUES', 500)": 1,
+    "select j from tableName where (x=3)": 2,
+    "select f from tableName where (b=1) and t<3": 4,
+    "select f from tableName where (x=0 and b=1) or c=7 or (t = 7)": 2,
+    "select f from tableName where t between 1 and 1000": 2
+  },
+  "qps": 15000,
+  "numMessagesPerSecInKafkaTopic":1000,
+  "numRecordsPerPush":1000000000,
+  "tableType": "HYBRID",
+  "latencySLA": 500,
+
+  "rulesToExecute": {
+    "recommendInvertedSortedIndexJoint": true
+  },
+  "partitionRuleParams": {

Review comment:
       Remove all the params and just keep `invertedSortedIndexJointRuleParams` since that is what is used by test case




-- 
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 #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/io/InputManager.java
##########
@@ -259,14 +264,6 @@ private void reorderDimsAndBuildMap()
     LOGGER.debug("_dimNames{}", _dimNames);
     LOGGER.debug("_metricNames{}", _metricNames);
     LOGGER.debug("_dateTimeNames{}", _dateTimeNames);
-    _metricNames.forEach(name -> {

Review comment:
       Removing this because the applicable list has all 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 #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/io/InputManager.java
##########
@@ -237,11 +237,16 @@ private void reorderDimsAndBuildMap()
     _colNameToIntMap = new HashMap<>();
 
     _dimNamesInvertedSortedIndexApplicable = new HashSet<>(_dimNames);
+    _dimNamesInvertedSortedIndexApplicable.addAll(_metricNames);

Review comment:
       (nit) add a comment highlighting that all types of comments will be considered by the 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] siddharthteotia commented on a change in pull request #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/io/InputManager.java
##########
@@ -236,39 +236,28 @@ private void reorderDimsAndBuildMap()
     _intToColNameMap = new String[_dimNames.size() + _metricNames.size() + _dateTimeNames.size()];
     _colNameToIntMap = new HashMap<>();
 
-    _dimNamesInvertedSortedIndexApplicable = new HashSet<>(_dimNames);
-    _dimNamesInvertedSortedIndexApplicable.remove(sortedColumn);
-    _dimNamesInvertedSortedIndexApplicable.removeAll(invertedIndexColumns);
-    _dimNamesInvertedSortedIndexApplicable.removeAll(noDictionaryColumns);
+    // Inverted index and sorted index will be recommended on all types of columns : dimensions, metrics and date time
+    _columnNamesInvertedSortedIndexApplicable = new HashSet<>(_dimNames);
+    _columnNamesInvertedSortedIndexApplicable.addAll(_metricNames);
+    _columnNamesInvertedSortedIndexApplicable.addAll(_dateTimeNames);
 
-    HashSet<String> dimNamesInveredSortedIndexNotApplicable = new HashSet<>(_dimNames);
-    dimNamesInveredSortedIndexNotApplicable.removeAll(_dimNamesInvertedSortedIndexApplicable);
-
-    LOGGER.debug("_dimNamesInveredSortedIndexApplicable {}", _dimNamesInvertedSortedIndexApplicable);
     AtomicInteger counter = new AtomicInteger(0);
-    _dimNamesInvertedSortedIndexApplicable.forEach(name -> {
+    _columnNamesInvertedSortedIndexApplicable.forEach(name -> {
       _intToColNameMap[counter.get()] = name;
       _colNameToIntMap.put(name, counter.getAndIncrement());
     });
 
-    dimNamesInveredSortedIndexNotApplicable.forEach(name -> {
-      _intToColNameMap[counter.get()] = name;
-      _colNameToIntMap.put(name, counter.getAndIncrement());
-    });
+    _columnNamesInvertedSortedIndexApplicable.remove(sortedColumn);
+    _columnNamesInvertedSortedIndexApplicable.removeAll(invertedIndexColumns);
+    _columnNamesInvertedSortedIndexApplicable.removeAll(noDictionaryColumns);
+
+    LOGGER.debug("_columnNamesInvertedSortedIndexApplicable {}", _columnNamesInvertedSortedIndexApplicable);
 
     LOGGER.debug("_dimNames{}", _dimNames);
     LOGGER.debug("_metricNames{}", _metricNames);
     LOGGER.debug("_dateTimeNames{}", _dateTimeNames);
-    _metricNames.forEach(name -> {
-      _intToColNameMap[counter.get()] = name;
-      _colNameToIntMap.put(name, counter.getAndIncrement());
-    });
-    _dateTimeNames.forEach(name -> {
-      _intToColNameMap[counter.get()] = name;
-      _colNameToIntMap.put(name, counter.getAndIncrement());
-    });
 
-    LOGGER.info("*Num dims we can apply index on: {}", getNumDimsInvertedSortedApplicable());
+    LOGGER.info("*Num dims we can apply index on: {}", getNumColumnsInvertedSortedApplicable());

Review comment:
       Should "Num cols we  ...."
   
   On that note, we should do a PR to fix logging. Too much of info level logs here. We should look at what can be info and what can be moved to debug. 




-- 
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 #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/io/InputManager.java
##########
@@ -539,7 +528,7 @@ public int getNumKafkaPartitions() {
   }
 
   public boolean isIndexableDim(String colName) {

Review comment:
       Delete this method. It is not being used anywhere. If we need, we will add it back




-- 
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] GSharayu commented on a change in pull request #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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



##########
File path: pinot-controller/src/test/java/org/apache/pinot/controller/recommender/TestConfigEngine.java
##########
@@ -144,9 +156,9 @@ void testQueryInvertedSortedIndexRecommender()
             .build();
 
     Set<String> results = new HashSet<String>() {{
-      add("[[PredicateParseResult{dims{[1]}, AND, BITMAP, nESI=1.568, selected=0.068, nESIWithIdx=0.618}, PredicateParseResult{dims{[0]}, AND, BITMAP, nESI=1.568, selected=0.068, nESIWithIdx=0.767}, PredicateParseResult{dims{[]}, AND, NESTED, nESI=1.568, selected=0.068, nESIWithIdx=1.568}]]");
-      add("[[PredicateParseResult{dims{[5]}, AND, BITMAP, nESI=0.150, selected=0.015, nESIWithIdx=0.058}, PredicateParseResult{dims{[]}, AND, NESTED, nESI=0.150, selected=0.015, nESIWithIdx=0.150}], [PredicateParseResult{dims{[3, 7]}, AND, BITMAP, nESI=12.000, selected=0.500, nESIWithIdx=4.000}, PredicateParseResult{dims{[]}, AND, NESTED, nESI=12.000, selected=0.500, nESIWithIdx=12.000}]]");
-      add("[[PredicateParseResult{dims{[0, 2]}, AND, BITMAP, nESI=7.250, selected=0.047, nESIWithIdx=1.122}, PredicateParseResult{dims{[]}, AND, NESTED, nESI=7.250, selected=0.047, nESIWithIdx=7.250}]]");
+      add("[[PredicateParseResult{dims{[3]}, AND, BITMAP, nESI=1.645, selected=0.034, nESIWithIdx=0.695}, PredicateParseResult{dims{[2]}, AND, BITMAP, nESI=1.645, selected=0.034, nESIWithIdx=0.835}, PredicateParseResult{dims{[]}, AND, NESTED, nESI=1.645, selected=0.034, nESIWithIdx=1.645}]]");

Review comment:
       These values will be updated because of changes to how Atomic number to column map generation is refactored in InputManager




-- 
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] GSharayu commented on a change in pull request #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/io/InputManager.java
##########
@@ -237,11 +237,16 @@ private void reorderDimsAndBuildMap()
     _colNameToIntMap = new HashMap<>();
 
     _dimNamesInvertedSortedIndexApplicable = new HashSet<>(_dimNames);
+    _dimNamesInvertedSortedIndexApplicable.addAll(_metricNames);
+    _dimNamesInvertedSortedIndexApplicable.addAll(_dateTimeNames);
+
     _dimNamesInvertedSortedIndexApplicable.remove(sortedColumn);
     _dimNamesInvertedSortedIndexApplicable.removeAll(invertedIndexColumns);
     _dimNamesInvertedSortedIndexApplicable.removeAll(noDictionaryColumns);
 
     HashSet<String> dimNamesInveredSortedIndexNotApplicable = new HashSet<>(_dimNames);
+    dimNamesInveredSortedIndexNotApplicable.addAll(_metricNames);

Review comment:
       updated! This change will also update the Atomic number value in test case for QueryInvertedSortedIndexRecommender. 




-- 
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 #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/7031?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#7031](https://codecov.io/gh/apache/incubator-pinot/pull/7031?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6c45c77) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/a1c9b631381a25ddd6d3164d6a9ce337c3939b9f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (a1c9b63) will **decrease** coverage by `31.26%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/7031/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/7031?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #7031       +/-   ##
   =============================================
   - Coverage     73.38%   42.12%   -31.27%     
   + Complexity       12        7        -5     
   =============================================
     Files          1453     1453               
     Lines         72032    72025        -7     
     Branches      10427    10426        -1     
   =============================================
   - Hits          52863    30341    -22522     
   - Misses        15643    39084    +23441     
   + Partials       3526     2600      -926     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `42.12% <0.00%> (-0.01%)` | :arrow_down: |
   | unittests | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/7031?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../pinot/controller/recommender/io/InputManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/7031/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9yZWNvbW1lbmRlci9pby9JbnB1dE1hbmFnZXIuamF2YQ==) | `0.00% <0.00%> (-91.87%)` | :arrow_down: |
   | [...les/utils/QueryInvertedSortedIndexRecommender.java](https://codecov.io/gh/apache/incubator-pinot/pull/7031/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9yZWNvbW1lbmRlci9ydWxlcy91dGlscy9RdWVyeUludmVydGVkU29ydGVkSW5kZXhSZWNvbW1lbmRlci5qYXZh) | `0.00% <ø> (-81.69%)` | :arrow_down: |
   | [...c/main/java/org/apache/pinot/common/tier/Tier.java](https://codecov.io/gh/apache/incubator-pinot/pull/7031/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdGllci9UaWVyLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/data/MetricFieldSpec.java](https://codecov.io/gh/apache/incubator-pinot/pull/7031/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9NZXRyaWNGaWVsZFNwZWMuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...a/org/apache/pinot/spi/config/table/TableType.java](https://codecov.io/gh/apache/incubator-pinot/pull/7031/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL1RhYmxlVHlwZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...a/org/apache/pinot/spi/data/DateTimeFieldSpec.java](https://codecov.io/gh/apache/incubator-pinot/pull/7031/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9EYXRlVGltZUZpZWxkU3BlYy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/pinot/spi/data/DimensionFieldSpec.java](https://codecov.io/gh/apache/incubator-pinot/pull/7031/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9EaW1lbnNpb25GaWVsZFNwZWMuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/pinot/spi/data/readers/FileFormat.java](https://codecov.io/gh/apache/incubator-pinot/pull/7031/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9yZWFkZXJzL0ZpbGVGb3JtYXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/common/minion/Granularity.java](https://codecov.io/gh/apache/incubator-pinot/pull/7031/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWluaW9uL0dyYW51bGFyaXR5LmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...org/apache/pinot/spi/config/table/QuotaConfig.java](https://codecov.io/gh/apache/incubator-pinot/pull/7031/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL1F1b3RhQ29uZmlnLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [913 more](https://codecov.io/gh/apache/incubator-pinot/pull/7031/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/7031?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/7031?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [e91f426...6c45c77](https://codecov.io/gh/apache/incubator-pinot/pull/7031?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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 #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/io/InputManager.java
##########
@@ -237,11 +237,16 @@ private void reorderDimsAndBuildMap()
     _colNameToIntMap = new HashMap<>();
 
     _dimNamesInvertedSortedIndexApplicable = new HashSet<>(_dimNames);
+    _dimNamesInvertedSortedIndexApplicable.addAll(_metricNames);
+    _dimNamesInvertedSortedIndexApplicable.addAll(_dateTimeNames);
+
     _dimNamesInvertedSortedIndexApplicable.remove(sortedColumn);
     _dimNamesInvertedSortedIndexApplicable.removeAll(invertedIndexColumns);
     _dimNamesInvertedSortedIndexApplicable.removeAll(noDictionaryColumns);
 
     HashSet<String> dimNamesInveredSortedIndexNotApplicable = new HashSet<>(_dimNames);
+    dimNamesInveredSortedIndexNotApplicable.addAll(_metricNames);

Review comment:
       Why do we need `dimNamesInveredSortedIndexNotApplicable` ? I feel we can remove this and do some cleanup




-- 
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 #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/io/InputManager.java
##########
@@ -237,11 +237,16 @@ private void reorderDimsAndBuildMap()
     _colNameToIntMap = new HashMap<>();
 
     _dimNamesInvertedSortedIndexApplicable = new HashSet<>(_dimNames);
+    _dimNamesInvertedSortedIndexApplicable.addAll(_metricNames);

Review comment:
       Also suggest renaming this to `_columnNamesInvertedSortedIndexApplicable` since this PR removes the restriction that it has to be a dimension




-- 
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 #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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



##########
File path: pinot-controller/src/test/java/org/apache/pinot/controller/recommender/TestConfigEngine.java
##########
@@ -125,6 +125,18 @@ void testInvertedSortedIndexJointRule()
     assertEquals(output.getIndexConfig().getSortedColumn(), "c");
   }
 
+  @Test

Review comment:
       There is an existing test `testInvertedSortedIndexJointRule()`
   Between that and this newly added test, can we ensure that recommendation is covered for dimensions, metrics, timeSpec (deprecated) and dateTimes. I think metric is not covered. Rest seems to be covered ?




-- 
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] GSharayu commented on a change in pull request #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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



##########
File path: pinot-controller/src/test/resources/recommenderInput/SortedInvertedIndexInputWithDateTimeColumn.json
##########
@@ -0,0 +1,87 @@
+{
+  "schema":{
+    "schemaName": "tableSchema",
+    "dimensionFieldSpecs": [
+      {
+        "name": "b",
+        "dataType": "DOUBLE",
+        "cardinality":6,
+        "singleValueField": false,
+        "numValuesPerEntry":1.5
+      },
+      {
+        "name": "c",
+        "dataType": "FLOAT",
+        "cardinality":7,
+        "numValuesPerEntry":1
+      },
+      {
+        "name": "d",
+        "dataType": "STRING",
+        "cardinality":41,
+        "singleValueField": false,
+        "numValuesPerEntry":2,
+        "averageLength" : 27
+      }
+    ],
+    "metricFieldSpecs": [
+      {
+        "name": "p",
+        "dataType": "DOUBLE",
+        "cardinality":10000,
+        "numValuesPerEntry":1
+      }
+    ],
+    "dateTimeFieldSpecs": [
+      {
+        "name": "x",
+        "dataType": "INT",
+        "format": "1:DAYS:EPOCH",
+        "granularity": "1:DAYS",
+        "cardinality": 3
+      }
+    ],
+    "timeFieldSpec": {
+      "incomingGranularitySpec": {
+        "dataType": "INT",
+        "name": "t",
+        "timeType": "DAYS",
+        "cardinality":10000,
+        "numValuesPerEntry":1
+      }
+    }
+  },
+  "queriesWithWeights":{
+    "select i from tableName where b in (2,4) and (c = 7) and t in ('#VALUES', 500)": 1,
+    "select j from tableName where (x=3)": 2,
+    "select f from tableName where (b=1) and t<3": 4,
+    "select f from tableName where (x=0 and b=1) or c=7 or (t = 7)": 2,
+    "select f from tableName where t between 1 and 1000": 2
+  },
+  "qps": 15000,
+  "numMessagesPerSecInKafkaTopic":1000,
+  "numRecordsPerPush":1000000000,
+  "tableType": "HYBRID",
+  "latencySLA": 500,
+
+  "rulesToExecute": {
+    "recommendInvertedSortedIndexJoint": true
+  },
+  "partitionRuleParams": {

Review comment:
       updated!




-- 
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] GSharayu commented on a change in pull request #7031: Changes in QueryInvertedSortedIndexRecommender to handle non-dimension columns for sorted and inverted index(#7018)

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



##########
File path: pinot-controller/src/test/java/org/apache/pinot/controller/recommender/TestConfigEngine.java
##########
@@ -125,6 +125,18 @@ void testInvertedSortedIndexJointRule()
     assertEquals(output.getIndexConfig().getSortedColumn(), "c");
   }
 
+  @Test

Review comment:
       right, will update
   




-- 
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