You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by zh...@apache.org on 2023/04/03 08:11:28 UTC

[doris] branch branch-1.2-lts updated (54228425b3 -> f4456fc0e9)

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

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


    from 54228425b3 [cherry-pick](query exec) should print error status when query meet error #18247
     new 058fb4e420 [bugfix](publish) fix TabletLoadInfo may released by delete txn (#17986)
     new 4c22234520 [fix](merge-on-write) fix that missed rows don't match merged rows for base compaction (#18262)
     new f4456fc0e9 [fix](load) handle null map right in OlapDataConvertor (#18236)

The 3 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/compaction.cpp                       | 10 ++++++----
 be/src/olap/task/engine_publish_version_task.cpp | 10 +++++++---
 be/src/olap/txn_manager.cpp                      | 10 +++++-----
 be/src/vec/olap/olap_data_convertor.cpp          | 17 +++++++++--------
 be/src/vec/olap/olap_data_convertor.h            | 16 ++++++++--------
 5 files changed, 35 insertions(+), 28 deletions(-)


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


[doris] 02/03: [fix](merge-on-write) fix that missed rows don't match merged rows for base compaction (#18262)

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

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

commit 4c22234520fd0418a17158c78105f38cbaa1a177
Author: Xin Liao <li...@126.com>
AuthorDate: Fri Mar 31 15:06:51 2023 +0800

    [fix](merge-on-write) fix that missed rows don't match merged rows for base compaction (#18262)
---
 be/src/olap/compaction.cpp | 10 ++++++----
 1 file changed, 6 insertions(+), 4 deletions(-)

diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp
index 48ef55c347..c154667e6e 100644
--- a/be/src/olap/compaction.cpp
+++ b/be/src/olap/compaction.cpp
@@ -473,10 +473,12 @@ Status Compaction::modify_rowsets(const Merger::Statistics* stats) {
             _tablet->calc_compaction_output_rowset_delete_bitmap(
                     _input_rowsets, _rowid_conversion, version.second, UINT64_MAX, &missed_rows,
                     &location_map, &output_rowset_delete_bitmap);
-            DCHECK_EQ(missed_rows.size(), missed_rows_size);
-            if (missed_rows.size() != missed_rows_size) {
-                LOG(WARNING) << "missed rows don't match, before: " << missed_rows_size
-                             << " after: " << missed_rows.size();
+            if (compaction_type() == READER_CUMULATIVE_COMPACTION) {
+                DCHECK_EQ(missed_rows.size(), missed_rows_size);
+                if (missed_rows.size() != missed_rows_size) {
+                    LOG(WARNING) << "missed rows don't match, before: " << missed_rows_size
+                                 << " after: " << missed_rows.size();
+                }
             }
 
             RETURN_IF_ERROR(_tablet->check_rowid_conversion(_output_rowset, location_map));


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


[doris] 03/03: [fix](load) handle null map right in OlapDataConvertor (#18236)

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

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

commit f4456fc0e91ba8dcb757db5f23238d87f7ee909b
Author: Xin Liao <li...@126.com>
AuthorDate: Mon Apr 3 09:14:05 2023 +0800

    [fix](load) handle null map right in OlapDataConvertor (#18236)
    
    The offset of _nullmap and _value are inconsistent in OlapDataConvertor, so the obtained null flag is incorrect when calling get_ data_ at function. When the key column or sequence column has null values, the encoding of the short key index or primary key index may be wrong.
    This was introduced by #10883 #10925.
---
 be/src/vec/olap/olap_data_convertor.cpp | 17 +++++++++--------
 be/src/vec/olap/olap_data_convertor.h   | 16 ++++++++--------
 2 files changed, 17 insertions(+), 16 deletions(-)

diff --git a/be/src/vec/olap/olap_data_convertor.cpp b/be/src/vec/olap/olap_data_convertor.cpp
index e9b4c8fd24..476f20c9a1 100644
--- a/be/src/vec/olap/olap_data_convertor.cpp
+++ b/be/src/vec/olap/olap_data_convertor.cpp
@@ -172,8 +172,9 @@ void OlapBlockDataConvertor::OlapColumnDataConvertorBase::clear_source_column()
     _nullmap = nullptr;
 }
 
-// This should be called only in SegmentWriter. If you want to access nullmap in Convertor,
-// use `_nullmap` directly.
+// Obtain the converted nullmap with an offset of _row_pos.
+// This should be called only in SegmentWriter and `get_data_at` in Convertor.
+// If you want to access origin nullmap without offset, use `_nullmap` directly.
 const UInt8* OlapBlockDataConvertor::OlapColumnDataConvertorBase::get_nullmap() const {
     assert(_typed_column.column);
     return _nullmap ? _nullmap + _row_pos : nullptr;
@@ -195,8 +196,8 @@ const void* OlapBlockDataConvertor::OlapColumnDataConvertorObject::get_data() co
 const void* OlapBlockDataConvertor::OlapColumnDataConvertorObject::get_data_at(
         size_t offset) const {
     UInt8 null_flag = 0;
-    if (_nullmap) {
-        null_flag = _nullmap[offset];
+    if (get_nullmap()) {
+        null_flag = get_nullmap()[offset];
     }
     return null_flag ? nullptr : _slice.data() + offset;
 }
@@ -372,8 +373,8 @@ const void* OlapBlockDataConvertor::OlapColumnDataConvertorChar::get_data() cons
 
 const void* OlapBlockDataConvertor::OlapColumnDataConvertorChar::get_data_at(size_t offset) const {
     UInt8 null_flag = 0;
-    if (_nullmap) {
-        null_flag = _nullmap[offset];
+    if (get_nullmap()) {
+        null_flag = get_nullmap()[offset];
     }
     return null_flag ? nullptr : _slice.data() + offset;
 }
@@ -428,8 +429,8 @@ const void* OlapBlockDataConvertor::OlapColumnDataConvertorVarChar::get_data_at(
         size_t offset) const {
     assert(offset < _slice.size());
     UInt8 null_flag = 0;
-    if (_nullmap) {
-        null_flag = _nullmap[offset];
+    if (get_nullmap()) {
+        null_flag = get_nullmap()[offset];
     }
     return null_flag ? nullptr : _slice.data() + offset;
 }
diff --git a/be/src/vec/olap/olap_data_convertor.h b/be/src/vec/olap/olap_data_convertor.h
index 6898b44a9d..037a1f4a70 100644
--- a/be/src/vec/olap/olap_data_convertor.h
+++ b/be/src/vec/olap/olap_data_convertor.h
@@ -179,8 +179,8 @@ private:
         const void* get_data() const override { return _values.data(); }
         const void* get_data_at(size_t offset) const override {
             UInt8 null_flag = 0;
-            if (_nullmap) {
-                null_flag = _nullmap[offset];
+            if (get_nullmap()) {
+                null_flag = get_nullmap()[offset];
             }
             return null_flag ? nullptr : _values.data() + offset;
         }
@@ -221,8 +221,8 @@ private:
         const void* get_data_at(size_t offset) const override {
             assert(offset < _num_rows);
             UInt8 null_flag = 0;
-            if (_nullmap) {
-                null_flag = _nullmap[offset];
+            if (get_nullmap()) {
+                null_flag = get_nullmap()[offset];
             }
             return null_flag ? nullptr : _values + offset;
         }
@@ -263,8 +263,8 @@ private:
         const void* get_data_at(size_t offset) const override {
             assert(offset < _num_rows);
             UInt8 null_flag = 0;
-            if (_nullmap) {
-                null_flag = _nullmap[offset];
+            if (get_nullmap()) {
+                null_flag = get_nullmap()[offset];
             }
             return null_flag ? nullptr : values_ + offset;
         }
@@ -305,8 +305,8 @@ private:
         const void* get_data_at(size_t offset) const override {
             assert(offset < _num_rows);
             UInt8 null_flag = 0;
-            if (_nullmap) {
-                null_flag = _nullmap[offset];
+            if (get_nullmap()) {
+                null_flag = get_nullmap()[offset];
             }
             return null_flag ? nullptr : values_ + offset;
         }


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


[doris] 01/03: [bugfix](publish) fix TabletLoadInfo may released by delete txn (#17986)

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

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

commit 058fb4e420cf38034d421b55811347a3e739c153
Author: yixiutt <10...@users.noreply.github.com>
AuthorDate: Fri Mar 24 20:14:34 2023 +0800

    [bugfix](publish) fix TabletLoadInfo may released by delete txn (#17986)
---
 be/src/olap/task/engine_publish_version_task.cpp | 10 +++++++---
 be/src/olap/txn_manager.cpp                      | 10 +++++-----
 2 files changed, 12 insertions(+), 8 deletions(-)

diff --git a/be/src/olap/task/engine_publish_version_task.cpp b/be/src/olap/task/engine_publish_version_task.cpp
index 0422912944..f8847c173c 100644
--- a/be/src/olap/task/engine_publish_version_task.cpp
+++ b/be/src/olap/task/engine_publish_version_task.cpp
@@ -183,9 +183,13 @@ Status EnginePublishVersionTask::finish() {
         }
     }
 
-    LOG(INFO) << "finish to publish version on transaction."
-              << "transaction_id=" << transaction_id << ", cost(us): " << watch.get_elapse_time_us()
-              << ", error_tablet_size=" << _error_tablet_ids->size() << ", res=" << res.to_string();
+    if (!res.is<PUBLISH_VERSION_NOT_CONTINUOUS>()) {
+        LOG(INFO) << "finish to publish version on transaction."
+                  << "transaction_id=" << transaction_id
+                  << ", cost(us): " << watch.get_elapse_time_us()
+                  << ", error_tablet_size=" << _error_tablet_ids->size()
+                  << ", res=" << res.to_string();
+    }
     return res;
 }
 
diff --git a/be/src/olap/txn_manager.cpp b/be/src/olap/txn_manager.cpp
index e953ed87e2..474b4c8936 100644
--- a/be/src/olap/txn_manager.cpp
+++ b/be/src/olap/txn_manager.cpp
@@ -298,7 +298,7 @@ Status TxnManager::publish_txn(OlapMeta* meta, TPartitionId partition_id,
     pair<int64_t, int64_t> key(partition_id, transaction_id);
     TabletInfo tablet_info(tablet_id, schema_hash, tablet_uid);
     RowsetSharedPtr rowset_ptr = nullptr;
-    TabletTxnInfo* load_info = nullptr;
+    TabletTxnInfo load_info;
     {
         {
             std::unique_lock<std::mutex> txn_rlock(_get_txn_lock(transaction_id));
@@ -310,8 +310,8 @@ Status TxnManager::publish_txn(OlapMeta* meta, TPartitionId partition_id,
                 if (load_itr != it->second.end()) {
                     // found load for txn,tablet
                     // case 1: user commit rowset, then the load id must be equal
-                    load_info = &load_itr->second;
-                    rowset_ptr = load_info->rowset;
+                    load_info = load_itr->second;
+                    rowset_ptr = load_info.rowset;
                 }
             }
         }
@@ -323,13 +323,13 @@ Status TxnManager::publish_txn(OlapMeta* meta, TPartitionId partition_id,
             rowset_ptr->make_visible(version);
             // update delete_bitmap
             {
-                if (load_info != nullptr && load_info->unique_key_merge_on_write) {
+                if (load_info.unique_key_merge_on_write) {
                     auto tablet =
                             StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id);
                     if (tablet == nullptr) {
                         return Status::OK();
                     }
-                    RETURN_IF_ERROR(tablet->update_delete_bitmap(rowset_ptr, load_info));
+                    RETURN_IF_ERROR(tablet->update_delete_bitmap(rowset_ptr, &load_info));
                     std::shared_lock rlock(tablet->get_header_lock());
                     tablet->save_meta();
                 }


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