You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2023/01/13 06:13:51 UTC

[doris] branch branch-1.2-lts updated (6b69980c75 -> 750bf85030)

This is an automated email from the ASF dual-hosted git repository.

morningman pushed a change to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git


    from 6b69980c75 [fix](compile) fix vscan node compile error (#15805)
     new 32373051cd [enhancement](unique key) disable concurrent flush memtable for unique key (#15802)
     new 34b817625a [Bug](bitmap) Fix bitmap_from_string for null constant (#15698)
     new 915abc8c53 [fix](olap) dictionary cannot be sorted after inserting some null values (#15829)
     new 03289c9c0f [fix](bitmap) fix bitmap iterator comparison error (#15779)
     new 750bf85030 [fix](parquet-reader) fix coredump when load datatime data to doris from parquet (#15794)

The 5 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/olap/delta_writer.cpp                       |  6 +--
 be/src/runtime/types.h                             |  1 +
 be/src/util/bitmap_value.h                         |  2 +-
 be/src/vec/columns/column_const.cpp                |  3 +-
 be/src/vec/columns/column_dictionary.h             |  6 +++
 be/src/vec/exec/format/parquet/schema_desc.cpp     | 63 +++++++++++-----------
 be/src/vec/functions/function_bitmap.cpp           | 16 ++++--
 be/test/util/bitmap_value_test.cpp                 | 10 ++--
 .../bitmap/test_bitmap_const.out}                  |  2 +-
 .../data/query_p0/join/test_bitmap_filter.out      |  7 +++
 .../datatype_p0/bitmap/test_bitmap_const.groovy    |  6 +--
 .../suites/query_p0/join/test_bitmap_filter.groovy |  4 ++
 .../test_dict_with_null.groovy}                    | 42 +++++++--------
 13 files changed, 100 insertions(+), 68 deletions(-)
 copy regression-test/data/{correctness_p0/test_join_with_projection.out => datatype_p0/bitmap/test_bitmap_const.out} (97%)
 copy fe/fe-core/src/main/java/org/apache/doris/analysis/CancelStmt.java => regression-test/suites/datatype_p0/bitmap/test_bitmap_const.groovy (89%)
 copy regression-test/suites/{performance_p0/test_streamload_perfomance.groovy => query_p0/test_dict_with_null.groovy} (53%)


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


[doris] 04/05: [fix](bitmap) fix bitmap iterator comparison error (#15779)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 03289c9c0f57a00b6fc609f35cf6ed6cc454bce9
Author: luozenglin <37...@users.noreply.github.com>
AuthorDate: Fri Jan 13 11:37:07 2023 +0800

    [fix](bitmap) fix bitmap iterator comparison error (#15779)
    
    Fix the bug that bitmap.begin() == bitmap.end() is always true when the bitmap contains a single value.
---
 be/src/util/bitmap_value.h                                     |  2 +-
 be/test/util/bitmap_value_test.cpp                             | 10 +++++++---
 regression-test/data/query_p0/join/test_bitmap_filter.out      |  7 +++++++
 regression-test/suites/query_p0/join/test_bitmap_filter.groovy |  4 ++++
 4 files changed, 19 insertions(+), 4 deletions(-)

diff --git a/be/src/util/bitmap_value.h b/be/src/util/bitmap_value.h
index 2717f18895..21fa5f217e 100644
--- a/be/src/util/bitmap_value.h
+++ b/be/src/util/bitmap_value.h
@@ -1975,7 +1975,7 @@ public:
         case BitmapValue::BitmapDataType::EMPTY:
             return other._bitmap._type == BitmapValue::BitmapDataType::EMPTY;
         case BitmapValue::BitmapDataType::SINGLE:
-            return _sv == other._sv;
+            return _end == other._end && _sv == other._sv;
         case BitmapValue::BitmapDataType::BITMAP:
             return *_iter == *(other._iter);
         default:
diff --git a/be/test/util/bitmap_value_test.cpp b/be/test/util/bitmap_value_test.cpp
index 256fbbe914..ee950744db 100644
--- a/be/test/util/bitmap_value_test.cpp
+++ b/be/test/util/bitmap_value_test.cpp
@@ -390,9 +390,13 @@ TEST(BitmapValueTest, bitmap_value_iterator_test) {
     }
 
     BitmapValue single(1024);
-    for (auto iter = single.begin(); iter != single.end(); ++iter) {
-        EXPECT_EQ(1024, *iter);
-    }
+    auto single_iter = single.begin();
+    EXPECT_EQ(1024, *single_iter);
+    EXPECT_TRUE(single_iter == BitmapValue {1024}.begin());
+    EXPECT_TRUE(single_iter != single.end());
+
+    ++single_iter;
+    EXPECT_TRUE(single_iter == single.end());
 
     int i = 0;
     BitmapValue bitmap({0, 1025, 1026, UINT32_MAX, UINT64_MAX});
diff --git a/regression-test/data/query_p0/join/test_bitmap_filter.out b/regression-test/data/query_p0/join/test_bitmap_filter.out
index 9484c6770d..f44900b0e0 100644
--- a/regression-test/data/query_p0/join/test_bitmap_filter.out
+++ b/regression-test/data/query_p0/join/test_bitmap_filter.out
@@ -91,3 +91,10 @@
 1991
 32767
 
+-- !sql13 --
+10	1991
+
+-- !sql14 --
+1	1989
+10	1991
+
diff --git a/regression-test/suites/query_p0/join/test_bitmap_filter.groovy b/regression-test/suites/query_p0/join/test_bitmap_filter.groovy
index 78b33040b4..53522289c9 100644
--- a/regression-test/suites/query_p0/join/test_bitmap_filter.groovy
+++ b/regression-test/suites/query_p0/join/test_bitmap_filter.groovy
@@ -66,6 +66,10 @@ suite("test_bitmap_filter", "query_p0") {
         select * from (select * from w1 union select * from w2) tmp order by 1;
     """
 
+    qt_sql13 "select k1, k2 from ${tbl1} where k1 in (select to_bitmap(10)) order by 1, 2"
+
+    qt_sql14 "select k1, k2 from ${tbl1} where k1 in (select bitmap_from_string('1,10')) order by 1, 2"
+
     test {
         sql "select k1, k2 from ${tbl1} b1 where k1 in (select k2 from ${tbl2} b2 where b1.k2 = b2.k1) order by k1;"
         exception "In bitmap does not support correlated subquery"


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


[doris] 02/05: [Bug](bitmap) Fix bitmap_from_string for null constant (#15698)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 34b817625a59ea29191f5e08d52e3caa34d01490
Author: Gabriel <ga...@gmail.com>
AuthorDate: Mon Jan 9 10:21:08 2023 +0800

    [Bug](bitmap) Fix bitmap_from_string for null constant (#15698)
---
 be/src/vec/columns/column_const.cpp                |  3 ++-
 be/src/vec/functions/function_bitmap.cpp           | 16 +++++++++++-----
 .../data/datatype_p0/bitmap/test_bitmap_const.out  |  4 ++++
 .../datatype_p0/bitmap/test_bitmap_const.groovy    | 22 ++++++++++++++++++++++
 4 files changed, 39 insertions(+), 6 deletions(-)

diff --git a/be/src/vec/columns/column_const.cpp b/be/src/vec/columns/column_const.cpp
index 4720b4cd02..d2cb9ef7e7 100644
--- a/be/src/vec/columns/column_const.cpp
+++ b/be/src/vec/columns/column_const.cpp
@@ -36,7 +36,8 @@ ColumnConst::ColumnConst(const ColumnPtr& data_, size_t s_) : data(data_), s(s_)
 
     if (data->size() != 1) {
         LOG(FATAL) << fmt::format(
-                "Incorrect size of nested column in constructor of ColumnConst: {}, must be 1.");
+                "Incorrect size of nested column in constructor of ColumnConst: {}, must be 1.",
+                data->size());
     }
 }
 
diff --git a/be/src/vec/functions/function_bitmap.cpp b/be/src/vec/functions/function_bitmap.cpp
index 247dcab19b..4350d3b256 100644
--- a/be/src/vec/functions/function_bitmap.cpp
+++ b/be/src/vec/functions/function_bitmap.cpp
@@ -183,11 +183,17 @@ struct BitmapFromString {
     static constexpr auto name = "bitmap_from_string";
 
     static Status vector(const ColumnString::Chars& data, const ColumnString::Offsets& offsets,
-                         std::vector<BitmapValue>& res, NullMap& null_map) {
-        auto size = offsets.size();
-        res.reserve(size);
+                         std::vector<BitmapValue>& res, NullMap& null_map,
+                         size_t input_rows_count) {
+        res.reserve(input_rows_count);
         std::vector<uint64_t> bits;
-        for (size_t i = 0; i < size; ++i) {
+        if (offsets.size() == 0 && input_rows_count == 1) {
+            // For NULL constant
+            res.emplace_back();
+            null_map[0] = 1;
+            return Status::OK();
+        }
+        for (size_t i = 0; i < input_rows_count; ++i) {
             const char* raw_str = reinterpret_cast<const char*>(&data[offsets[i - 1]]);
             int64_t str_size = offsets[i] - offsets[i - 1];
 
@@ -272,7 +278,7 @@ public:
             const auto& str_column = static_cast<const ColumnString&>(*argument_column);
             const ColumnString::Chars& data = str_column.get_chars();
             const ColumnString::Offsets& offsets = str_column.get_offsets();
-            Impl::vector(data, offsets, res, null_map);
+            Impl::vector(data, offsets, res, null_map, input_rows_count);
         } else if constexpr (std::is_same_v<typename Impl::ArgumentType, DataTypeArray>) {
             auto argument_type = remove_nullable(
                     assert_cast<const DataTypeArray&>(*block.get_by_position(arguments[0]).type)
diff --git a/regression-test/data/datatype_p0/bitmap/test_bitmap_const.out b/regression-test/data/datatype_p0/bitmap/test_bitmap_const.out
new file mode 100644
index 0000000000..2871306509
--- /dev/null
+++ b/regression-test/data/datatype_p0/bitmap/test_bitmap_const.out
@@ -0,0 +1,4 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !select --
+\N
+
diff --git a/regression-test/suites/datatype_p0/bitmap/test_bitmap_const.groovy b/regression-test/suites/datatype_p0/bitmap/test_bitmap_const.groovy
new file mode 100644
index 0000000000..e824ed631b
--- /dev/null
+++ b/regression-test/suites/datatype_p0/bitmap/test_bitmap_const.groovy
@@ -0,0 +1,22 @@
+// 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.
+
+suite("test_bitmap_const") {
+    qt_select "select bitmap_from_string(     cast(null as TEXT));"
+}
+
+


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


[doris] 01/05: [enhancement](unique key) disable concurrent flush memtable for unique key (#15802)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 32373051cd79a29d396694c871622939fb976857
Author: zhannngchen <48...@users.noreply.github.com>
AuthorDate: Thu Jan 12 12:10:50 2023 +0800

    [enhancement](unique key) disable concurrent flush memtable for unique key (#15802)
---
 be/src/olap/delta_writer.cpp | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/be/src/olap/delta_writer.cpp b/be/src/olap/delta_writer.cpp
index 09afa54be9..0a05a03006 100644
--- a/be/src/olap/delta_writer.cpp
+++ b/be/src/olap/delta_writer.cpp
@@ -141,9 +141,9 @@ Status DeltaWriter::init() {
     _reset_mem_table();
 
     // create flush handler
-    // unique key merge on write should flush serial cause calc delete bitmap should load segment serial
-    bool should_serial = (_tablet->keys_type() == KeysType::UNIQUE_KEYS &&
-                          _tablet->enable_unique_key_merge_on_write());
+    // unique key should flush serial because we need to make sure same key should sort
+    // in the same order in all replica.
+    bool should_serial = _tablet->keys_type() == KeysType::UNIQUE_KEYS;
     RETURN_NOT_OK(_storage_engine->memtable_flush_executor()->create_flush_token(
             &_flush_token, _rowset_writer->type(), should_serial, _req.is_high_priority));
 


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


[doris] 05/05: [fix](parquet-reader) fix coredump when load datatime data to doris from parquet (#15794)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 750bf85030e609c8c3dd7433ff1fb5201a9cde5a
Author: Ashin Gau <As...@users.noreply.github.com>
AuthorDate: Fri Jan 13 11:51:11 2023 +0800

    [fix](parquet-reader) fix coredump when load datatime data to doris from parquet (#15794)
    
    `date_time_v2` will check scale when constructed datatimev2:
    ```
    LOG(FATAL) << fmt::format("Scale {} is out of bounds", scale);
    ```
    
    This [PR](https://github.com/apache/doris/pull/15510) has fixed this issue, but parquet does not use constructor to create `TypeDescriptor`, leading the `scale = -1` when reading datetimev2 data.
---
 be/src/runtime/types.h                         |  1 +
 be/src/vec/exec/format/parquet/schema_desc.cpp | 63 ++++++++++++++------------
 2 files changed, 34 insertions(+), 30 deletions(-)

diff --git a/be/src/runtime/types.h b/be/src/runtime/types.h
index 06629a655e..aca6336f6b 100644
--- a/be/src/runtime/types.h
+++ b/be/src/runtime/types.h
@@ -74,6 +74,7 @@ struct TypeDescriptor {
             precision = 27;
             scale = 9;
         } else if (type == TYPE_DATETIMEV2) {
+            precision = 18;
             scale = 6;
         }
     }
diff --git a/be/src/vec/exec/format/parquet/schema_desc.cpp b/be/src/vec/exec/format/parquet/schema_desc.cpp
index 2af4d40ea2..b8b9b07184 100644
--- a/be/src/vec/exec/format/parquet/schema_desc.cpp
+++ b/be/src/vec/exec/format/parquet/schema_desc.cpp
@@ -167,24 +167,27 @@ TypeDescriptor FieldDescriptor::get_doris_type(const tparquet::SchemaElement& ph
     if (type.type == INVALID_TYPE) {
         switch (physical_schema.type) {
         case tparquet::Type::BOOLEAN:
-            type.type = TYPE_BOOLEAN;
+            type = TypeDescriptor(TYPE_BOOLEAN);
             break;
         case tparquet::Type::INT32:
-            type.type = TYPE_INT;
+            type = TypeDescriptor(TYPE_INT);
             break;
         case tparquet::Type::INT64:
+            type = TypeDescriptor(TYPE_BIGINT);
+            break;
         case tparquet::Type::INT96:
-            type.type = TYPE_BIGINT;
+            // in most cases, it's a nano timestamp
+            type = TypeDescriptor(TYPE_DATETIMEV2);
             break;
         case tparquet::Type::FLOAT:
-            type.type = TYPE_FLOAT;
+            type = TypeDescriptor(TYPE_FLOAT);
             break;
         case tparquet::Type::DOUBLE:
-            type.type = TYPE_DOUBLE;
+            type = TypeDescriptor(TYPE_DOUBLE);
             break;
         case tparquet::Type::BYTE_ARRAY:
         case tparquet::Type::FIXED_LEN_BYTE_ARRAY:
-            type.type = TYPE_STRING;
+            type = TypeDescriptor(TYPE_STRING);
             break;
         default:
             break;
@@ -196,33 +199,31 @@ TypeDescriptor FieldDescriptor::get_doris_type(const tparquet::SchemaElement& ph
 TypeDescriptor FieldDescriptor::convert_to_doris_type(tparquet::LogicalType logicalType) {
     TypeDescriptor type;
     if (logicalType.__isset.STRING) {
-        type.type = TYPE_STRING;
+        type = TypeDescriptor(TYPE_STRING);
     } else if (logicalType.__isset.DECIMAL) {
-        type.type = TYPE_DECIMALV2;
-        type.precision = 27;
-        type.scale = 9;
+        type = TypeDescriptor(TYPE_DECIMALV2);
     } else if (logicalType.__isset.DATE) {
-        type.type = TYPE_DATEV2;
+        type = TypeDescriptor(TYPE_DATEV2);
     } else if (logicalType.__isset.INTEGER) {
         if (logicalType.INTEGER.isSigned) {
             if (logicalType.INTEGER.bitWidth <= 32) {
-                type.type = TYPE_INT;
+                type = TypeDescriptor(TYPE_INT);
             } else {
-                type.type = TYPE_BIGINT;
+                type = TypeDescriptor(TYPE_BIGINT);
             }
         } else {
             if (logicalType.INTEGER.bitWidth <= 16) {
-                type.type = TYPE_INT;
+                type = TypeDescriptor(TYPE_INT);
             } else {
-                type.type = TYPE_BIGINT;
+                type = TypeDescriptor(TYPE_BIGINT);
             }
         }
     } else if (logicalType.__isset.TIME) {
-        type.type = TYPE_TIMEV2;
+        type = TypeDescriptor(TYPE_TIMEV2);
     } else if (logicalType.__isset.TIMESTAMP) {
-        type.type = TYPE_DATETIMEV2;
+        type = TypeDescriptor(TYPE_DATETIMEV2);
     } else {
-        type.type = INVALID_TYPE;
+        type = TypeDescriptor(INVALID_TYPE);
     }
     return type;
 }
@@ -231,39 +232,41 @@ TypeDescriptor FieldDescriptor::convert_to_doris_type(tparquet::ConvertedType::t
     TypeDescriptor type;
     switch (convertedType) {
     case tparquet::ConvertedType::type::UTF8:
-        type.type = TYPE_STRING;
+        type = TypeDescriptor(TYPE_STRING);
         break;
     case tparquet::ConvertedType::type::DECIMAL:
-        type.type = TYPE_DECIMALV2;
-        type.precision = 27;
-        type.scale = 9;
+        type = TypeDescriptor(TYPE_DECIMALV2);
         break;
     case tparquet::ConvertedType::type::DATE:
-        type.type = TYPE_DATEV2;
+        type = TypeDescriptor(TYPE_DATEV2);
         break;
     case tparquet::ConvertedType::type::TIME_MILLIS:
     case tparquet::ConvertedType::type::TIME_MICROS:
-        type.type = TYPE_TIMEV2;
+        type = TypeDescriptor(TYPE_TIMEV2);
         break;
     case tparquet::ConvertedType::type::TIMESTAMP_MILLIS:
     case tparquet::ConvertedType::type::TIMESTAMP_MICROS:
-        type.type = TYPE_DATETIMEV2;
+        type = TypeDescriptor(TYPE_DATETIMEV2);
         break;
-    case tparquet::ConvertedType::type::UINT_8:
-    case tparquet::ConvertedType::type::UINT_16:
     case tparquet::ConvertedType::type::INT_8:
+        type = TypeDescriptor(TYPE_TINYINT);
+        break;
+    case tparquet::ConvertedType::type::UINT_8:
     case tparquet::ConvertedType::type::INT_16:
+        type = TypeDescriptor(TYPE_SMALLINT);
+        break;
+    case tparquet::ConvertedType::type::UINT_16:
     case tparquet::ConvertedType::type::INT_32:
-        type.type = TYPE_INT;
+        type = TypeDescriptor(TYPE_INT);
         break;
     case tparquet::ConvertedType::type::UINT_32:
     case tparquet::ConvertedType::type::UINT_64:
     case tparquet::ConvertedType::type::INT_64:
-        type.type = TYPE_BIGINT;
+        type = TypeDescriptor(TYPE_BIGINT);
         break;
     default:
         LOG(WARNING) << "Not supported parquet ConvertedType: " << convertedType;
-        type = INVALID_TYPE;
+        type = TypeDescriptor(INVALID_TYPE);
         break;
     }
     return type;


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


[doris] 03/05: [fix](olap) dictionary cannot be sorted after inserting some null values (#15829)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 915abc8c53414a6169f8018e4cac45cb38a1a9d1
Author: Jerry Hu <mr...@gmail.com>
AuthorDate: Fri Jan 13 09:28:55 2023 +0800

    [fix](olap) dictionary cannot be sorted after inserting some null values (#15829)
---
 be/src/vec/columns/column_dictionary.h             |  6 +++
 .../suites/query_p0/test_dict_with_null.groovy     | 46 ++++++++++++++++++++++
 2 files changed, 52 insertions(+)

diff --git a/be/src/vec/columns/column_dictionary.h b/be/src/vec/columns/column_dictionary.h
index 422e2fdfbb..95cd848763 100644
--- a/be/src/vec/columns/column_dictionary.h
+++ b/be/src/vec/columns/column_dictionary.h
@@ -240,6 +240,12 @@ public:
     }
 
     void convert_dict_codes_if_necessary() override {
+        // Avoid setting `_dict_sorted` to true when `_dict` is empty.
+        // Because `_dict` maybe keep empty after inserting some null rows.
+        if (_dict.empty()) {
+            return;
+        }
+
         if (!is_dict_sorted()) {
             _dict.sort();
             _dict_sorted = true;
diff --git a/regression-test/suites/query_p0/test_dict_with_null.groovy b/regression-test/suites/query_p0/test_dict_with_null.groovy
new file mode 100644
index 0000000000..a5c84444ae
--- /dev/null
+++ b/regression-test/suites/query_p0/test_dict_with_null.groovy
@@ -0,0 +1,46 @@
+// 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.
+
+suite("dict_with_null", "query") {
+    def tableName = "test_dict_with_null"
+    sql "DROP TABLE IF EXISTS ${tableName}"
+    sql """
+        CREATE TABLE IF NOT EXISTS ${tableName} (
+          c_int INT,
+          c_string VARCHAR(10)
+        )
+        DISTRIBUTED BY HASH(c_int) BUCKETS 1
+        PROPERTIES (
+          "replication_num" = "1"
+        )
+    """
+
+    // Here insert all rows in one statement to make sure they are in one same segment.
+    // Insert 100 + 1 rows because `SegmentIterator` will read 100 rows in the first time.
+    // The first 100 rows are all null to make no record be inserted into dictionary at the first read time.
+    def insert_sql = "insert into ${tableName} values "
+    for (int i in 1..100) {
+        if (i != 1) {
+            insert_sql += ", "
+        }
+        insert_sql += "(${i}, null)"
+    }
+    insert_sql += ", (101, 'abc')"
+
+    sql insert_sql
+    sql "select * from test_dict_with_null where c_string > '0'"
+}
\ No newline at end of file


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