You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by kx...@apache.org on 2023/06/27 16:50:32 UTC

[doris] branch branch-2.0 updated (932a38b4ec -> cd908f28ea)

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

kxiao pushed a change to branch branch-2.0
in repository https://gitbox.apache.org/repos/asf/doris.git


    from 932a38b4ec [Fix](inverted index) check inverted index file existence befor data compaction (#21173)
     new 650322f33d [Fix](invert index)fix s3 failed to check the directory (#21232)
     new af960a6d47 [fix](s3_writer) init member's value to avoid undefined behavior (#21233)
     new cff0918deb [fix](inverted index) update output rowset index meta with input rowset when drop inverted index (#21248)
     new 72615699c8 [improvement](scripts) change the arch name in build-for-release scripts (#21240)
     new cd908f28ea [improve](error) include detailed  messages in rowset reader init error (#21229)

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/io/fs/s3_file_write_bufferpool.h             |  4 ++--
 be/src/olap/olap_server.cpp                         |  5 ++---
 be/src/olap/rowset/beta_rowset_reader.cpp           |  4 ++--
 .../inverted_index_compound_directory.cpp           |  7 +++++--
 be/src/olap/tablet_schema.cpp                       | 11 +++++++++++
 be/src/olap/tablet_schema.h                         |  1 +
 be/src/olap/task/index_builder.cpp                  | 21 +++++++++++++++++----
 be/src/olap/task/index_builder.h                    |  2 --
 build-for-release.sh                                | 11 +++++++++++
 .../java/org/apache/doris/alter/IndexChangeJob.java |  2 +-
 .../apache/doris/task/AlterInvertedIndexTask.java   |  7 ++++++-
 11 files changed, 58 insertions(+), 17 deletions(-)


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


[doris] 05/05: [improve](error) include detailed messages in rowset reader init error (#21229)

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

kxiao pushed a commit to branch branch-2.0
in repository https://gitbox.apache.org/repos/asf/doris.git

commit cd908f28ea02e2b4887cd04b7e38ac3a9360af21
Author: Kaijie Chen <ck...@apache.org>
AuthorDate: Tue Jun 27 20:45:14 2023 +0800

    [improve](error) include detailed  messages in rowset reader init error (#21229)
---
 be/src/olap/rowset/beta_rowset_reader.cpp | 4 ++--
 be/src/olap/task/index_builder.cpp        | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/be/src/olap/rowset/beta_rowset_reader.cpp b/be/src/olap/rowset/beta_rowset_reader.cpp
index 5564c08740..3ec62de7a8 100644
--- a/be/src/olap/rowset/beta_rowset_reader.cpp
+++ b/be/src/olap/rowset/beta_rowset_reader.cpp
@@ -224,7 +224,7 @@ Status BetaRowsetReader::get_segment_iterators(RowsetReaderContext* read_context
         auto s = seg_ptr->new_iterator(_input_schema, _read_options, &iter);
         if (!s.ok()) {
             LOG(WARNING) << "failed to create iterator[" << seg_ptr->id() << "]: " << s.to_string();
-            return Status::Error<ROWSET_READER_INIT>();
+            return Status::Error<ROWSET_READER_INIT>(s.to_string());
         }
         if (iter->empty()) {
             continue;
@@ -268,7 +268,7 @@ Status BetaRowsetReader::init(RowsetReaderContext* read_context,
     if (!s.ok()) {
         LOG(WARNING) << "failed to init iterator: " << s.to_string();
         _iterator.reset();
-        return Status::Error<ROWSET_READER_INIT>();
+        return Status::Error<ROWSET_READER_INIT>(s.to_string());
     }
     return Status::OK();
 }
diff --git a/be/src/olap/task/index_builder.cpp b/be/src/olap/task/index_builder.cpp
index 4a82240474..4717dff9b3 100644
--- a/be/src/olap/task/index_builder.cpp
+++ b/be/src/olap/task/index_builder.cpp
@@ -189,7 +189,7 @@ Status IndexBuilder::handle_single_rowset(RowsetMetaSharedPtr output_rowset_meta
             if (!res.ok()) {
                 LOG(WARNING) << "failed to create iterator[" << seg_ptr->id()
                              << "]: " << res.to_string();
-                return Status::Error<ErrorCode::ROWSET_READER_INIT>();
+                return Status::Error<ErrorCode::ROWSET_READER_INIT>(res.to_string());
             }
 
             std::shared_ptr<vectorized::Block> block = std::make_shared<vectorized::Block>(


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


[doris] 03/05: [fix](inverted index) update output rowset index meta with input rowset when drop inverted index (#21248)

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

kxiao pushed a commit to branch branch-2.0
in repository https://gitbox.apache.org/repos/asf/doris.git

commit cff0918deb806a82cc3a1f6a683749f90b5ab410
Author: YueW <45...@users.noreply.github.com>
AuthorDate: Tue Jun 27 23:54:35 2023 +0800

    [fix](inverted index) update output rowset index meta with input rowset when drop inverted index (#21248)
---
 be/src/olap/olap_server.cpp                           |  5 ++---
 be/src/olap/tablet_schema.cpp                         | 11 +++++++++++
 be/src/olap/tablet_schema.h                           |  1 +
 be/src/olap/task/index_builder.cpp                    | 19 ++++++++++++++++---
 be/src/olap/task/index_builder.h                      |  2 --
 .../java/org/apache/doris/alter/IndexChangeJob.java   |  2 +-
 .../org/apache/doris/task/AlterInvertedIndexTask.java |  7 ++++++-
 7 files changed, 37 insertions(+), 10 deletions(-)

diff --git a/be/src/olap/olap_server.cpp b/be/src/olap/olap_server.cpp
index 57bf40a147..09875b9e52 100644
--- a/be/src/olap/olap_server.cpp
+++ b/be/src/olap/olap_server.cpp
@@ -1021,9 +1021,8 @@ Status StorageEngine::process_index_change_task(const TAlterInvertedIndexReq& re
         return Status::InternalError("tablet not exist, tablet_id={}.", tablet_id);
     }
 
-    IndexBuilderSharedPtr index_builder =
-            std::make_shared<IndexBuilder>(tablet, request.columns, request.indexes_desc,
-                                           request.alter_inverted_indexes, request.is_drop_op);
+    IndexBuilderSharedPtr index_builder = std::make_shared<IndexBuilder>(
+            tablet, request.columns, request.alter_inverted_indexes, request.is_drop_op);
     RETURN_IF_ERROR(_handle_index_change(index_builder));
     return Status::OK();
 }
diff --git a/be/src/olap/tablet_schema.cpp b/be/src/olap/tablet_schema.cpp
index f5f62e6857..414036ab58 100644
--- a/be/src/olap/tablet_schema.cpp
+++ b/be/src/olap/tablet_schema.cpp
@@ -637,6 +637,17 @@ void TabletSchema::append_index(TabletIndex index) {
     _indexes.push_back(std::move(index));
 }
 
+void TabletSchema::remove_index(int64_t index_id) {
+    std::vector<TabletIndex> indexes;
+    for (auto index : _indexes) {
+        if (index.index_id() == index_id) {
+            continue;
+        }
+        indexes.emplace_back(std::move(index));
+    }
+    _indexes = std::move(indexes);
+}
+
 void TabletSchema::clear_columns() {
     _field_name_to_index.clear();
     _field_id_to_index.clear();
diff --git a/be/src/olap/tablet_schema.h b/be/src/olap/tablet_schema.h
index ceadf76a0a..5a521f3610 100644
--- a/be/src/olap/tablet_schema.h
+++ b/be/src/olap/tablet_schema.h
@@ -207,6 +207,7 @@ public:
     void to_schema_pb(TabletSchemaPB* tablet_meta_pb) const;
     void append_column(TabletColumn column, bool is_dropped_column = false);
     void append_index(TabletIndex index);
+    void remove_index(int64_t index_id);
     // Must make sure the row column is always the last column
     void add_row_column();
     void copy_from(const TabletSchema& tablet_schema);
diff --git a/be/src/olap/task/index_builder.cpp b/be/src/olap/task/index_builder.cpp
index 05ba099f90..4a82240474 100644
--- a/be/src/olap/task/index_builder.cpp
+++ b/be/src/olap/task/index_builder.cpp
@@ -30,12 +30,10 @@
 namespace doris {
 
 IndexBuilder::IndexBuilder(const TabletSharedPtr& tablet, const std::vector<TColumn>& columns,
-                           const std::vector<TOlapTableIndex> exist_indexes,
                            const std::vector<doris::TOlapTableIndex>& alter_inverted_indexes,
                            bool is_drop_op)
         : _tablet(tablet),
           _columns(columns),
-          _exist_indexes(exist_indexes),
           _alter_inverted_indexes(alter_inverted_indexes),
           _is_drop_op(is_drop_op) {
     _olap_data_convertor = std::make_unique<vectorized::OlapBlockDataConvertor>();
@@ -63,8 +61,16 @@ Status IndexBuilder::update_inverted_index_info() {
         auto input_rs_tablet_schema = input_rowset->tablet_schema();
         output_rs_tablet_schema->copy_from(*input_rs_tablet_schema);
         if (_is_drop_op) {
-            output_rs_tablet_schema->update_indexes_from_thrift(_exist_indexes);
+            // base on input rowset's tablet_schema to build
+            // output rowset's tablet_schema which only remove
+            // the indexes specified in this drop index request
+            for (auto t_inverted_index : _alter_inverted_indexes) {
+                output_rs_tablet_schema->remove_index(t_inverted_index.index_id);
+            }
         } else {
+            // base on input rowset's tablet_schema to build
+            // output rowset's tablet_schema which only add
+            // the indexes specified in this build index request
             for (auto t_inverted_index : _alter_inverted_indexes) {
                 TabletIndex index;
                 index.init_from_thrift(t_inverted_index, *input_rs_tablet_schema);
@@ -427,6 +433,13 @@ Status IndexBuilder::do_build_inverted_index() {
 }
 
 Status IndexBuilder::modify_rowsets(const Merger::Statistics* stats) {
+    for (auto rowset_ptr : _output_rowsets) {
+        auto rowset_id = rowset_ptr->rowset_id();
+        if (StorageEngine::instance()->check_rowset_id_in_unused_rowsets(rowset_id)) {
+            DCHECK(false) << "output rowset: " << rowset_id.to_string() << " in unused rowsets";
+        }
+    }
+
     if (_tablet->keys_type() == KeysType::UNIQUE_KEYS &&
         _tablet->enable_unique_key_merge_on_write()) {
         std::lock_guard<std::mutex> rwlock(_tablet->get_rowset_update_lock());
diff --git a/be/src/olap/task/index_builder.h b/be/src/olap/task/index_builder.h
index 562cb1148d..9e406c22c1 100644
--- a/be/src/olap/task/index_builder.h
+++ b/be/src/olap/task/index_builder.h
@@ -36,7 +36,6 @@ using RowsetWriterUniquePtr = std::unique_ptr<RowsetWriter>;
 class IndexBuilder {
 public:
     IndexBuilder(const TabletSharedPtr& tablet, const std::vector<TColumn>& columns,
-                 const std::vector<TOlapTableIndex> exist_indexes,
                  const std::vector<doris::TOlapTableIndex>& alter_inverted_indexes,
                  bool is_drop_op = false);
     ~IndexBuilder();
@@ -65,7 +64,6 @@ private:
 private:
     TabletSharedPtr _tablet;
     std::vector<TColumn> _columns;
-    std::vector<TOlapTableIndex> _exist_indexes;
     std::vector<doris::TOlapTableIndex> _alter_inverted_indexes;
     bool _is_drop_op;
     std::unordered_map<std::string, std::set<int32_t>> _rowset_alter_index_column_ids;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java
index fecfe4b99b..6ba3e68ffa 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java
@@ -290,7 +290,7 @@ public class IndexChangeJob implements Writable {
                             partitionId, originIndexId, originTabletId,
                             originSchemaHash, olapTable.getIndexes(),
                             alterInvertedIndexes, originSchemaColumns,
-                            isDropOp, taskSignature);
+                            isDropOp, taskSignature, jobId);
                     invertedIndexBatchTask.addTask(alterInvertedIndexTask);
                 }
             } // end for tablet
diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/AlterInvertedIndexTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/AlterInvertedIndexTask.java
index c199d1d482..caf7733165 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/task/AlterInvertedIndexTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/task/AlterInvertedIndexTask.java
@@ -44,11 +44,13 @@ public class AlterInvertedIndexTask extends AgentTask {
     private List<Column> schemaColumns;
     private List<Index> existIndexes;
     private boolean isDropOp = false;
+    private long jobId;
 
     public AlterInvertedIndexTask(long backendId, long dbId, long tableId,
             long partitionId, long indexId, long tabletId, int schemaHash,
             List<Index> existIndexes, List<Index> alterInvertedIndexes,
-            List<Column> schemaColumns, boolean isDropOp, long taskSignature) {
+            List<Column> schemaColumns, boolean isDropOp, long taskSignature,
+            long jobId) {
         super(null, backendId, TTaskType.ALTER_INVERTED_INDEX, dbId, tableId,
                 partitionId, indexId, tabletId, taskSignature);
         this.tabletId = tabletId;
@@ -57,6 +59,7 @@ public class AlterInvertedIndexTask extends AgentTask {
         this.alterInvertedIndexes = alterInvertedIndexes;
         this.schemaColumns = schemaColumns;
         this.isDropOp = isDropOp;
+        this.jobId = jobId;
     }
 
     public long getTabletId() {
@@ -94,6 +97,8 @@ public class AlterInvertedIndexTask extends AgentTask {
         req.setTabletId(tabletId);
         req.setSchemaHash(schemaHash);
         req.setIsDropOp(isDropOp);
+        // set jonId for debugging in BE
+        req.setJobId(jobId);
 
         if (!alterInvertedIndexes.isEmpty()) {
             List<TOlapTableIndex> tIndexes = new ArrayList<>();


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


[doris] 04/05: [improvement](scripts) change the arch name in build-for-release scripts (#21240)

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

kxiao pushed a commit to branch branch-2.0
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 72615699c8accfffde5961f1f16fe2645a043cc3
Author: Mingyu Chen <mo...@163.com>
AuthorDate: Tue Jun 27 18:11:17 2023 +0800

    [improvement](scripts) change the arch name in build-for-release scripts (#21240)
---
 build-for-release.sh | 11 +++++++++++
 1 file changed, 11 insertions(+)

diff --git a/build-for-release.sh b/build-for-release.sh
index 10046bf98d..83b430215c 100755
--- a/build-for-release.sh
+++ b/build-for-release.sh
@@ -110,6 +110,17 @@ echo "Get params:
 
 ARCH="$(uname -m)"
 
+if [[ "${ARCH}" == "aarch64" ]]; then
+    ARCH="arm64"
+elif [[ "${ARCH}" == "x86_64" ]]; then
+    ARCH="x64"
+else
+    echo "Unknown arch: ${ARCH}"
+    exit 1
+fi
+
+echo "ARCH: ${ARCH}"
+
 ORI_OUTPUT="${ROOT}/output"
 
 FE="fe"


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


[doris] 01/05: [Fix](invert index)fix s3 failed to check the directory (#21232)

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

kxiao pushed a commit to branch branch-2.0
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 650322f33d0362244d3060914597fb00e2f57bc9
Author: zzzxl <33...@users.noreply.github.com>
AuthorDate: Tue Jun 27 20:01:46 2023 +0800

    [Fix](invert index)fix s3 failed to check the directory (#21232)
---
 .../olap/rowset/segment_v2/inverted_index_compound_directory.cpp   | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)

diff --git a/be/src/olap/rowset/segment_v2/inverted_index_compound_directory.cpp b/be/src/olap/rowset/segment_v2/inverted_index_compound_directory.cpp
index be19be580a..83660cca66 100644
--- a/be/src/olap/rowset/segment_v2/inverted_index_compound_directory.cpp
+++ b/be/src/olap/rowset/segment_v2/inverted_index_compound_directory.cpp
@@ -467,7 +467,6 @@ void DorisCompoundDirectory::init(const io::FileSystemSPtr& _fs, const char* _pa
 
     if (lock_factory == nullptr) {
         lock_factory = _CLNEW lucene::store::NoLockFactory();
-        fs->create_directory(directory);
     }
 
     setLockFactory(lock_factory);
@@ -476,6 +475,10 @@ void DorisCompoundDirectory::init(const io::FileSystemSPtr& _fs, const char* _pa
         lockFactory->setLockPrefix(nullptr);
     }
 
+    // It's fail checking directory existence in S3.
+    if (fs->type() == io::FileSystemType::S3) {
+        return;
+    }
     bool exists = false;
     Status status = fs->exists(directory, &exists);
     if (!status.ok()) {
@@ -579,7 +582,7 @@ DorisCompoundDirectory* DorisCompoundDirectory::getDirectory(
     bool exists = false;
     _fs->exists(file, &exists);
     if (!exists) {
-        mkdir(file, 0777);
+        _fs->create_directory(file);
     }
 
     dir = _CLNEW DorisCompoundDirectory();


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


[doris] 02/05: [fix](s3_writer) init member's value to avoid undefined behavior (#21233)

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

kxiao pushed a commit to branch branch-2.0
in repository https://gitbox.apache.org/repos/asf/doris.git

commit af960a6d4792701c70c5a187dd2035e35f93df66
Author: Mingyu Chen <mo...@163.com>
AuthorDate: Tue Jun 27 20:01:20 2023 +0800

    [fix](s3_writer) init member's value to avoid undefined behavior (#21233)
---
 be/src/io/fs/s3_file_write_bufferpool.h | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/be/src/io/fs/s3_file_write_bufferpool.h b/be/src/io/fs/s3_file_write_bufferpool.h
index b69964b48e..660cbc8e8a 100644
--- a/be/src/io/fs/s3_file_write_bufferpool.h
+++ b/be/src/io/fs/s3_file_write_bufferpool.h
@@ -104,8 +104,8 @@ struct S3FileBuffer : public std::enable_shared_from_this<S3FileBuffer> {
     // caller of this buf could use this callback to do syncronization
     Callback _on_finish_upload = nullptr;
     Status _status;
-    size_t _offset;
-    size_t _size;
+    size_t _offset {0};
+    size_t _size {0};
     std::shared_ptr<std::iostream> _stream_ptr;
     // only served as one reserved buffer
     Slice _buf;


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