You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2022/07/05 04:20:58 UTC

[GitHub] [doris] dataroaring commented on a diff in pull request #10136: [Feature] Lightweight schema change of add/drop column

dataroaring commented on code in PR #10136:
URL: https://github.com/apache/doris/pull/10136#discussion_r912369533


##########
be/src/olap/rowset/rowset.cpp:
##########
@@ -35,6 +34,12 @@ Rowset::Rowset(const TabletSchema* schema, const FilePathDesc& rowset_path_desc,
         Version version = _rowset_meta->version();
         _is_cumulative = version.first != version.second;
     }
+    // build schema from RowsetMeta.tablet_schema or Tablet.tablet_schema
+    if (_rowset_meta->tablet_schema() != nullptr) {
+        _schema = _rowset_meta->tablet_schema();
+    } else {
+        _schema = schema;

Review Comment:
   _schema = _rowset_meta->tablet_schema() != nullptr ? _schema = _rowset_meta->tablet_schema() : schema;



##########
be/src/olap/delta_writer.cpp:
##########
@@ -197,7 +197,7 @@ Status DeltaWriter::write(const vectorized::Block* block, const std::vector<int>
     if (_is_cancelled) {
         return Status::OLAPInternalError(OLAP_ERR_ALREADY_CANCELLED);
     }
-
+    LOG(INFO) << "3 block columns: " << block->columns();

Review Comment:
   remove this log.



##########
be/src/olap/rowset/segment_v2/segment_writer.cpp:
##########
@@ -116,6 +116,7 @@ Status SegmentWriter::append_block(const vectorized::Block* block, size_t row_po
                                    size_t num_rows) {
     assert(block && num_rows > 0 && row_pos + num_rows <= block->rows() &&
            block->columns() == _column_writers.size());
+    LOG(INFO) << "block columns: " << block->columns();

Review Comment:
   remove the log please.



##########
be/src/olap/segment_loader.cpp:
##########
@@ -58,16 +59,17 @@ void SegmentLoader::_insert(const SegmentLoader::CacheKey& key, SegmentLoader::C
 }
 
 Status SegmentLoader::load_segments(const BetaRowsetSharedPtr& rowset,
-                                    SegmentCacheHandle* cache_handle, bool use_cache) {
-    SegmentLoader::CacheKey cache_key(rowset->rowset_id());
-    if (_lookup(cache_key, cache_handle)) {
+                                    SegmentCacheHandle* cache_handle,
+                                    const TabletSchema* read_tablet_schema, bool use_cache) {
+    SegmentLoader::CacheKey cache_key(rowset->rowset_id(), *read_tablet_schema);

Review Comment:
   Could we use schema version as part of the cache key instead of schema it self?



##########
be/src/exec/olap_scanner.cpp:
##########
@@ -86,6 +88,14 @@ Status OlapScanner::prepare(
             LOG(WARNING) << ss.str();
             return Status::InternalError(ss.str());
         }
+        _tablet_schema = _tablet->tablet_schema();
+        if (!_parent->_olap_scan_node.columns_desc.empty() &&
+            _parent->_olap_scan_node.columns_desc[0].col_unique_id >= 0) {
+            _tablet_schema.clear_columns();
+            for (const auto& column_desc : _parent->_olap_scan_node.columns_desc) {
+                _tablet_schema.append_column(TabletColumn(column_desc));
+            }
+        }

Review Comment:
   We'd better put a comment here. e.g. Originally scanner get TabletSchema from tablet object in be. To support lightweight schema change for adding / dropping columns, tabletschema is bounded to rowset and tablet's schema maybe outdated, so we have to use schema from a query plan, fe puts schema in query plans.



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

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


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