You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2022/12/19 03:31:48 UTC

[GitHub] [doris] xiaokang commented on a diff in pull request #11579: [Feature](NGram BloomFilter Index) add new ngram bloom filter index to speed up like query

xiaokang commented on code in PR #11579:
URL: https://github.com/apache/doris/pull/11579#discussion_r1051746186


##########
fe/fe-core/src/main/java/org/apache/doris/analysis/IndexDef.java:
##########
@@ -177,6 +188,30 @@ public void checkColumn(Column column, KeysType keysType) throws AnalysisExcepti
 
             if (indexType == IndexType.INVERTED) {
                 InvertedIndexUtil.checkInvertedIndexParser(indexColName, colType, properties);
+            } else if (indexType == IndexType.NGRAM_BF) {
+                if (colType != PrimitiveType.CHAR && colType != PrimitiveType.VARCHAR) {

Review Comment:
   is STRING missed?



##########
gensrc/proto/olap_file.proto:
##########
@@ -199,12 +199,16 @@ message ColumnPB {
     optional bool visible = 16 [default=true];
     repeated ColumnPB children_columns = 17;
     repeated string children_column_names = 18;
+    optional bool ngram_bf_column = 19 [default=false];

Review Comment:
   can we use properties in IndexPB



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java:
##########
@@ -685,6 +685,17 @@ public void setIndexFlag(TColumn tColumn, OlapTable olapTable) {
                 if (tColumn.getColumnName().equals(columns.get(0))) {
                     tColumn.setHasBitmapIndex(true);
                 }
+            } else if (index.getIndexType() == IndexDef.IndexType.NGRAM_BF) {
+                List<String> columns = index.getColumns();
+                if (tColumn.getColumnName().equals(columns.get(0))) {
+                    tColumn.setHasNgramBfIndex(true);
+                    String ngramSize = index.getProperties().getOrDefault(IndexDef.NGRAM_SIZE_KEY,
+                            IndexDef.DEFAULT_NGRAM_SIZE);
+                    String bfSize = index.getProperties().getOrDefault(IndexDef.NGRAM_BF_SIZE_KEY,
+                            IndexDef.DEFAULT_NGRAM_BF_SIZE);
+                    tColumn.setGramSize(Integer.parseInt(ngramSize));

Review Comment:
   can we use properties instead of column field?



##########
be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp:
##########
@@ -203,5 +260,22 @@ Status BloomFilterIndexWriter::create(const BloomFilterOptions& bf_options,
     return Status::OK();
 }
 
+Status NGramBloomFilterIndexWriterImpl::create(const BloomFilterOptions& bf_options,
+                                               const TypeInfo* typeinfo, uint8_t gram_size,
+                                               uint16_t gram_bf_size,
+                                               std::unique_ptr<BloomFilterIndexWriter>* res) {
+    FieldType type = typeinfo->type();
+    switch (type) {
+    case OLAP_FIELD_TYPE_CHAR:
+    case OLAP_FIELD_TYPE_VARCHAR:

Review Comment:
   OLAP_FIELD_TYPE_STRING



##########
be/src/olap/tablet_meta.cpp:
##########
@@ -284,6 +288,12 @@ void TabletMeta::init_column_from_tcolumn(uint32_t unique_id, const TColumn& tco
     if (tcolumn.__isset.is_bloom_filter_column) {
         column->set_is_bf_column(tcolumn.is_bloom_filter_column);
     }
+    if (tcolumn.has_ngram_bf_index) {

Review Comment:
   can we use index properties



##########
fe/fe-core/src/main/java/org/apache/doris/analysis/IndexDef.java:
##########
@@ -37,6 +37,11 @@ public class IndexDef {
     private String comment;
     private Map<String, String> properties;
 
+    public static final String NGRAM_SIZE_KEY = "gram_size";
+    public static final String NGRAM_BF_SIZE_KEY = "bf_size";

Review Comment:
   can we make it easier for users to set FPR of bf and calculate bf size automatically, just like https://www.postgresql.org/docs/14/brin-builtin-opclasses.html



##########
be/src/olap/itoken_extractor.h:
##########
@@ -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.
+
+#ifndef DORIS_ITOKEN_EXTRACTOR_H
+#define DORIS_ITOKEN_EXTRACTOR_H
+
+#include <stddef.h>
+
+#include <string>
+
+#include "olap/rowset/segment_v2/bloom_filter.h"
+
+namespace doris {
+
+/// Interface for string parsers.
+struct ITokenExtractor {
+    virtual ~ITokenExtractor() = default;
+
+    /// Fast inplace implementation for regular use.
+    /// Gets string (data ptr and len) and start position for extracting next token (state of extractor).
+    /// Returns false if parsing is finished, otherwise returns true.
+    virtual bool nextInString(const char* data, size_t length, size_t* __restrict pos,

Review Comment:
   function name with lower case words separated by underscore is recommend. 



-- 
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@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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