You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "shenyu0127 (via GitHub)" <gi...@apache.org> on 2023/07/11 17:22:47 UTC

[GitHub] [pinot] shenyu0127 opened a new pull request, #11078: Filtering NULL support.

shenyu0127 opened a new pull request, #11078:
URL: https://github.com/apache/pinot/pull/11078

   Enable filtering NULL support by creating an `AndFilterOperator` consists of the leaf filter operator and a filter operator constructed from a non-NULL bitmap.
   
   This technique applies when
   - The filter is not based on a transform function.
   - The predicate type is not `IS NULL` or `IS NOT NULL`.
   
   This PR disables the Filtering NULL support introduced in https://github.com/apache/pinot/pull/9173 by passing `nullHandlingEnabled = false` to the `ScanBasedFilterOperator` constructor. Will revert the no longer needed changes in https://github.com/apache/pinot/pull/9173 in a follow-up PR.
   
   


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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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] [pinot] Jackie-Jiang commented on a diff in pull request #11078: Column filter operators NULL support.

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11078:
URL: https://github.com/apache/pinot/pull/11078#discussion_r1275644261


##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/BaseColumnFilterOperator.java:
##########
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.operator.filter;
+
+import java.util.Arrays;
+import org.apache.pinot.core.common.BlockDocIdSet;
+import org.apache.pinot.core.operator.docidsets.AndDocIdSet;
+import org.apache.pinot.core.operator.docidsets.BitmapDocIdSet;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.segment.spi.datasource.DataSource;
+import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader;
+import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
+
+
+public abstract class BaseColumnFilterOperator extends BaseFilterOperator {
+  protected final QueryContext _queryContext;
+  protected final DataSource _dataSource;
+
+  protected BaseColumnFilterOperator(QueryContext queryContext, DataSource dataSource, int numDocs) {
+    super(numDocs, queryContext.isNullHandlingEnabled());
+    _queryContext = queryContext;
+    _dataSource = dataSource;
+  }
+
+  protected abstract BlockDocIdSet getNextBlockWithoutNullHandling();
+
+  @Override
+  protected BlockDocIdSet getTrues() {
+    if (_nullHandlingEnabled) {
+      ImmutableRoaringBitmap nullBitmap = getNullBitmap();
+      if (nullBitmap != null && !nullBitmap.isEmpty()) {
+        return excludeNulls(getNextBlockWithoutNullHandling(), nullBitmap);
+      }
+    }
+    return getNextBlockWithoutNullHandling();
+  }
+
+
+  @Override
+  protected BlockDocIdSet getNulls() {
+    return new BitmapDocIdSet(getNullBitmap(), _numDocs);

Review Comment:
   Is it possible that `getNullBitmap()` returns null?



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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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] [pinot] Jackie-Jiang commented on a diff in pull request #11078: Filtering NULL support.

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11078:
URL: https://github.com/apache/pinot/pull/11078#discussion_r1270418833


##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/NullHandlingSupportedFilterOperator.java:
##########
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.operator.filter;
+
+import java.util.Arrays;
+import org.apache.pinot.core.common.BlockDocIdSet;
+import org.apache.pinot.core.operator.blocks.FilterBlock;
+import org.apache.pinot.core.operator.docidsets.AndDocIdSet;
+import org.apache.pinot.core.operator.docidsets.BitmapDocIdSet;
+import org.apache.pinot.core.operator.docidsets.MatchAllDocIdSet;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.segment.spi.datasource.DataSource;
+import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader;
+
+
+public abstract class NullHandlingSupportedFilterOperator extends BaseFilterOperator {
+  protected final QueryContext _queryContext;
+  protected final DataSource _dataSource;
+  protected final int _numDocs;
+
+  protected NullHandlingSupportedFilterOperator(QueryContext queryContext, DataSource dataSource, int numDocs) {
+    _queryContext = queryContext;
+    _dataSource = dataSource;
+    _numDocs = numDocs;
+  }
+
+  @Override
+  protected FilterBlock getNextBlock() {
+    if (_queryContext.isNullHandlingEnabled()) {
+      return new FilterBlock(excludeNulls(getNextBlockWithoutNullHandling()));
+    } else {
+      return new FilterBlock(getNextBlockWithoutNullHandling());
+    }
+  }
+
+  protected abstract BlockDocIdSet getNextBlockWithoutNullHandling();
+
+  private BlockDocIdSet excludeNulls(BlockDocIdSet blockDocIdSet) {
+    return new AndDocIdSet(

Review Comment:
   Do not create `AndDocIdSet` when there is no null value



##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/NullHandlingSupportedFilterOperator.java:
##########
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.operator.filter;
+
+import java.util.Arrays;
+import org.apache.pinot.core.common.BlockDocIdSet;
+import org.apache.pinot.core.operator.blocks.FilterBlock;
+import org.apache.pinot.core.operator.docidsets.AndDocIdSet;
+import org.apache.pinot.core.operator.docidsets.BitmapDocIdSet;
+import org.apache.pinot.core.operator.docidsets.MatchAllDocIdSet;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.segment.spi.datasource.DataSource;
+import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader;
+
+
+public abstract class NullHandlingSupportedFilterOperator extends BaseFilterOperator {

Review Comment:
   Do we need this extra layer of abstract class? In the end state all filter should have null handling and we can just put the new methods into `BaseFilterOperator`



##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/NullHandlingSupportedFilterOperator.java:
##########
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.operator.filter;
+
+import java.util.Arrays;
+import org.apache.pinot.core.common.BlockDocIdSet;
+import org.apache.pinot.core.operator.blocks.FilterBlock;
+import org.apache.pinot.core.operator.docidsets.AndDocIdSet;
+import org.apache.pinot.core.operator.docidsets.BitmapDocIdSet;
+import org.apache.pinot.core.operator.docidsets.MatchAllDocIdSet;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.segment.spi.datasource.DataSource;
+import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader;
+
+
+public abstract class NullHandlingSupportedFilterOperator extends BaseFilterOperator {
+  protected final QueryContext _queryContext;
+  protected final DataSource _dataSource;
+  protected final int _numDocs;
+
+  protected NullHandlingSupportedFilterOperator(QueryContext queryContext, DataSource dataSource, int numDocs) {
+    _queryContext = queryContext;
+    _dataSource = dataSource;
+    _numDocs = numDocs;
+  }
+
+  @Override
+  protected FilterBlock getNextBlock() {
+    if (_queryContext.isNullHandlingEnabled()) {
+      return new FilterBlock(excludeNulls(getNextBlockWithoutNullHandling()));
+    } else {
+      return new FilterBlock(getNextBlockWithoutNullHandling());
+    }
+  }
+
+  protected abstract BlockDocIdSet getNextBlockWithoutNullHandling();
+
+  private BlockDocIdSet excludeNulls(BlockDocIdSet blockDocIdSet) {
+    return new AndDocIdSet(
+        Arrays.asList(blockDocIdSet, fromNonNullBitmap()),
+        _queryContext.getQueryOptions());

Review Comment:
   (format) Format seems incorrect



##########
pinot-core/src/main/java/org/apache/pinot/core/operator/docidsets/BitmapDocIdSet.java:
##########
@@ -27,6 +27,13 @@ public class BitmapDocIdSet implements BlockDocIdSet {
   private final BitmapDocIdIterator _iterator;
 
   public BitmapDocIdSet(ImmutableRoaringBitmap docIds, int numDocs) {
+    this(docIds, numDocs, false);
+  }
+
+  public BitmapDocIdSet(ImmutableRoaringBitmap docIds, int numDocs, boolean exclusive) {

Review Comment:
   I'd suggest flipping the bitmap on the caller side before creating a `BitmapDocIdSet` which IMO is more clear



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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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] [pinot] shenyu0127 commented on a diff in pull request #11078: Filtering NULL support.

Posted by "shenyu0127 (via GitHub)" <gi...@apache.org>.
shenyu0127 commented on code in PR #11078:
URL: https://github.com/apache/pinot/pull/11078#discussion_r1270846159


##########
pinot-core/src/main/java/org/apache/pinot/core/operator/docidsets/BitmapDocIdSet.java:
##########
@@ -27,6 +27,13 @@ public class BitmapDocIdSet implements BlockDocIdSet {
   private final BitmapDocIdIterator _iterator;
 
   public BitmapDocIdSet(ImmutableRoaringBitmap docIds, int numDocs) {
+    this(docIds, numDocs, false);
+  }
+
+  public BitmapDocIdSet(ImmutableRoaringBitmap docIds, int numDocs, boolean exclusive) {

Review Comment:
   Good suggestion. Done.



##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/NullHandlingSupportedFilterOperator.java:
##########
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.operator.filter;
+
+import java.util.Arrays;
+import org.apache.pinot.core.common.BlockDocIdSet;
+import org.apache.pinot.core.operator.blocks.FilterBlock;
+import org.apache.pinot.core.operator.docidsets.AndDocIdSet;
+import org.apache.pinot.core.operator.docidsets.BitmapDocIdSet;
+import org.apache.pinot.core.operator.docidsets.MatchAllDocIdSet;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.segment.spi.datasource.DataSource;
+import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader;
+
+
+public abstract class NullHandlingSupportedFilterOperator extends BaseFilterOperator {
+  protected final QueryContext _queryContext;
+  protected final DataSource _dataSource;
+  protected final int _numDocs;
+
+  protected NullHandlingSupportedFilterOperator(QueryContext queryContext, DataSource dataSource, int numDocs) {
+    _queryContext = queryContext;
+    _dataSource = dataSource;
+    _numDocs = numDocs;
+  }
+
+  @Override
+  protected FilterBlock getNextBlock() {
+    if (_queryContext.isNullHandlingEnabled()) {
+      return new FilterBlock(excludeNulls(getNextBlockWithoutNullHandling()));
+    } else {
+      return new FilterBlock(getNextBlockWithoutNullHandling());
+    }
+  }
+
+  protected abstract BlockDocIdSet getNextBlockWithoutNullHandling();
+
+  private BlockDocIdSet excludeNulls(BlockDocIdSet blockDocIdSet) {
+    return new AndDocIdSet(
+        Arrays.asList(blockDocIdSet, fromNonNullBitmap()),
+        _queryContext.getQueryOptions());

Review Comment:
   Good catch. This comment no longer applies because I changed this code block.



##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/NullHandlingSupportedFilterOperator.java:
##########
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.operator.filter;
+
+import java.util.Arrays;
+import org.apache.pinot.core.common.BlockDocIdSet;
+import org.apache.pinot.core.operator.blocks.FilterBlock;
+import org.apache.pinot.core.operator.docidsets.AndDocIdSet;
+import org.apache.pinot.core.operator.docidsets.BitmapDocIdSet;
+import org.apache.pinot.core.operator.docidsets.MatchAllDocIdSet;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.segment.spi.datasource.DataSource;
+import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader;
+
+
+public abstract class NullHandlingSupportedFilterOperator extends BaseFilterOperator {

Review Comment:
   We can't put this logic inside `BaseFilterOperator` because some subclasses do not share the same logic. Here are some examples:
   
   - Non-leaf filter operators, e.g. `AndFilterOperator`
   - `TextMatchFilterOperator`
   - `ExpressionFilterOperator` (the `nullBitmap` is from `TransformFunction::getNullBitmap` so we don't need `_dataSource`)
   
   I renamed the class to `NullHandlingSupportedFilterOperator` to indicate it's the base class of a subset of filter operators.
   
   We might do some tweaks to make some leaf filter operators a subclass of this class, but I'd defer that to not increase the scope of this PR.



##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/NullHandlingSupportedFilterOperator.java:
##########
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.operator.filter;
+
+import java.util.Arrays;
+import org.apache.pinot.core.common.BlockDocIdSet;
+import org.apache.pinot.core.operator.blocks.FilterBlock;
+import org.apache.pinot.core.operator.docidsets.AndDocIdSet;
+import org.apache.pinot.core.operator.docidsets.BitmapDocIdSet;
+import org.apache.pinot.core.operator.docidsets.MatchAllDocIdSet;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.segment.spi.datasource.DataSource;
+import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader;
+
+
+public abstract class NullHandlingSupportedFilterOperator extends BaseFilterOperator {
+  protected final QueryContext _queryContext;
+  protected final DataSource _dataSource;
+  protected final int _numDocs;
+
+  protected NullHandlingSupportedFilterOperator(QueryContext queryContext, DataSource dataSource, int numDocs) {
+    _queryContext = queryContext;
+    _dataSource = dataSource;
+    _numDocs = numDocs;
+  }
+
+  @Override
+  protected FilterBlock getNextBlock() {
+    if (_queryContext.isNullHandlingEnabled()) {
+      return new FilterBlock(excludeNulls(getNextBlockWithoutNullHandling()));
+    } else {
+      return new FilterBlock(getNextBlockWithoutNullHandling());
+    }
+  }
+
+  protected abstract BlockDocIdSet getNextBlockWithoutNullHandling();
+
+  private BlockDocIdSet excludeNulls(BlockDocIdSet blockDocIdSet) {
+    return new AndDocIdSet(

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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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] [pinot] shenyu0127 commented on a diff in pull request #11078: Filtering NULL support.

Posted by "shenyu0127 (via GitHub)" <gi...@apache.org>.
shenyu0127 commented on code in PR #11078:
URL: https://github.com/apache/pinot/pull/11078#discussion_r1270863550


##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/NullHandlingSupportedFilterOperator.java:
##########
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.operator.filter;
+
+import java.util.Arrays;
+import org.apache.pinot.core.common.BlockDocIdSet;
+import org.apache.pinot.core.operator.blocks.FilterBlock;
+import org.apache.pinot.core.operator.docidsets.AndDocIdSet;
+import org.apache.pinot.core.operator.docidsets.BitmapDocIdSet;
+import org.apache.pinot.core.operator.docidsets.MatchAllDocIdSet;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.segment.spi.datasource.DataSource;
+import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader;
+
+
+public abstract class NullHandlingSupportedFilterOperator extends BaseFilterOperator {

Review Comment:
   We can't put this logic inside `BaseFilterOperator` because some subclasses do not share the same logic. Here are some examples:
   
   - Non-leaf filter operators, e.g. `AndFilterOperator`
   - `TextMatchFilterOperator`
   - `ExpressionFilterOperator` (the `nullBitmap` is from `TransformFunction::getNullBitmap` so we don't need `_dataSource`)
   
   I renamed the class to `NullHandlingSupportedSingleColumnLeafFilterOperator` to indicate it's the base class of a subset of filter operators.
   
   We might do some tweaks to make some leaf filter operators a subclass of this class, but I'd defer that to not increase the scope of this PR.



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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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] [pinot] shenyu0127 commented on pull request #11078: Column filter operators NULL support.

Posted by "shenyu0127 (via GitHub)" <gi...@apache.org>.
shenyu0127 commented on PR #11078:
URL: https://github.com/apache/pinot/pull/11078#issuecomment-1652790028

   @Jackie-Jiang I merged this branch with updated master.  Please take another look.


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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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] [pinot] Jackie-Jiang merged pull request #11078: Column filter operators NULL support.

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang merged PR #11078:
URL: https://github.com/apache/pinot/pull/11078


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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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] [pinot] shenyu0127 commented on a diff in pull request #11078: Column filter operators NULL support.

Posted by "shenyu0127 (via GitHub)" <gi...@apache.org>.
shenyu0127 commented on code in PR #11078:
URL: https://github.com/apache/pinot/pull/11078#discussion_r1275660929


##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/BaseColumnFilterOperator.java:
##########
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.operator.filter;
+
+import java.util.Arrays;
+import org.apache.pinot.core.common.BlockDocIdSet;
+import org.apache.pinot.core.operator.docidsets.AndDocIdSet;
+import org.apache.pinot.core.operator.docidsets.BitmapDocIdSet;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.segment.spi.datasource.DataSource;
+import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader;
+import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
+
+
+public abstract class BaseColumnFilterOperator extends BaseFilterOperator {
+  protected final QueryContext _queryContext;
+  protected final DataSource _dataSource;
+
+  protected BaseColumnFilterOperator(QueryContext queryContext, DataSource dataSource, int numDocs) {
+    super(numDocs, queryContext.isNullHandlingEnabled());
+    _queryContext = queryContext;
+    _dataSource = dataSource;
+  }
+
+  protected abstract BlockDocIdSet getNextBlockWithoutNullHandling();
+
+  @Override
+  protected BlockDocIdSet getTrues() {
+    if (_nullHandlingEnabled) {
+      ImmutableRoaringBitmap nullBitmap = getNullBitmap();
+      if (nullBitmap != null && !nullBitmap.isEmpty()) {
+        return excludeNulls(getNextBlockWithoutNullHandling(), nullBitmap);
+      }
+    }
+    return getNextBlockWithoutNullHandling();
+  }
+
+
+  @Override
+  protected BlockDocIdSet getNulls() {
+    return new BitmapDocIdSet(getNullBitmap(), _numDocs);

Review Comment:
   Good catch.
   
   Fixed the bug and added a unit test.



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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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] [pinot] codecov-commenter commented on pull request #11078: Filtering NULL support.

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #11078:
URL: https://github.com/apache/pinot/pull/11078#issuecomment-1631257947

   ## [Codecov](https://app.codecov.io/gh/apache/pinot/pull/11078?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   > Merging [#11078](https://app.codecov.io/gh/apache/pinot/pull/11078?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (f9db56c) into [master](https://app.codecov.io/gh/apache/pinot/commit/9bb3cbb3d6ebf5a9fcd8a5410a5794aee02c7ee4?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (9bb3cbb) will **increase** coverage by `0.00%`.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #11078     +/-   ##
   =========================================
     Coverage    0.11%    0.11%             
   =========================================
     Files        2201     2147     -54     
     Lines      118091   115573   -2518     
     Branches    17872    17568    -304     
   =========================================
     Hits          137      137             
   + Misses     117934   115416   -2518     
     Partials       20       20             
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1temurin11 | `?` | |
   | integration1temurin17 | `?` | |
   | integration1temurin20 | `?` | |
   | integration2temurin11 | `?` | |
   | integration2temurin17 | `?` | |
   | integration2temurin20 | `?` | |
   | unittests1temurin11 | `?` | |
   | unittests1temurin17 | `?` | |
   | unittests1temurin20 | `?` | |
   | unittests2temurin11 | `?` | |
   | unittests2temurin17 | `0.11% <0.00%> (ø)` | |
   | unittests2temurin20 | `0.11% <0.00%> (ø)` | |
   
   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=apache#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://app.codecov.io/gh/apache/pinot/pull/11078?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [...inot/core/operator/filter/FilterOperatorUtils.java](https://app.codecov.io/gh/apache/pinot/pull/11078?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9maWx0ZXIvRmlsdGVyT3BlcmF0b3JVdGlscy5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...ava/org/apache/pinot/core/plan/FilterPlanNode.java](https://app.codecov.io/gh/apache/pinot/pull/11078?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9wbGFuL0ZpbHRlclBsYW5Ob2RlLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...core/startree/operator/StarTreeFilterOperator.java](https://app.codecov.io/gh/apache/pinot/pull/11078?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9zdGFydHJlZS9vcGVyYXRvci9TdGFyVHJlZUZpbHRlck9wZXJhdG9yLmphdmE=) | `0.00% <0.00%> (ø)` | |
   
   ... and [56 files with indirect coverage changes](https://app.codecov.io/gh/apache/pinot/pull/11078/indirect-changes?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   


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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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