From 10009dc062046759151a1bdbd17ac03d0f0d9f11 Mon Sep 17 00:00:00 2001 From: Sun Chenyang Date: Sat, 16 Nov 2024 16:29:00 +0800 Subject: [PATCH] [fix](variant) fix index in variant (#43375) (#43971) Problem Summary: 1. Fixing error checks, `InvertedIndexColumnWriter::check_support_inverted_index`, It is not appropriate to determine support for other indexes by checking the inverted index. 2. Fix `TableSchema::update_index()` pick from master #43375 --- .../rowset/segment_v2/segment_iterator.cpp | 7 + .../olap/rowset/segment_v2/segment_writer.cpp | 44 +++--- .../segment_v2/vertical_segment_writer.cpp | 42 ++---- be/src/olap/tablet_schema.cpp | 12 +- be/src/olap/tablet_schema.h | 2 +- be/src/vec/common/schema_util.cpp | 11 +- be/src/vec/common/schema_util.h | 2 + be/test/common/schema_util_test.cpp | 109 ++++++++++++++ .../test_variant_bloom_filter.out | 40 +++++ .../test_variant_bloom_filter.groovy | 140 ++++++++++++++++++ 10 files changed, 349 insertions(+), 60 deletions(-) create mode 100644 be/test/common/schema_util_test.cpp create mode 100644 regression-test/data/fault_injection_p0/test_variant_bloom_filter.out create mode 100644 regression-test/suites/fault_injection_p0/test_variant_bloom_filter.groovy diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp index 3a3ce68cb84e6e..43ed3720cbd0c0 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp +++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp @@ -584,6 +584,13 @@ Status SegmentIterator::_get_row_ranges_from_conditions(RowRanges* condition_row pre_size = condition_row_ranges->count(); RowRanges::ranges_intersection(*condition_row_ranges, bf_row_ranges, condition_row_ranges); _opts.stats->rows_bf_filtered += (pre_size - condition_row_ranges->count()); + + DBUG_EXECUTE_IF("bloom_filter_must_filter_data", { + if (pre_size - condition_row_ranges->count() == 0) { + return Status::Error( + "Bloom filter did not filter the data."); + } + }) } { diff --git a/be/src/olap/rowset/segment_v2/segment_writer.cpp b/be/src/olap/rowset/segment_v2/segment_writer.cpp index 7c9e608e828f0f..e1696340a361b7 100644 --- a/be/src/olap/rowset/segment_v2/segment_writer.cpp +++ b/be/src/olap/rowset/segment_v2/segment_writer.cpp @@ -230,11 +230,6 @@ Status SegmentWriter::init(const std::vector& col_ids, bool has_key) { } // indexes for this column opts.indexes = std::move(_tablet_schema->get_indexes_for_column(column)); - if (!InvertedIndexColumnWriter::check_support_inverted_index(column)) { - opts.need_zone_map = false; - opts.need_bloom_filter = false; - opts.need_bitmap_index = false; - } opts.inverted_index_file_writer = _inverted_index_file_writer.get(); for (auto index : opts.indexes) { if (!skip_inverted_index && index->index_type() == IndexType::INVERTED) { @@ -244,27 +239,24 @@ Status SegmentWriter::init(const std::vector& col_ids, bool has_key) { break; } } -#define CHECK_FIELD_TYPE(TYPE, type_name) \ - if (column.type() == FieldType::OLAP_FIELD_TYPE_##TYPE) { \ - opts.need_zone_map = false; \ - if (opts.need_bloom_filter) { \ - return Status::NotSupported("Do not support bloom filter for " type_name " type"); \ - } \ - if (opts.need_bitmap_index) { \ - return Status::NotSupported("Do not support bitmap index for " type_name " type"); \ - } \ - } - - CHECK_FIELD_TYPE(STRUCT, "struct") - CHECK_FIELD_TYPE(ARRAY, "array") - CHECK_FIELD_TYPE(JSONB, "jsonb") - CHECK_FIELD_TYPE(AGG_STATE, "agg_state") - CHECK_FIELD_TYPE(MAP, "map") - CHECK_FIELD_TYPE(OBJECT, "object") - CHECK_FIELD_TYPE(HLL, "hll") - CHECK_FIELD_TYPE(QUANTILE_STATE, "quantile_state") - -#undef CHECK_FIELD_TYPE +#define DISABLE_INDEX_IF_FIELD_TYPE(TYPE, type_name) \ + if (column.type() == FieldType::OLAP_FIELD_TYPE_##TYPE) { \ + opts.need_zone_map = false; \ + opts.need_bloom_filter = false; \ + opts.need_bitmap_index = false; \ + } + + DISABLE_INDEX_IF_FIELD_TYPE(STRUCT, "struct") + DISABLE_INDEX_IF_FIELD_TYPE(ARRAY, "array") + DISABLE_INDEX_IF_FIELD_TYPE(JSONB, "jsonb") + DISABLE_INDEX_IF_FIELD_TYPE(AGG_STATE, "agg_state") + DISABLE_INDEX_IF_FIELD_TYPE(MAP, "map") + DISABLE_INDEX_IF_FIELD_TYPE(OBJECT, "object") + DISABLE_INDEX_IF_FIELD_TYPE(HLL, "hll") + DISABLE_INDEX_IF_FIELD_TYPE(QUANTILE_STATE, "quantile_state") + DISABLE_INDEX_IF_FIELD_TYPE(VARIANT, "variant") + +#undef DISABLE_INDEX_IF_FIELD_TYPE if (column.is_row_store_column()) { // smaller page size for row store column diff --git a/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp b/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp index 2a929f79d507f8..fcaad8f555a206 100644 --- a/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp +++ b/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp @@ -185,13 +185,6 @@ Status VerticalSegmentWriter::_create_column_writer(uint32_t cid, const TabletCo } // indexes for this column opts.indexes = tablet_schema->get_indexes_for_column(column); - if (!InvertedIndexColumnWriter::check_support_inverted_index(column)) { - // skip inverted index if invalid - opts.indexes.clear(); - opts.need_zone_map = false; - opts.need_bloom_filter = false; - opts.need_bitmap_index = false; - } for (auto index : opts.indexes) { if (!skip_inverted_index && index->index_type() == IndexType::INVERTED) { opts.inverted_index = index; @@ -202,27 +195,24 @@ Status VerticalSegmentWriter::_create_column_writer(uint32_t cid, const TabletCo } opts.inverted_index_file_writer = _inverted_index_file_writer.get(); -#define CHECK_FIELD_TYPE(TYPE, type_name) \ - if (column.type() == FieldType::OLAP_FIELD_TYPE_##TYPE) { \ - opts.need_zone_map = false; \ - if (opts.need_bloom_filter) { \ - return Status::NotSupported("Do not support bloom filter for " type_name " type"); \ - } \ - if (opts.need_bitmap_index) { \ - return Status::NotSupported("Do not support bitmap index for " type_name " type"); \ - } \ +#define DISABLE_INDEX_IF_FIELD_TYPE(TYPE, type_name) \ + if (column.type() == FieldType::OLAP_FIELD_TYPE_##TYPE) { \ + opts.need_zone_map = false; \ + opts.need_bloom_filter = false; \ + opts.need_bitmap_index = false; \ } - CHECK_FIELD_TYPE(STRUCT, "struct") - CHECK_FIELD_TYPE(ARRAY, "array") - CHECK_FIELD_TYPE(JSONB, "jsonb") - CHECK_FIELD_TYPE(AGG_STATE, "agg_state") - CHECK_FIELD_TYPE(MAP, "map") - CHECK_FIELD_TYPE(OBJECT, "object") - CHECK_FIELD_TYPE(HLL, "hll") - CHECK_FIELD_TYPE(QUANTILE_STATE, "quantile_state") - -#undef CHECK_FIELD_TYPE + DISABLE_INDEX_IF_FIELD_TYPE(STRUCT, "struct") + DISABLE_INDEX_IF_FIELD_TYPE(ARRAY, "array") + DISABLE_INDEX_IF_FIELD_TYPE(JSONB, "jsonb") + DISABLE_INDEX_IF_FIELD_TYPE(AGG_STATE, "agg_state") + DISABLE_INDEX_IF_FIELD_TYPE(MAP, "map") + DISABLE_INDEX_IF_FIELD_TYPE(OBJECT, "object") + DISABLE_INDEX_IF_FIELD_TYPE(HLL, "hll") + DISABLE_INDEX_IF_FIELD_TYPE(QUANTILE_STATE, "quantile_state") + DISABLE_INDEX_IF_FIELD_TYPE(VARIANT, "variant") + +#undef DISABLE_INDEX_IF_FIELD_TYPE if (column.is_row_store_column()) { // smaller page size for row store column diff --git a/be/src/olap/tablet_schema.cpp b/be/src/olap/tablet_schema.cpp index f6146c7379857b..ef0ff5120396c6 100644 --- a/be/src/olap/tablet_schema.cpp +++ b/be/src/olap/tablet_schema.cpp @@ -899,14 +899,17 @@ void TabletSchema::append_index(TabletIndex index) { _indexes.push_back(std::move(index)); } -void TabletSchema::update_index(const TabletColumn& col, TabletIndex index) { - int32_t col_unique_id = col.unique_id(); +void TabletSchema::update_index(const TabletColumn& col, const IndexType& index_type, + TabletIndex&& index) { + int32_t col_unique_id = col.is_extracted_column() ? col.parent_unique_id() : col.unique_id(); const std::string& suffix_path = col.has_path_info() ? escape_for_path_name(col.path_info_ptr()->get_path()) : ""; for (size_t i = 0; i < _indexes.size(); i++) { for (int32_t id : _indexes[i].col_unique_ids()) { - if (id == col_unique_id && _indexes[i].get_index_suffix() == suffix_path) { - _indexes[i] = index; + if (_indexes[i].index_type() == index_type && id == col_unique_id && + _indexes[i].get_index_suffix() == suffix_path) { + _indexes[i] = std::move(index); + break; } } } @@ -1442,7 +1445,6 @@ const TabletIndex* TabletSchema::get_ngram_bf_index(int32_t col_unique_id) const } } } - return nullptr; } diff --git a/be/src/olap/tablet_schema.h b/be/src/olap/tablet_schema.h index 159ecf78d73103..79a885dfa1ee88 100644 --- a/be/src/olap/tablet_schema.h +++ b/be/src/olap/tablet_schema.h @@ -293,7 +293,7 @@ class TabletSchema { void to_schema_pb(TabletSchemaPB* tablet_meta_pb) const; void append_column(TabletColumn column, ColumnType col_type = ColumnType::NORMAL); void append_index(TabletIndex index); - void update_index(const TabletColumn& column, TabletIndex index); + void update_index(const TabletColumn& column, const IndexType& index_type, TabletIndex&& index); void remove_index(int64_t index_id); void clear_index(); // Must make sure the row column is always the last column diff --git a/be/src/vec/common/schema_util.cpp b/be/src/vec/common/schema_util.cpp index 7ad7cf972bd5e1..b373dbd13479d8 100644 --- a/be/src/vec/common/schema_util.cpp +++ b/be/src/vec/common/schema_util.cpp @@ -375,6 +375,10 @@ void update_least_sparse_column(const std::vector& schemas, void inherit_column_attributes(const TabletColumn& source, TabletColumn& target, TabletSchemaSPtr& target_schema) { + DCHECK(target.is_extracted_column()); + target.set_aggregation_method(source.aggregation()); + + // 1. bloom filter if (target.type() != FieldType::OLAP_FIELD_TYPE_TINYINT && target.type() != FieldType::OLAP_FIELD_TYPE_ARRAY && target.type() != FieldType::OLAP_FIELD_TYPE_DOUBLE && @@ -382,7 +386,8 @@ void inherit_column_attributes(const TabletColumn& source, TabletColumn& target, // above types are not supported in bf target.set_is_bf_column(source.is_bf_column()); } - target.set_aggregation_method(source.aggregation()); + + // 2. inverted index const auto* source_index_meta = target_schema->get_inverted_index(source.unique_id(), ""); if (source_index_meta != nullptr) { // add index meta @@ -392,11 +397,13 @@ void inherit_column_attributes(const TabletColumn& source, TabletColumn& target, const auto* target_index_meta = target_schema->get_inverted_index(target, false); if (target_index_meta != nullptr) { // already exist - target_schema->update_index(target, index_info); + target_schema->update_index(target, IndexType::INVERTED, std::move(index_info)); } else { target_schema->append_index(index_info); } } + + // 3. TODO: gnragm bf index } void inherit_column_attributes(TabletSchemaSPtr& schema) { diff --git a/be/src/vec/common/schema_util.h b/be/src/vec/common/schema_util.h index 4fb61fe592ad85..c2917fb8f4df4c 100644 --- a/be/src/vec/common/schema_util.h +++ b/be/src/vec/common/schema_util.h @@ -118,6 +118,8 @@ void update_least_sparse_column(const std::vector& schemas, // inherit attributes like index/agg info from it's parent column void inherit_column_attributes(TabletSchemaSPtr& schema); +// source: variant column +// target: extracted column from variant column void inherit_column_attributes(const TabletColumn& source, TabletColumn& target, TabletSchemaSPtr& target_schema); diff --git a/be/test/common/schema_util_test.cpp b/be/test/common/schema_util_test.cpp new file mode 100644 index 00000000000000..5fd157756bf674 --- /dev/null +++ b/be/test/common/schema_util_test.cpp @@ -0,0 +1,109 @@ +// 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. + +#include "vec/common/schema_util.h" + +#include + +namespace doris { + +class SchemaUtilTest : public testing::Test {}; + +void construct_column(ColumnPB* column_pb, TabletIndexPB* tablet_index, int64_t index_id, + const std::string& index_name, int32_t col_unique_id, + const std::string& column_type, const std::string& column_name, + const IndexType& index_type) { + column_pb->set_unique_id(col_unique_id); + column_pb->set_name(column_name); + column_pb->set_type(column_type); + column_pb->set_is_nullable(true); + column_pb->set_is_bf_column(true); + tablet_index->set_index_id(index_id); + tablet_index->set_index_name(index_name); + tablet_index->set_index_type(index_type); + tablet_index->add_col_unique_id(col_unique_id); +} + +void construct_subcolumn(TabletSchemaSPtr schema, const FieldType& type, int32_t col_unique_id, + std::string_view path, std::vector* subcolumns) { + TabletColumn subcol; + subcol.set_type(type); + subcol.set_is_nullable(true); + subcol.set_unique_id(-1); + subcol.set_parent_unique_id(col_unique_id); + vectorized::PathInData col_path(path); + subcol.set_path_info(col_path); + subcol.set_name(col_path.get_path()); + schema->append_column(subcol); + subcolumns->emplace_back(std::move(subcol)); +} + +TEST_F(SchemaUtilTest, inherit_column_attributes) { + TabletSchemaPB schema_pb; + schema_pb.set_keys_type(KeysType::DUP_KEYS); + schema_pb.set_inverted_index_storage_format(InvertedIndexStorageFormatPB::V2); + + construct_column(schema_pb.add_column(), schema_pb.add_index(), 10000, "key_index", 0, "INT", + "key", IndexType::INVERTED); + construct_column(schema_pb.add_column(), schema_pb.add_index(), 10001, "v1_index", 1, "VARIANT", + "v1", IndexType::INVERTED); + construct_column(schema_pb.add_column(), schema_pb.add_index(), 10003, "v3_index", 3, "VARIANT", + "v3", IndexType::INVERTED); + + TabletSchemaSPtr tablet_schema = std::make_shared(); + tablet_schema->init_from_pb(schema_pb); + std::vector subcolumns; + + construct_subcolumn(tablet_schema, FieldType::OLAP_FIELD_TYPE_STRING, 1, "v1.b", &subcolumns); + construct_subcolumn(tablet_schema, FieldType::OLAP_FIELD_TYPE_INT, 1, "v1.c", &subcolumns); + + construct_subcolumn(tablet_schema, FieldType::OLAP_FIELD_TYPE_ARRAY, 3, "v3.d", &subcolumns); + construct_subcolumn(tablet_schema, FieldType::OLAP_FIELD_TYPE_FLOAT, 3, "v3.a", &subcolumns); + + vectorized::schema_util::inherit_column_attributes(tablet_schema); + for (const auto& col : subcolumns) { + switch (col._parent_col_unique_id) { + case 1: + EXPECT_TRUE(tablet_schema->get_inverted_index(col) != nullptr); + break; + case 3: + EXPECT_TRUE(tablet_schema->get_inverted_index(col) == nullptr); + break; + default: + EXPECT_TRUE(false); + } + } + EXPECT_EQ(tablet_schema->indexes().size(), 7); + + for (const auto& col : tablet_schema->_cols) { + if (!col->is_extracted_column()) { + continue; + } + switch (col->_parent_col_unique_id) { + case 1: + EXPECT_TRUE(col->is_bf_column()); + break; + case 3: + EXPECT_TRUE(!col->is_bf_column()); + break; + default: + EXPECT_TRUE(false); + } + } +} + +} // namespace doris diff --git a/regression-test/data/fault_injection_p0/test_variant_bloom_filter.out b/regression-test/data/fault_injection_p0/test_variant_bloom_filter.out new file mode 100644 index 00000000000000..2a9d418249afc2 --- /dev/null +++ b/regression-test/data/fault_injection_p0/test_variant_bloom_filter.out @@ -0,0 +1,40 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql1 -- +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 +20291263 + +-- !sql2 -- +ridget/dotfiles +ridget/dotfiles +ridget/dotfiles +ridget/dotfiles +ridget/dotfiles + diff --git a/regression-test/suites/fault_injection_p0/test_variant_bloom_filter.groovy b/regression-test/suites/fault_injection_p0/test_variant_bloom_filter.groovy new file mode 100644 index 00000000000000..33f93d1bef0a6f --- /dev/null +++ b/regression-test/suites/fault_injection_p0/test_variant_bloom_filter.groovy @@ -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. + +import java.util.concurrent.TimeUnit +import org.awaitility.Awaitility + +suite("test_variant_bloom_filter", "nonConcurrent") { + + def index_table = "test_variant_bloom_filter" + + def load_json_data = {table_name, file_name -> + // load the json data + streamLoad { + table "${table_name}" + + // set http request header params + set 'read_json_by_line', 'true' + set 'format', 'json' + set 'max_filter_ratio', '0.1' + set 'memtable_on_sink_node', 'true' + file file_name // import json file + time 10000 // limit inflight 10s + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + logger.info("Stream load ${file_name} result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + // assertEquals(json.NumberTotalRows, json.NumberLoadedRows + json.NumberUnselectedRows) + assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes > 0) + } + } + } + + sql """DROP TABLE IF EXISTS ${index_table}""" + sql """ + CREATE TABLE IF NOT EXISTS ${index_table} ( + k bigint, + v variant + ) + DUPLICATE KEY(`k`) + DISTRIBUTED BY HASH(k) BUCKETS 1 + properties("replication_num" = "1", "disable_auto_compaction" = "false", "bloom_filter_columns" = "v"); + """ + load_json_data.call(index_table, """${getS3Url() + '/regression/gharchive.m/2015-01-01-0.json'}""") + load_json_data.call(index_table, """${getS3Url() + '/regression/gharchive.m/2015-01-01-0.json'}""") + load_json_data.call(index_table, """${getS3Url() + '/regression/gharchive.m/2015-01-01-0.json'}""") + load_json_data.call(index_table, """${getS3Url() + '/regression/gharchive.m/2015-01-01-0.json'}""") + load_json_data.call(index_table, """${getS3Url() + '/regression/gharchive.m/2015-01-01-0.json'}""") + + def backendId_to_backendIP = [:] + def backendId_to_backendHttpPort = [:] + getBackendIpHttpPort(backendId_to_backendIP, backendId_to_backendHttpPort); + def tablets = sql_return_maparray """ show tablets from ${index_table}; """ + + int beforeSegmentCount = 0 + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + beforeSegmentCount += Integer.parseInt(rowset.split(" ")[1]) + } + } + assertEquals(beforeSegmentCount, 5) + + // trigger compactions for all tablets in ${tableName} + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_run_full_compaction(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactJson = parseJson(out.trim()) + assertEquals("success", compactJson.status.toLowerCase()) + } + + // wait for all compactions done + for (def tablet in tablets) { + Awaitility.await().atMost(3, TimeUnit.MINUTES).untilAsserted(() -> { + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("compaction task for this tablet is not running", compactionStatus.msg.toLowerCase()) + return compactionStatus.run_status; + }); + } + + int afterSegmentCount = 0 + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + logger.info("rowset is: " + rowset) + afterSegmentCount += Integer.parseInt(rowset.split(" ")[1]) + } + } + assertEquals(afterSegmentCount, 1) + + try { + GetDebugPoint().enableDebugPointForAllBEs("bloom_filter_must_filter_data") + + // number + qt_sql1 """ select cast(v['repo']['id'] as int) from ${index_table} where cast(v['repo']['id'] as int) = 20291263; """ + + // string + qt_sql2 """ select cast(v['repo']['name'] as text) from ${index_table} where cast(v['repo']['name'] as text) = "ridget/dotfiles"; """ + } finally { + GetDebugPoint().disableDebugPointForAllBEs("bloom_filter_must_filter_data") + } +} \ No newline at end of file