github-actions[bot] commented on code in PR #24929:
URL: https://github.com/apache/doris/pull/24929#discussion_r1362104834


##########
be/src/olap/base_tablet.cpp:
##########
@@ -62,26 +53,16 @@ Status BaseTablet::set_tablet_state(TabletState state) {
                 "could not change tablet state from shutdown to {}", state);
     }
     _tablet_meta->set_tablet_state(state);
-    _state = state;
     return Status::OK();
 }
 
-void BaseTablet::_gen_tablet_path() {
-    if (_data_dir != nullptr && _tablet_meta != nullptr) {
-        _tablet_path = fmt::format("{}/{}/{}/{}/{}", _data_dir->path(), 
DATA_PREFIX, shard_id(),
-                                   tablet_id(), schema_hash());
-    }
-}
-
-bool BaseTablet::set_tablet_schema_into_rowset_meta() {
-    bool flag = false;
-    for (RowsetMetaSharedPtr rowset_meta : 
_tablet_meta->all_mutable_rs_metas()) {
-        if (!rowset_meta->tablet_schema()) {
-            rowset_meta->set_tablet_schema(_schema);
-            flag = true;
-        }
+void BaseTablet::update_max_version_schema(const TabletSchemaSPtr& 
tablet_schema) {

Review Comment:
   warning: method 'update_max_version_schema' can be made static 
[readability-convert-member-functions-to-static]
   
   be/src/olap/base_tablet.h:65:
   ```diff
   -     void update_max_version_schema(const TabletSchemaSPtr& tablet_schema);
   +     static void update_max_version_schema(const TabletSchemaSPtr& 
tablet_schema);
   ```
   



##########
be/src/olap/rowset/rowset_meta.cpp:
##########
@@ -0,0 +1,154 @@
+// 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 "olap/rowset/rowset_meta.h"
+
+#include "common/logging.h"
+#include "google/protobuf/util/message_differencer.h"
+#include "io/fs/local_file_system.h"
+#include "json2pb/json_to_pb.h"
+#include "json2pb/pb_to_json.h"
+#include "olap/olap_common.h"
+#include "olap/storage_policy.h"
+#include "olap/tablet_schema.h"
+#include "olap/tablet_schema_cache.h"
+
+namespace doris {
+
+RowsetMeta::~RowsetMeta() = default;
+
+bool RowsetMeta::init(const std::string& pb_rowset_meta) {
+    bool ret = _deserialize_from_pb(pb_rowset_meta);
+    if (!ret) {
+        return false;
+    }
+    _init();
+    return true;
+}
+
+bool RowsetMeta::init_from_pb(const RowsetMetaPB& rowset_meta_pb) {
+    if (rowset_meta_pb.has_tablet_schema()) {
+        _schema = TabletSchemaCache::instance()->insert(
+                rowset_meta_pb.tablet_schema().SerializeAsString());
+    }
+    // Release ownership of TabletSchemaPB from `rowset_meta_pb` and then set 
it back to `rowset_meta_pb`,
+    // this won't break const semantics of `rowset_meta_pb`, because 
`rowset_meta_pb` is not changed
+    // before and after call this method.
+    auto& mut_rowset_meta_pb = const_cast<RowsetMetaPB&>(rowset_meta_pb);
+    auto schema = mut_rowset_meta_pb.release_tablet_schema();
+    _rowset_meta_pb = mut_rowset_meta_pb;
+    mut_rowset_meta_pb.set_allocated_tablet_schema(schema);
+    _init();
+    return true;
+}
+
+bool RowsetMeta::init_from_json(const std::string& json_rowset_meta) {
+    bool ret = json2pb::JsonToProtoMessage(json_rowset_meta, &_rowset_meta_pb);
+    if (!ret) {
+        return false;
+    }
+    _init();
+    return true;
+}
+
+bool RowsetMeta::json_rowset_meta(std::string* json_rowset_meta) {

Review Comment:
   warning: method 'json_rowset_meta' can be made static 
[readability-convert-member-functions-to-static]
   
   ```suggestion
   static bool RowsetMeta::json_rowset_meta(std::string* json_rowset_meta) {
   ```
   



##########
be/src/olap/base_tablet.h:
##########
@@ -17,131 +17,86 @@
 
 #pragma once
 
-#include <butil/macros.h>
-#include <gen_cpp/olap_file.pb.h>
-#include <stdint.h>
-
 #include <memory>
+#include <shared_mutex>
 #include <string>
 
 #include "common/status.h"
-#include "olap/olap_common.h"
+#include "olap/tablet_fwd.h"
 #include "olap/tablet_meta.h"
-#include "olap/tablet_schema.h"
 #include "util/metrics.h"
 
 namespace doris {
+struct RowSetSplits;
+struct RowsetWriterContext;
+class RowsetWriter;
 
-class DataDir;
-
-// Base class for all tablet classes, currently only olap/Tablet
-// The fields and methods in this class is not final, it will change as memory
-// storage engine evolves.
+// Base class for all tablet classes
 class BaseTablet {
 public:
-    BaseTablet(TabletMetaSharedPtr tablet_meta, DataDir* data_dir);
+    explicit BaseTablet(TabletMetaSharedPtr tablet_meta);
     virtual ~BaseTablet();
+    BaseTablet(const BaseTablet&) = delete;
+    BaseTablet& operator=(const BaseTablet&) = delete;
 
-    DataDir* data_dir() const;
-    const std::string& tablet_path() const;
-
-    TabletState tablet_state() const { return _state; }
+    const std::string& tablet_path() const { return _tablet_path; }
+    TabletState tablet_state() const { return _tablet_meta->tablet_state(); }
     Status set_tablet_state(TabletState state);
+    int64_t table_id() const { return _tablet_meta->table_id(); }
+    int64_t partition_id() const { return _tablet_meta->partition_id(); }
+    int64_t tablet_id() const { return _tablet_meta->tablet_id(); }
+    int32_t schema_hash() const { return _tablet_meta->schema_hash(); }
+    KeysType keys_type() const { return 
_tablet_meta->tablet_schema()->keys_type(); }
+    size_t num_key_columns() const { return 
_tablet_meta->tablet_schema()->num_key_columns(); }
+    bool enable_unique_key_merge_on_write() const {
+#ifdef BE_TEST
+        if (_tablet_meta == nullptr) {
+            return false;
+        }
+#endif
+        return _tablet_meta->enable_unique_key_merge_on_write();
+    }
 
     // Property encapsulated in TabletMeta
-    const TabletMetaSharedPtr& tablet_meta();
+    const TabletMetaSharedPtr& tablet_meta() { return _tablet_meta; }
 
-    TabletUid tablet_uid() const;
-    int64_t table_id() const;
-    // Returns a string can be used to uniquely identify a tablet.
-    // The result string will often be printed to the log.
-    const std::string full_name() const;
-    int64_t partition_id() const;
-    int64_t tablet_id() const;
-    int64_t replica_id() const;
-    int32_t schema_hash() const;
-    int16_t shard_id() const;
+    // FIXME(plat1ko): It is not appropriate to expose this lock
+    std::shared_mutex& get_header_lock() { return _meta_lock; }
 
-    int64_t storage_policy_id() const { return 
_tablet_meta->storage_policy_id(); }
+    void update_max_version_schema(const TabletSchemaSPtr& tablet_schema);
 
-    void set_storage_policy_id(int64_t id) { 
_tablet_meta->set_storage_policy_id(id); }
+    TabletSchemaSPtr tablet_schema() const {

Review Comment:
   warning: method 'tablet_schema' can be made static 
[readability-convert-member-functions-to-static]
   
   ```suggestion
       static TabletSchemaSPtr tablet_schema() {
   ```
   



##########
be/src/olap/rowset/rowset_meta.cpp:
##########
@@ -0,0 +1,154 @@
+// 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 "olap/rowset/rowset_meta.h"
+
+#include "common/logging.h"
+#include "google/protobuf/util/message_differencer.h"
+#include "io/fs/local_file_system.h"
+#include "json2pb/json_to_pb.h"
+#include "json2pb/pb_to_json.h"
+#include "olap/olap_common.h"
+#include "olap/storage_policy.h"
+#include "olap/tablet_schema.h"
+#include "olap/tablet_schema_cache.h"
+
+namespace doris {
+
+RowsetMeta::~RowsetMeta() = default;
+
+bool RowsetMeta::init(const std::string& pb_rowset_meta) {
+    bool ret = _deserialize_from_pb(pb_rowset_meta);
+    if (!ret) {
+        return false;
+    }
+    _init();
+    return true;
+}
+
+bool RowsetMeta::init_from_pb(const RowsetMetaPB& rowset_meta_pb) {
+    if (rowset_meta_pb.has_tablet_schema()) {
+        _schema = TabletSchemaCache::instance()->insert(
+                rowset_meta_pb.tablet_schema().SerializeAsString());
+    }
+    // Release ownership of TabletSchemaPB from `rowset_meta_pb` and then set 
it back to `rowset_meta_pb`,
+    // this won't break const semantics of `rowset_meta_pb`, because 
`rowset_meta_pb` is not changed
+    // before and after call this method.
+    auto& mut_rowset_meta_pb = const_cast<RowsetMetaPB&>(rowset_meta_pb);
+    auto schema = mut_rowset_meta_pb.release_tablet_schema();
+    _rowset_meta_pb = mut_rowset_meta_pb;
+    mut_rowset_meta_pb.set_allocated_tablet_schema(schema);
+    _init();
+    return true;
+}
+
+bool RowsetMeta::init_from_json(const std::string& json_rowset_meta) {
+    bool ret = json2pb::JsonToProtoMessage(json_rowset_meta, &_rowset_meta_pb);
+    if (!ret) {
+        return false;
+    }
+    _init();
+    return true;
+}
+
+bool RowsetMeta::json_rowset_meta(std::string* json_rowset_meta) {
+    json2pb::Pb2JsonOptions json_options;
+    json_options.pretty_json = true;
+    bool ret = json2pb::ProtoMessageToJson(_rowset_meta_pb, json_rowset_meta, 
json_options);
+    return ret;
+}
+
+const io::FileSystemSPtr& RowsetMeta::fs() {
+    if (!_fs) {
+        if (is_local()) {
+            _fs = io::global_local_filesystem();
+        } else {
+            _fs = get_filesystem(resource_id());
+            LOG_IF(WARNING, !_fs) << "Cannot get file system: " << 
resource_id();
+        }
+    }
+    return _fs;
+}
+
+void RowsetMeta::set_fs(io::FileSystemSPtr fs) {

Review Comment:
   warning: method 'set_fs' can be made static 
[readability-convert-member-functions-to-static]
   
   be/src/olap/rowset/rowset_meta.h:50:
   ```diff
   -     void set_fs(io::FileSystemSPtr fs);
   +     static void set_fs(io::FileSystemSPtr fs);
   ```
   



##########
be/src/cloud/config.cpp:
##########
@@ -0,0 +1,26 @@
+// 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 "cloud/config.h"
+
+namespace doris {
+namespace config {

Review Comment:
   warning: nested namespaces can be concatenated 
[modernize-concat-nested-namespaces]
   
   ```suggestion
   namespace doris::config {
   ```
   
   be/src/cloud/config.cpp:24:
   ```diff
   - } // namespace config
   - } // namespace doris
   + } // namespace doris
   ```
   



##########
be/src/olap/rowset/rowset_meta.cpp:
##########
@@ -0,0 +1,154 @@
+// 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 "olap/rowset/rowset_meta.h"
+
+#include "common/logging.h"
+#include "google/protobuf/util/message_differencer.h"
+#include "io/fs/local_file_system.h"
+#include "json2pb/json_to_pb.h"
+#include "json2pb/pb_to_json.h"
+#include "olap/olap_common.h"
+#include "olap/storage_policy.h"
+#include "olap/tablet_schema.h"
+#include "olap/tablet_schema_cache.h"
+
+namespace doris {
+
+RowsetMeta::~RowsetMeta() = default;
+
+bool RowsetMeta::init(const std::string& pb_rowset_meta) {
+    bool ret = _deserialize_from_pb(pb_rowset_meta);
+    if (!ret) {
+        return false;
+    }
+    _init();
+    return true;
+}
+
+bool RowsetMeta::init_from_pb(const RowsetMetaPB& rowset_meta_pb) {
+    if (rowset_meta_pb.has_tablet_schema()) {
+        _schema = TabletSchemaCache::instance()->insert(
+                rowset_meta_pb.tablet_schema().SerializeAsString());
+    }
+    // Release ownership of TabletSchemaPB from `rowset_meta_pb` and then set 
it back to `rowset_meta_pb`,
+    // this won't break const semantics of `rowset_meta_pb`, because 
`rowset_meta_pb` is not changed
+    // before and after call this method.
+    auto& mut_rowset_meta_pb = const_cast<RowsetMetaPB&>(rowset_meta_pb);
+    auto schema = mut_rowset_meta_pb.release_tablet_schema();
+    _rowset_meta_pb = mut_rowset_meta_pb;
+    mut_rowset_meta_pb.set_allocated_tablet_schema(schema);
+    _init();
+    return true;
+}
+
+bool RowsetMeta::init_from_json(const std::string& json_rowset_meta) {
+    bool ret = json2pb::JsonToProtoMessage(json_rowset_meta, &_rowset_meta_pb);
+    if (!ret) {
+        return false;
+    }
+    _init();
+    return true;
+}
+
+bool RowsetMeta::json_rowset_meta(std::string* json_rowset_meta) {
+    json2pb::Pb2JsonOptions json_options;
+    json_options.pretty_json = true;
+    bool ret = json2pb::ProtoMessageToJson(_rowset_meta_pb, json_rowset_meta, 
json_options);
+    return ret;
+}
+
+const io::FileSystemSPtr& RowsetMeta::fs() {
+    if (!_fs) {
+        if (is_local()) {
+            _fs = io::global_local_filesystem();
+        } else {
+            _fs = get_filesystem(resource_id());
+            LOG_IF(WARNING, !_fs) << "Cannot get file system: " << 
resource_id();
+        }
+    }
+    return _fs;
+}
+
+void RowsetMeta::set_fs(io::FileSystemSPtr fs) {
+    if (fs && fs->type() != io::FileSystemType::LOCAL) {
+        _rowset_meta_pb.set_resource_id(fs->id());
+    }
+    _fs = std::move(fs);
+}
+
+void RowsetMeta::to_rowset_pb(RowsetMetaPB* rs_meta_pb) const {
+    *rs_meta_pb = _rowset_meta_pb;
+    if (_schema) {
+        _schema->to_schema_pb(rs_meta_pb->mutable_tablet_schema());
+    }
+}
+
+RowsetMetaPB RowsetMeta::get_rowset_pb() {
+    RowsetMetaPB rowset_meta_pb = _rowset_meta_pb;
+    if (_schema) {
+        _schema->to_schema_pb(rowset_meta_pb.mutable_tablet_schema());
+    }
+    return rowset_meta_pb;
+}
+
+void RowsetMeta::set_tablet_schema(const TabletSchemaSPtr& tablet_schema) {
+    _schema = TabletSchemaCache::instance()->insert(tablet_schema->to_key());
+}
+
+bool RowsetMeta::_deserialize_from_pb(const std::string& value) {
+    RowsetMetaPB rowset_meta_pb;
+    if (!rowset_meta_pb.ParseFromString(value)) {
+        return false;
+    }
+    if (rowset_meta_pb.has_tablet_schema()) {
+        _schema = TabletSchemaCache::instance()->insert(
+                rowset_meta_pb.tablet_schema().SerializeAsString());
+        rowset_meta_pb.clear_tablet_schema();
+    }
+    _rowset_meta_pb = rowset_meta_pb;
+    return true;
+}
+
+bool RowsetMeta::_serialize_to_pb(std::string* value) {

Review Comment:
   warning: method '_serialize_to_pb' can be made static 
[readability-convert-member-functions-to-static]
   
   ```suggestion
   static bool RowsetMeta::_serialize_to_pb(std::string* value) {
   ```
   



##########
be/src/olap/rowset/rowset_meta.cpp:
##########
@@ -0,0 +1,154 @@
+// 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 "olap/rowset/rowset_meta.h"
+
+#include "common/logging.h"
+#include "google/protobuf/util/message_differencer.h"
+#include "io/fs/local_file_system.h"
+#include "json2pb/json_to_pb.h"
+#include "json2pb/pb_to_json.h"
+#include "olap/olap_common.h"
+#include "olap/storage_policy.h"
+#include "olap/tablet_schema.h"
+#include "olap/tablet_schema_cache.h"
+
+namespace doris {
+
+RowsetMeta::~RowsetMeta() = default;
+
+bool RowsetMeta::init(const std::string& pb_rowset_meta) {
+    bool ret = _deserialize_from_pb(pb_rowset_meta);
+    if (!ret) {
+        return false;
+    }
+    _init();
+    return true;
+}
+
+bool RowsetMeta::init_from_pb(const RowsetMetaPB& rowset_meta_pb) {
+    if (rowset_meta_pb.has_tablet_schema()) {
+        _schema = TabletSchemaCache::instance()->insert(
+                rowset_meta_pb.tablet_schema().SerializeAsString());
+    }
+    // Release ownership of TabletSchemaPB from `rowset_meta_pb` and then set 
it back to `rowset_meta_pb`,
+    // this won't break const semantics of `rowset_meta_pb`, because 
`rowset_meta_pb` is not changed
+    // before and after call this method.
+    auto& mut_rowset_meta_pb = const_cast<RowsetMetaPB&>(rowset_meta_pb);
+    auto schema = mut_rowset_meta_pb.release_tablet_schema();
+    _rowset_meta_pb = mut_rowset_meta_pb;
+    mut_rowset_meta_pb.set_allocated_tablet_schema(schema);
+    _init();
+    return true;
+}
+
+bool RowsetMeta::init_from_json(const std::string& json_rowset_meta) {
+    bool ret = json2pb::JsonToProtoMessage(json_rowset_meta, &_rowset_meta_pb);
+    if (!ret) {
+        return false;
+    }
+    _init();
+    return true;
+}
+
+bool RowsetMeta::json_rowset_meta(std::string* json_rowset_meta) {
+    json2pb::Pb2JsonOptions json_options;
+    json_options.pretty_json = true;
+    bool ret = json2pb::ProtoMessageToJson(_rowset_meta_pb, json_rowset_meta, 
json_options);
+    return ret;
+}
+
+const io::FileSystemSPtr& RowsetMeta::fs() {
+    if (!_fs) {
+        if (is_local()) {
+            _fs = io::global_local_filesystem();
+        } else {
+            _fs = get_filesystem(resource_id());
+            LOG_IF(WARNING, !_fs) << "Cannot get file system: " << 
resource_id();
+        }
+    }
+    return _fs;
+}
+
+void RowsetMeta::set_fs(io::FileSystemSPtr fs) {
+    if (fs && fs->type() != io::FileSystemType::LOCAL) {
+        _rowset_meta_pb.set_resource_id(fs->id());
+    }
+    _fs = std::move(fs);
+}
+
+void RowsetMeta::to_rowset_pb(RowsetMetaPB* rs_meta_pb) const {

Review Comment:
   warning: method 'to_rowset_pb' can be made static 
[readability-convert-member-functions-to-static]
   
   ```suggestion
   static void RowsetMeta::to_rowset_pb(RowsetMetaPB* rs_meta_pb) {
   ```
   



##########
be/src/olap/rowset/rowset_meta.cpp:
##########
@@ -0,0 +1,154 @@
+// 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 "olap/rowset/rowset_meta.h"
+
+#include "common/logging.h"
+#include "google/protobuf/util/message_differencer.h"
+#include "io/fs/local_file_system.h"
+#include "json2pb/json_to_pb.h"
+#include "json2pb/pb_to_json.h"
+#include "olap/olap_common.h"
+#include "olap/storage_policy.h"
+#include "olap/tablet_schema.h"
+#include "olap/tablet_schema_cache.h"
+
+namespace doris {
+
+RowsetMeta::~RowsetMeta() = default;
+
+bool RowsetMeta::init(const std::string& pb_rowset_meta) {
+    bool ret = _deserialize_from_pb(pb_rowset_meta);
+    if (!ret) {
+        return false;
+    }
+    _init();
+    return true;
+}
+
+bool RowsetMeta::init_from_pb(const RowsetMetaPB& rowset_meta_pb) {
+    if (rowset_meta_pb.has_tablet_schema()) {
+        _schema = TabletSchemaCache::instance()->insert(
+                rowset_meta_pb.tablet_schema().SerializeAsString());
+    }
+    // Release ownership of TabletSchemaPB from `rowset_meta_pb` and then set 
it back to `rowset_meta_pb`,
+    // this won't break const semantics of `rowset_meta_pb`, because 
`rowset_meta_pb` is not changed
+    // before and after call this method.
+    auto& mut_rowset_meta_pb = const_cast<RowsetMetaPB&>(rowset_meta_pb);
+    auto schema = mut_rowset_meta_pb.release_tablet_schema();
+    _rowset_meta_pb = mut_rowset_meta_pb;
+    mut_rowset_meta_pb.set_allocated_tablet_schema(schema);
+    _init();
+    return true;
+}
+
+bool RowsetMeta::init_from_json(const std::string& json_rowset_meta) {
+    bool ret = json2pb::JsonToProtoMessage(json_rowset_meta, &_rowset_meta_pb);
+    if (!ret) {
+        return false;
+    }
+    _init();
+    return true;
+}
+
+bool RowsetMeta::json_rowset_meta(std::string* json_rowset_meta) {
+    json2pb::Pb2JsonOptions json_options;
+    json_options.pretty_json = true;
+    bool ret = json2pb::ProtoMessageToJson(_rowset_meta_pb, json_rowset_meta, 
json_options);
+    return ret;
+}
+
+const io::FileSystemSPtr& RowsetMeta::fs() {
+    if (!_fs) {
+        if (is_local()) {
+            _fs = io::global_local_filesystem();
+        } else {
+            _fs = get_filesystem(resource_id());
+            LOG_IF(WARNING, !_fs) << "Cannot get file system: " << 
resource_id();
+        }
+    }
+    return _fs;
+}
+
+void RowsetMeta::set_fs(io::FileSystemSPtr fs) {
+    if (fs && fs->type() != io::FileSystemType::LOCAL) {
+        _rowset_meta_pb.set_resource_id(fs->id());
+    }
+    _fs = std::move(fs);
+}
+
+void RowsetMeta::to_rowset_pb(RowsetMetaPB* rs_meta_pb) const {
+    *rs_meta_pb = _rowset_meta_pb;
+    if (_schema) {
+        _schema->to_schema_pb(rs_meta_pb->mutable_tablet_schema());
+    }
+}
+
+RowsetMetaPB RowsetMeta::get_rowset_pb() {
+    RowsetMetaPB rowset_meta_pb = _rowset_meta_pb;
+    if (_schema) {
+        _schema->to_schema_pb(rowset_meta_pb.mutable_tablet_schema());
+    }
+    return rowset_meta_pb;
+}
+
+void RowsetMeta::set_tablet_schema(const TabletSchemaSPtr& tablet_schema) {
+    _schema = TabletSchemaCache::instance()->insert(tablet_schema->to_key());
+}
+
+bool RowsetMeta::_deserialize_from_pb(const std::string& value) {

Review Comment:
   warning: method '_deserialize_from_pb' can be made static 
[readability-convert-member-functions-to-static]
   
   ```suggestion
   static bool RowsetMeta::_deserialize_from_pb(const std::string& value) {
   ```
   



##########
be/src/cloud/config.h:
##########
@@ -0,0 +1,28 @@
+// 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.
+
+#pragma once
+
+#include "common/config.h"
+
+namespace doris {
+namespace config {

Review Comment:
   warning: nested namespaces can be concatenated 
[modernize-concat-nested-namespaces]
   
   ```suggestion
   namespace doris::config {
   ```
   
   be/src/cloud/config.h:26:
   ```diff
   - } // namespace config
   - } // namespace doris
   + } // namespace doris
   ```
   



##########
be/src/olap/rowset/rowset_meta.cpp:
##########
@@ -0,0 +1,154 @@
+// 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 "olap/rowset/rowset_meta.h"
+
+#include "common/logging.h"
+#include "google/protobuf/util/message_differencer.h"
+#include "io/fs/local_file_system.h"
+#include "json2pb/json_to_pb.h"
+#include "json2pb/pb_to_json.h"
+#include "olap/olap_common.h"
+#include "olap/storage_policy.h"
+#include "olap/tablet_schema.h"
+#include "olap/tablet_schema_cache.h"
+
+namespace doris {
+
+RowsetMeta::~RowsetMeta() = default;
+
+bool RowsetMeta::init(const std::string& pb_rowset_meta) {
+    bool ret = _deserialize_from_pb(pb_rowset_meta);
+    if (!ret) {
+        return false;
+    }
+    _init();
+    return true;
+}
+
+bool RowsetMeta::init_from_pb(const RowsetMetaPB& rowset_meta_pb) {
+    if (rowset_meta_pb.has_tablet_schema()) {
+        _schema = TabletSchemaCache::instance()->insert(
+                rowset_meta_pb.tablet_schema().SerializeAsString());
+    }
+    // Release ownership of TabletSchemaPB from `rowset_meta_pb` and then set 
it back to `rowset_meta_pb`,
+    // this won't break const semantics of `rowset_meta_pb`, because 
`rowset_meta_pb` is not changed
+    // before and after call this method.
+    auto& mut_rowset_meta_pb = const_cast<RowsetMetaPB&>(rowset_meta_pb);
+    auto schema = mut_rowset_meta_pb.release_tablet_schema();
+    _rowset_meta_pb = mut_rowset_meta_pb;
+    mut_rowset_meta_pb.set_allocated_tablet_schema(schema);
+    _init();
+    return true;
+}
+
+bool RowsetMeta::init_from_json(const std::string& json_rowset_meta) {
+    bool ret = json2pb::JsonToProtoMessage(json_rowset_meta, &_rowset_meta_pb);
+    if (!ret) {
+        return false;
+    }
+    _init();
+    return true;
+}
+
+bool RowsetMeta::json_rowset_meta(std::string* json_rowset_meta) {
+    json2pb::Pb2JsonOptions json_options;
+    json_options.pretty_json = true;
+    bool ret = json2pb::ProtoMessageToJson(_rowset_meta_pb, json_rowset_meta, 
json_options);
+    return ret;
+}
+
+const io::FileSystemSPtr& RowsetMeta::fs() {
+    if (!_fs) {
+        if (is_local()) {
+            _fs = io::global_local_filesystem();
+        } else {
+            _fs = get_filesystem(resource_id());
+            LOG_IF(WARNING, !_fs) << "Cannot get file system: " << 
resource_id();
+        }
+    }
+    return _fs;
+}
+
+void RowsetMeta::set_fs(io::FileSystemSPtr fs) {
+    if (fs && fs->type() != io::FileSystemType::LOCAL) {
+        _rowset_meta_pb.set_resource_id(fs->id());
+    }
+    _fs = std::move(fs);
+}
+
+void RowsetMeta::to_rowset_pb(RowsetMetaPB* rs_meta_pb) const {
+    *rs_meta_pb = _rowset_meta_pb;
+    if (_schema) {
+        _schema->to_schema_pb(rs_meta_pb->mutable_tablet_schema());
+    }
+}
+
+RowsetMetaPB RowsetMeta::get_rowset_pb() {
+    RowsetMetaPB rowset_meta_pb = _rowset_meta_pb;
+    if (_schema) {
+        _schema->to_schema_pb(rowset_meta_pb.mutable_tablet_schema());
+    }
+    return rowset_meta_pb;
+}
+
+void RowsetMeta::set_tablet_schema(const TabletSchemaSPtr& tablet_schema) {
+    _schema = TabletSchemaCache::instance()->insert(tablet_schema->to_key());
+}
+
+bool RowsetMeta::_deserialize_from_pb(const std::string& value) {
+    RowsetMetaPB rowset_meta_pb;
+    if (!rowset_meta_pb.ParseFromString(value)) {
+        return false;
+    }
+    if (rowset_meta_pb.has_tablet_schema()) {
+        _schema = TabletSchemaCache::instance()->insert(
+                rowset_meta_pb.tablet_schema().SerializeAsString());
+        rowset_meta_pb.clear_tablet_schema();
+    }
+    _rowset_meta_pb = rowset_meta_pb;
+    return true;
+}
+
+bool RowsetMeta::_serialize_to_pb(std::string* value) {
+    if (value == nullptr) {
+        return false;
+    }
+    RowsetMetaPB rowset_meta_pb = _rowset_meta_pb;
+    if (_schema) {
+        _schema->to_schema_pb(rowset_meta_pb.mutable_tablet_schema());
+    }
+    return rowset_meta_pb.SerializeToString(value);
+}
+
+void RowsetMeta::_init() {

Review Comment:
   warning: method '_init' can be made static 
[readability-convert-member-functions-to-static]
   
   be/src/olap/rowset/rowset_meta.h:306:
   ```diff
   -     void _init();
   +     static void _init();
   ```
   



##########
be/src/olap/rowset/rowset_meta.h:
##########
@@ -328,20 +260,20 @@ class RowsetMeta {
 
     auto& get_segments_key_bounds() { return 
_rowset_meta_pb.segments_key_bounds(); }
 
-    virtual bool get_first_segment_key_bound(KeyBoundsPB* key_bounds) {
+    bool get_first_segment_key_bound(KeyBoundsPB* key_bounds) {

Review Comment:
   warning: method 'get_first_segment_key_bound' can be made static 
[readability-convert-member-functions-to-static]
   
   ```suggestion
       static bool get_first_segment_key_bound(KeyBoundsPB* key_bounds) {
   ```
   



##########
be/src/olap/rowset/rowset_meta.h:
##########
@@ -328,20 +260,20 @@ class RowsetMeta {
 
     auto& get_segments_key_bounds() { return 
_rowset_meta_pb.segments_key_bounds(); }
 
-    virtual bool get_first_segment_key_bound(KeyBoundsPB* key_bounds) {
+    bool get_first_segment_key_bound(KeyBoundsPB* key_bounds) {
         // for compatibility, old version has not segment key bounds
         if (_rowset_meta_pb.segments_key_bounds_size() == 0) {
             return false;
         }
-        *key_bounds = _rowset_meta_pb.segments_key_bounds(0);
+        *key_bounds = *_rowset_meta_pb.segments_key_bounds().begin();
         return true;
     }
-    virtual bool get_last_segment_key_bound(KeyBoundsPB* key_bounds) {
+
+    bool get_last_segment_key_bound(KeyBoundsPB* key_bounds) {

Review Comment:
   warning: method 'get_last_segment_key_bound' can be made static 
[readability-convert-member-functions-to-static]
   
   ```suggestion
       static bool get_last_segment_key_bound(KeyBoundsPB* key_bounds) {
   ```
   



##########
be/src/olap/rowset_builder.cpp:
##########
@@ -82,83 +83,108 @@ RowsetBuilder::~RowsetBuilder() {
     if (_calc_delete_bitmap_token != nullptr) {
         _calc_delete_bitmap_token->cancel();
     }
-
-    if (_tablet != nullptr) {
-        _tablet->data_dir()->remove_pending_ids(ROWSET_ID_PREFIX +
-                                                
_rowset_writer->rowset_id().to_string());
-    }
 }
 
 void RowsetBuilder::_garbage_collection() {
-    Status rollback_status = Status::OK();
-    TxnManager* txn_mgr = _storage_engine->txn_manager();
-    if (_tablet != nullptr) {
-        rollback_status = txn_mgr->rollback_txn(_req.partition_id, _tablet, 
_req.txn_id);
+    if (config::cloud_mode) {
+        return;
+    }
+    Status rollback_status;
+    TxnManager* txn_mgr = StorageEngine::instance()->txn_manager();
+    auto tablet = static_cast<Tablet*>(_tablet.get());
+    if (tablet != nullptr) {
+        rollback_status = txn_mgr->rollback_txn(_req.partition_id, *tablet, 
_req.txn_id);
     }
     // has to check rollback status, because the rowset maybe committed in 
this thread and
     // published in another thread, then rollback will fail.
     // when rollback failed should not delete rowset
     if (rollback_status.ok()) {
-        _storage_engine->add_unused_rowset(_rowset);
+        StorageEngine::instance()->add_unused_rowset(_rowset);
     }
 }
 
-Status RowsetBuilder::init() {
-    TabletManager* tablet_mgr = _storage_engine->tablet_manager();
-    _tablet = tablet_mgr->get_tablet(_req.tablet_id);
-    if (_tablet == nullptr) {
-        return Status::Error<TABLE_NOT_FOUND>("fail to find tablet. 
tablet_id={}, schema_hash={}",
-                                              _req.tablet_id, 
_req.schema_hash);
+Status RowsetBuilder::init_mow_context(std::shared_ptr<MowContext>& 
mow_context) {
+    if (config::cloud_mode) {
+        // TODO(plat1ko)
+        return Status::NotSupported("init_mow_context");
+    }
+    auto tablet = static_cast<Tablet*>(_tablet.get());
+    std::lock_guard<std::shared_mutex> lck(tablet->get_header_lock());
+    int64_t cur_max_version = tablet->max_version_unlocked().second;
+    // tablet is under alter process. The delete bitmap will be calculated 
after conversion.
+    if (tablet->tablet_state() == TABLET_NOTREADY &&
+        SchemaChangeHandler::tablet_in_converting(_tablet->tablet_id())) {
+        // Disable 'partial_update' when the tablet is undergoing a 'schema 
changing process'
+        if (_req.table_schema_param->is_partial_update()) {
+            return Status::InternalError(
+                    "Unable to do 'partial_update' when "
+                    "the tablet is undergoing a 'schema changing process'");
+        }
+        _rowset_ids.clear();
+    } else {
+        _rowset_ids = tablet->all_rs_id(cur_max_version);
+    }
+    _delete_bitmap = std::make_shared<DeleteBitmap>(tablet->tablet_id());
+    mow_context =
+            std::make_shared<MowContext>(cur_max_version, _req.txn_id, 
_rowset_ids, _delete_bitmap);
+    return Status::OK();
+}
+
+Status RowsetBuilder::check_tablet_version_count() {

Review Comment:
   warning: method 'check_tablet_version_count' can be made static 
[readability-convert-member-functions-to-static]
   
   be/src/olap/rowset_builder.h:98:
   ```diff
   -     Status check_tablet_version_count();
   +     static Status check_tablet_version_count();
   ```
   



##########
be/src/olap/tablet.cpp:
##########
@@ -272,20 +274,28 @@ Tablet::Tablet(TabletMetaSharedPtr tablet_meta, DataDir* 
data_dir,
     // And when upgrade to doris 1.2 version,
     // all rowset metas will be set the tablet schmea from tablet meta.
     if (_tablet_meta->all_rs_metas().empty() || 
!_tablet_meta->all_rs_metas()[0]->tablet_schema()) {
-        _max_version_schema = BaseTablet::tablet_schema();
+        _max_version_schema = _tablet_meta->tablet_schema();
     } else {
         _max_version_schema =
                 
rowset_meta_with_max_schema_version(_tablet_meta->all_rs_metas())->tablet_schema();
     }
     DCHECK(_max_version_schema);
+}
 
-    INT_COUNTER_METRIC_REGISTER(_metric_entity, flush_bytes);
-    INT_COUNTER_METRIC_REGISTER(_metric_entity, flush_finish_count);
+bool Tablet::set_tablet_schema_into_rowset_meta() {

Review Comment:
   warning: method 'set_tablet_schema_into_rowset_meta' can be made static 
[readability-convert-member-functions-to-static]
   
   be/src/olap/tablet.h:102:
   ```diff
   -     bool set_tablet_schema_into_rowset_meta();
   +     static bool set_tablet_schema_into_rowset_meta();
   ```
   



##########
be/src/olap/rowset_builder.cpp:
##########
@@ -265,20 +307,20 @@ Status RowsetBuilder::commit_txn() {
             return st;
         }
     }
-
+    auto storage_engine = StorageEngine::instance();

Review Comment:
   warning: 'auto storage_engine' can be declared as 'auto *storage_engine' 
[readability-qualified-auto]
   
   ```suggestion
       auto *storage_engine = StorageEngine::instance();
   ```
   



##########
be/src/olap/rowset_builder.cpp:
##########
@@ -82,83 +83,108 @@ RowsetBuilder::~RowsetBuilder() {
     if (_calc_delete_bitmap_token != nullptr) {
         _calc_delete_bitmap_token->cancel();
     }
-
-    if (_tablet != nullptr) {
-        _tablet->data_dir()->remove_pending_ids(ROWSET_ID_PREFIX +
-                                                
_rowset_writer->rowset_id().to_string());
-    }
 }
 
 void RowsetBuilder::_garbage_collection() {
-    Status rollback_status = Status::OK();
-    TxnManager* txn_mgr = _storage_engine->txn_manager();
-    if (_tablet != nullptr) {
-        rollback_status = txn_mgr->rollback_txn(_req.partition_id, _tablet, 
_req.txn_id);
+    if (config::cloud_mode) {
+        return;
+    }
+    Status rollback_status;
+    TxnManager* txn_mgr = StorageEngine::instance()->txn_manager();
+    auto tablet = static_cast<Tablet*>(_tablet.get());
+    if (tablet != nullptr) {
+        rollback_status = txn_mgr->rollback_txn(_req.partition_id, *tablet, 
_req.txn_id);
     }
     // has to check rollback status, because the rowset maybe committed in 
this thread and
     // published in another thread, then rollback will fail.
     // when rollback failed should not delete rowset
     if (rollback_status.ok()) {
-        _storage_engine->add_unused_rowset(_rowset);
+        StorageEngine::instance()->add_unused_rowset(_rowset);
     }
 }
 
-Status RowsetBuilder::init() {
-    TabletManager* tablet_mgr = _storage_engine->tablet_manager();
-    _tablet = tablet_mgr->get_tablet(_req.tablet_id);
-    if (_tablet == nullptr) {
-        return Status::Error<TABLE_NOT_FOUND>("fail to find tablet. 
tablet_id={}, schema_hash={}",
-                                              _req.tablet_id, 
_req.schema_hash);
+Status RowsetBuilder::init_mow_context(std::shared_ptr<MowContext>& 
mow_context) {
+    if (config::cloud_mode) {
+        // TODO(plat1ko)
+        return Status::NotSupported("init_mow_context");
+    }
+    auto tablet = static_cast<Tablet*>(_tablet.get());
+    std::lock_guard<std::shared_mutex> lck(tablet->get_header_lock());
+    int64_t cur_max_version = tablet->max_version_unlocked().second;
+    // tablet is under alter process. The delete bitmap will be calculated 
after conversion.
+    if (tablet->tablet_state() == TABLET_NOTREADY &&
+        SchemaChangeHandler::tablet_in_converting(_tablet->tablet_id())) {
+        // Disable 'partial_update' when the tablet is undergoing a 'schema 
changing process'
+        if (_req.table_schema_param->is_partial_update()) {
+            return Status::InternalError(
+                    "Unable to do 'partial_update' when "
+                    "the tablet is undergoing a 'schema changing process'");
+        }
+        _rowset_ids.clear();
+    } else {
+        _rowset_ids = tablet->all_rs_id(cur_max_version);
+    }
+    _delete_bitmap = std::make_shared<DeleteBitmap>(tablet->tablet_id());
+    mow_context =
+            std::make_shared<MowContext>(cur_max_version, _req.txn_id, 
_rowset_ids, _delete_bitmap);
+    return Status::OK();
+}
+
+Status RowsetBuilder::check_tablet_version_count() {
+    if (!_tablet->exceed_version_limit(config::max_tablet_version_num - 100) ||
+        MemInfo::is_exceed_soft_mem_limit(GB_EXCHANGE_BYTE)) {
+        return Status::OK();
+    }
+    if (config::cloud_mode) {
+        // TODO(plat1ko)
+        return Status::OK();
+    }
+    auto tablet = std::static_pointer_cast<Tablet>(_tablet);
+    //trigger compaction
+    auto st = StorageEngine::instance()->submit_compaction_task(
+            tablet, CompactionType::CUMULATIVE_COMPACTION, true);
+    if (!st.ok()) [[unlikely]] {
+        LOG(WARNING) << "failed to trigger compaction, tablet_id=" << 
_tablet->tablet_id() << " : "
+                     << st;
+    }
+    int version_count = tablet->version_count();
+    if (version_count > config::max_tablet_version_num) {
+        return Status::Error<TOO_MANY_VERSION>(
+                "failed to init rowset builder. version count: {}, exceed 
limit: {}, "
+                "tablet: {}",
+                version_count, config::max_tablet_version_num, 
tablet->tablet_id());
+    }
+    return Status::OK();
+}
+
+Status RowsetBuilder::prepare_txn() {

Review Comment:
   warning: method 'prepare_txn' can be made static 
[readability-convert-member-functions-to-static]
   
   be/src/olap/rowset_builder.h:100:
   ```diff
   -     Status prepare_txn();
   +     static Status prepare_txn();
   ```
   



##########
be/src/olap/rowset_builder.cpp:
##########
@@ -82,83 +83,108 @@ RowsetBuilder::~RowsetBuilder() {
     if (_calc_delete_bitmap_token != nullptr) {
         _calc_delete_bitmap_token->cancel();
     }
-
-    if (_tablet != nullptr) {
-        _tablet->data_dir()->remove_pending_ids(ROWSET_ID_PREFIX +
-                                                
_rowset_writer->rowset_id().to_string());
-    }
 }
 
 void RowsetBuilder::_garbage_collection() {
-    Status rollback_status = Status::OK();
-    TxnManager* txn_mgr = _storage_engine->txn_manager();
-    if (_tablet != nullptr) {
-        rollback_status = txn_mgr->rollback_txn(_req.partition_id, _tablet, 
_req.txn_id);
+    if (config::cloud_mode) {
+        return;
+    }
+    Status rollback_status;
+    TxnManager* txn_mgr = StorageEngine::instance()->txn_manager();
+    auto tablet = static_cast<Tablet*>(_tablet.get());
+    if (tablet != nullptr) {
+        rollback_status = txn_mgr->rollback_txn(_req.partition_id, *tablet, 
_req.txn_id);
     }
     // has to check rollback status, because the rowset maybe committed in 
this thread and
     // published in another thread, then rollback will fail.
     // when rollback failed should not delete rowset
     if (rollback_status.ok()) {
-        _storage_engine->add_unused_rowset(_rowset);
+        StorageEngine::instance()->add_unused_rowset(_rowset);
     }
 }
 
-Status RowsetBuilder::init() {
-    TabletManager* tablet_mgr = _storage_engine->tablet_manager();
-    _tablet = tablet_mgr->get_tablet(_req.tablet_id);
-    if (_tablet == nullptr) {
-        return Status::Error<TABLE_NOT_FOUND>("fail to find tablet. 
tablet_id={}, schema_hash={}",
-                                              _req.tablet_id, 
_req.schema_hash);
+Status RowsetBuilder::init_mow_context(std::shared_ptr<MowContext>& 
mow_context) {

Review Comment:
   warning: method 'init_mow_context' can be made static 
[readability-convert-member-functions-to-static]
   
   be/src/olap/rowset_builder.h:96:
   ```diff
   -     Status init_mow_context(std::shared_ptr<MowContext>& mow_context);
   +     static Status init_mow_context(std::shared_ptr<MowContext>& 
mow_context);
   ```
   



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

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


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


Reply via email to