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/06/07 14:35:05 UTC

[GitHub] [incubator-doris] BiteTheDDDDt opened a new pull request, #10003: [Feature] [Vectorized] Some pre-refactorings or interface additions for schema change part2

BiteTheDDDDt opened a new pull request, #10003:
URL: https://github.com/apache/incubator-doris/pull/10003

   # Proposed changes
   
   This pr is part of https://github.com/apache/incubator-doris/pull/9375, included some refactorings or interface additions.
   
   1. Pass a `DescriptorTable` from` fe` to `be`.
   2. Split `sc.process` to `process` and `inner_process` .
   3. Change singleton class `SchemaChangeHandler` to all static function class.
   4. Add a function `get_sc_procedure` wrapped some common logic.
   5. Some other refactoring
   
   ## Problem Summary:
   
   Describe the overview of changes.
   
   ## Checklist(Required)
   
   1. Does it affect the original behavior: (Yes/No/I Don't know)
   3. Has unit tests been added: (Yes/No/No Need)
   5. Has document been added or modified: (Yes/No/No Need)
   6. Does it need to update dependencies: (Yes/No)
   7. Are there any changes that cannot be rolled back: (Yes/No)
   
   ## Further comments
   
   If this is a relatively large or complex change, kick off the discussion at [dev@doris.apache.org](mailto:dev@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc...
   


-- 
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


[GitHub] [incubator-doris] HappenLee commented on a diff in pull request #10003: [Feature] [Vectorized] Some pre-refactorings or interface additions for schema change part2

Posted by GitBox <gi...@apache.org>.
HappenLee commented on code in PR #10003:
URL: https://github.com/apache/incubator-doris/pull/10003#discussion_r893341182


##########
be/src/vec/columns/column.h:
##########
@@ -199,14 +201,22 @@ class IColumn : public COW<IColumn> {
         LOG(FATAL) << "Method insert_many_binary_data is not supported for " << get_name();
     }
 
+    virtual void insert_many_data(const char* pos, size_t length, size_t data_num) {

Review Comment:
   the function seems not need be `virtual`



-- 
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


[GitHub] [incubator-doris] github-actions[bot] commented on pull request #10003: [Feature] [Vectorized] Some pre-refactorings or interface additions for schema change part2

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #10003:
URL: https://github.com/apache/incubator-doris/pull/10003#issuecomment-1156347722

   PR approved by anyone and no changes requested.


-- 
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


[GitHub] [incubator-doris] HappenLee commented on a diff in pull request #10003: [Feature] [Vectorized] Some pre-refactorings or interface additions for schema change part2

Posted by GitBox <gi...@apache.org>.
HappenLee commented on code in PR #10003:
URL: https://github.com/apache/incubator-doris/pull/10003#discussion_r892290286


##########
be/src/olap/schema_change.h:
##########
@@ -70,6 +59,8 @@ class RowBlockChanger {
     // delete handler for filtering data which use specified in DELETE_DATA
     const DeleteHandler* _delete_handler = nullptr;
 
+    DescriptorTbl _desc_tbl;

Review Comment:
   why add `_desc_tbl` seems only use in construct function?



##########
be/src/olap/schema_change.h:
##########
@@ -94,20 +85,62 @@ class SchemaChange {
     SchemaChange() : _filtered_rows(0), _merged_rows(0) {}
     virtual ~SchemaChange() = default;
 
-    virtual Status process(RowsetReaderSharedPtr rowset_reader, RowsetWriter* new_rowset_builder,
-                           TabletSharedPtr tablet, TabletSharedPtr base_tablet) = 0;
-
-    void add_filtered_rows(uint64_t filtered_rows) { _filtered_rows += filtered_rows; }
-
-    void add_merged_rows(uint64_t merged_rows) { _merged_rows += merged_rows; }
+    virtual Status process(RowsetReaderSharedPtr rowset_reader, RowsetWriter* rowset_writer,
+                           TabletSharedPtr new_tablet, TabletSharedPtr base_tablet) {
+        if (rowset_reader->rowset()->empty() || rowset_reader->rowset()->num_rows() == 0) {
+            RETURN_WITH_WARN_IF_ERROR(
+                    rowset_writer->flush(),
+                    Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR),
+                    fmt::format("create empty version for schema change failed. version= {}-{}",
+                                rowset_writer->version().first, rowset_writer->version().second));
+
+            return Status::OK();
+        }
+
+        _filtered_rows = 0;
+        _merged_rows = 0;
+
+        RETURN_IF_ERROR(_inner_process(rowset_reader, rowset_writer, new_tablet, base_tablet));
+        _add_filtered_rows(rowset_reader->filtered_rows());
+
+        // Check row num changes
+        if (config::row_nums_check) {

Review Comment:
   `if (config::row_nums_check && !_check_row_nums(rowset_reader, *rowset_writer))`



##########
be/src/olap/schema_change.cpp:
##########
@@ -807,22 +807,32 @@ bool RowBlockAllocator::is_memory_enough_for_sorting(size_t num_rows, size_t all
 
 RowBlockMerger::RowBlockMerger(TabletSharedPtr tablet) : _tablet(tablet) {}
 
-RowBlockMerger::~RowBlockMerger() {}
+RowBlockMerger::~RowBlockMerger() = default;
 
 bool RowBlockMerger::merge(const std::vector<RowBlock*>& row_block_arr, RowsetWriter* rowset_writer,
                            uint64_t* merged_rows) {
     uint64_t tmp_merged_rows = 0;
     RowCursor row_cursor;
     std::unique_ptr<MemPool> mem_pool(new MemPool("RowBlockMerger"));
     std::unique_ptr<ObjectPool> agg_object_pool(new ObjectPool());
+
+    auto merge_error = [&]() -> bool {
+        while (_heap.size() > 0) {

Review Comment:
   `!_heap.empty()` same in line 839



##########
be/src/olap/schema_change.cpp:
##########
@@ -917,37 +918,33 @@ void RowBlockMerger::_pop_heap() {
     return;

Review Comment:
   useless return



##########
be/src/olap/schema_change.cpp:
##########
@@ -92,19 +91,20 @@ class RowBlockMerger {
     std::priority_queue<MergeElement> _heap;
 };
 
-RowBlockChanger::RowBlockChanger(const TabletSchema& tablet_schema) {
+RowBlockChanger::RowBlockChanger(const TabletSchema& tablet_schema, DescriptorTbl desc_tbl)
+        : _desc_tbl(desc_tbl) {
     _schema_mapping.resize(tablet_schema.num_columns());
 }
 
 RowBlockChanger::RowBlockChanger(const TabletSchema& tablet_schema,
-                                 const DeleteHandler* delete_handler) {
+                                 const DeleteHandler* delete_handler, DescriptorTbl desc_tbl)
+        : _desc_tbl(desc_tbl) {
     _schema_mapping.resize(tablet_schema.num_columns());
     _delete_handler = delete_handler;
 }
 
 RowBlockChanger::~RowBlockChanger() {
-    SchemaMapping::iterator it = _schema_mapping.begin();
-    for (; it != _schema_mapping.end(); ++it) {
+    for (auto it = _schema_mapping.begin(); it != _schema_mapping.end(); ++it) {
         SAFE_DELETE(it->default_value);

Review Comment:
   recheck the logic? maybe we should do the work in `~ColumnMapping` and `RowBlockChanger::~RowBlockChanger() = default`



-- 
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


[GitHub] [incubator-doris] compiletheworld commented on pull request #10003: [Feature] [Vectorized] Some pre-refactorings or interface additions for schema change part2

Posted by GitBox <gi...@apache.org>.
compiletheworld commented on PR #10003:
URL: https://github.com/apache/incubator-doris/pull/10003#issuecomment-1149509089

   LGTM


-- 
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


[GitHub] [incubator-doris] BiteTheDDDDt commented on a diff in pull request #10003: [Feature] [Vectorized] Some pre-refactorings or interface additions for schema change part2

Posted by GitBox <gi...@apache.org>.
BiteTheDDDDt commented on code in PR #10003:
URL: https://github.com/apache/incubator-doris/pull/10003#discussion_r891899432


##########
be/src/olap/schema_change.cpp:
##########
@@ -1010,16 +1008,6 @@ Status SchemaChangeDirectly::process(RowsetReaderSharedPtr rowset_reader,
     }
 
     Status res = Status::OK();
-    if (rowset_reader->rowset()->empty() || rowset_reader->rowset()->num_rows() == 0) {

Review Comment:
   moved to `SchemaChange::process`



-- 
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


[GitHub] [incubator-doris] BiteTheDDDDt commented on a diff in pull request #10003: [Feature] [Vectorized] Some pre-refactorings or interface additions for schema change part2

Posted by GitBox <gi...@apache.org>.
BiteTheDDDDt commented on code in PR #10003:
URL: https://github.com/apache/incubator-doris/pull/10003#discussion_r892414205


##########
be/src/olap/reader.h:
##########
@@ -108,7 +108,7 @@ class TabletReader {
     virtual ~TabletReader();
 
     // Initialize TabletReader with tablet, data version and fetch range.
-    virtual Status init(const ReaderParams& read_params);
+    virtual Status init(const ReaderParams& read_params, bool is_alter_table = false);

Review Comment:
   done



-- 
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


[GitHub] [incubator-doris] BiteTheDDDDt commented on a diff in pull request #10003: [Feature] [Vectorized] Some pre-refactorings or interface additions for schema change part2

Posted by GitBox <gi...@apache.org>.
BiteTheDDDDt commented on code in PR #10003:
URL: https://github.com/apache/incubator-doris/pull/10003#discussion_r891899112


##########
be/src/olap/schema_change.cpp:
##########
@@ -1065,48 +1049,25 @@ Status SchemaChangeDirectly::process(RowsetReaderSharedPtr rowset_reader,
         return Status::OLAPInternalError(OLAP_ERR_ALTER_STATUS_ERR);
     }
 
-    // rows filtered by zone map against delete handler
-    add_filtered_rows(rowset_reader->filtered_rows());
-
-    // Check row num changes
-    if (config::row_nums_check) {

Review Comment:
   I move those common logic to `SchemaChange::process`, and some specific logic is placed in `_inner_process`



-- 
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


[GitHub] [incubator-doris] BiteTheDDDDt commented on a diff in pull request #10003: [Feature] [Vectorized] Some pre-refactorings or interface additions for schema change part2

Posted by GitBox <gi...@apache.org>.
BiteTheDDDDt commented on code in PR #10003:
URL: https://github.com/apache/incubator-doris/pull/10003#discussion_r892420206


##########
be/src/olap/schema_change.cpp:
##########
@@ -807,22 +807,32 @@ bool RowBlockAllocator::is_memory_enough_for_sorting(size_t num_rows, size_t all
 
 RowBlockMerger::RowBlockMerger(TabletSharedPtr tablet) : _tablet(tablet) {}
 
-RowBlockMerger::~RowBlockMerger() {}
+RowBlockMerger::~RowBlockMerger() = default;
 
 bool RowBlockMerger::merge(const std::vector<RowBlock*>& row_block_arr, RowsetWriter* rowset_writer,
                            uint64_t* merged_rows) {
     uint64_t tmp_merged_rows = 0;
     RowCursor row_cursor;
     std::unique_ptr<MemPool> mem_pool(new MemPool("RowBlockMerger"));
     std::unique_ptr<ObjectPool> agg_object_pool(new ObjectPool());
+
+    auto merge_error = [&]() -> bool {
+        while (_heap.size() > 0) {

Review Comment:
   done



##########
be/src/olap/schema_change.cpp:
##########
@@ -917,37 +918,33 @@ void RowBlockMerger::_pop_heap() {
     return;

Review Comment:
   removed



##########
be/src/olap/schema_change.h:
##########
@@ -94,20 +85,62 @@ class SchemaChange {
     SchemaChange() : _filtered_rows(0), _merged_rows(0) {}
     virtual ~SchemaChange() = default;
 
-    virtual Status process(RowsetReaderSharedPtr rowset_reader, RowsetWriter* new_rowset_builder,
-                           TabletSharedPtr tablet, TabletSharedPtr base_tablet) = 0;
-
-    void add_filtered_rows(uint64_t filtered_rows) { _filtered_rows += filtered_rows; }
-
-    void add_merged_rows(uint64_t merged_rows) { _merged_rows += merged_rows; }
+    virtual Status process(RowsetReaderSharedPtr rowset_reader, RowsetWriter* rowset_writer,
+                           TabletSharedPtr new_tablet, TabletSharedPtr base_tablet) {
+        if (rowset_reader->rowset()->empty() || rowset_reader->rowset()->num_rows() == 0) {
+            RETURN_WITH_WARN_IF_ERROR(
+                    rowset_writer->flush(),
+                    Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR),
+                    fmt::format("create empty version for schema change failed. version= {}-{}",
+                                rowset_writer->version().first, rowset_writer->version().second));
+
+            return Status::OK();
+        }
+
+        _filtered_rows = 0;
+        _merged_rows = 0;
+
+        RETURN_IF_ERROR(_inner_process(rowset_reader, rowset_writer, new_tablet, base_tablet));
+        _add_filtered_rows(rowset_reader->filtered_rows());
+
+        // Check row num changes
+        if (config::row_nums_check) {

Review Comment:
   done



-- 
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


[GitHub] [incubator-doris] HappenLee commented on a diff in pull request #10003: [Feature] [Vectorized] Some pre-refactorings or interface additions for schema change part2

Posted by GitBox <gi...@apache.org>.
HappenLee commented on code in PR #10003:
URL: https://github.com/apache/incubator-doris/pull/10003#discussion_r892117337


##########
be/src/olap/reader.h:
##########
@@ -108,7 +108,7 @@ class TabletReader {
     virtual ~TabletReader();
 
     // Initialize TabletReader with tablet, data version and fetch range.
-    virtual Status init(const ReaderParams& read_params);
+    virtual Status init(const ReaderParams& read_params, bool is_alter_table = false);

Review Comment:
   why not put the `is_alter_table` to `read_params`. 
   if `is_alter_table` is `true` the `read_params.reader_type == READER_ALTER_TABLE` ?



-- 
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


[GitHub] [incubator-doris] compiletheworld commented on a diff in pull request #10003: [Feature] [Vectorized] Some pre-refactorings or interface additions for schema change part2

Posted by GitBox <gi...@apache.org>.
compiletheworld commented on code in PR #10003:
URL: https://github.com/apache/incubator-doris/pull/10003#discussion_r891859191


##########
fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java:
##########
@@ -362,14 +365,26 @@ protected void runWaitingTxnJob() throws AlterCancelException {
                         }
                     }
 
+                    List<Column> fullSchema = tbl.getBaseSchema(true);
+                    DescriptorTable descTable = new DescriptorTable();
+                    for (Column column : fullSchema) {
+                        TupleDescriptor destTupleDesc = descTable.createTupleDescriptor();
+                        SlotDescriptor destSlotDesc = descTable.addSlotDescriptor(destTupleDesc);
+                        destSlotDesc.setIsMaterialized(true);
+                        destSlotDesc.setColumn(column);
+                        if (column.isAllowNull()) {

Review Comment:
   `destSlotDesc.setIsNullable(column.isAllowNull())` seems to be much more straightforward?



##########
be/src/olap/schema_change.cpp:
##########
@@ -807,22 +807,33 @@ bool RowBlockAllocator::is_memory_enough_for_sorting(size_t num_rows, size_t all
 
 RowBlockMerger::RowBlockMerger(TabletSharedPtr tablet) : _tablet(tablet) {}
 
-RowBlockMerger::~RowBlockMerger() {}
+RowBlockMerger::~RowBlockMerger() = default;
 
 bool RowBlockMerger::merge(const std::vector<RowBlock*>& row_block_arr, RowsetWriter* rowset_writer,
                            uint64_t* merged_rows) {
     uint64_t tmp_merged_rows = 0;
     RowCursor row_cursor;
     std::unique_ptr<MemPool> mem_pool(new MemPool("RowBlockMerger"));
     std::unique_ptr<ObjectPool> agg_object_pool(new ObjectPool());
+
+    auto merge_error = [&]() -> bool {
+        while (_heap.size() > 0) {
+            MergeElement element = _heap.top();
+            _heap.pop();
+            SAFE_DELETE(element.row_cursor);
+        }
+

Review Comment:
   lambdas are intended to be small and precise, blank lines should be eliminated.



##########
be/src/olap/schema_change.cpp:
##########
@@ -1010,16 +1008,6 @@ Status SchemaChangeDirectly::process(RowsetReaderSharedPtr rowset_reader,
     }
 
     Status res = Status::OK();
-    if (rowset_reader->rowset()->empty() || rowset_reader->rowset()->num_rows() == 0) {

Review Comment:
   For an empty rowset, this change will lead to missing data files of rowset, is it OK that we skip creating an empty rowset? 
   



##########
be/src/olap/schema_change.cpp:
##########
@@ -1065,48 +1049,25 @@ Status SchemaChangeDirectly::process(RowsetReaderSharedPtr rowset_reader,
         return Status::OLAPInternalError(OLAP_ERR_ALTER_STATUS_ERR);
     }
 
-    // rows filtered by zone map against delete handler
-    add_filtered_rows(rowset_reader->filtered_rows());
-
-    // Check row num changes
-    if (config::row_nums_check) {

Review Comment:
   Why do we skip this check? Is this redundant or useless?



##########
fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java:
##########
@@ -374,20 +379,57 @@ protected void runWaitingTxnJob() throws AlterCancelException {
 
         tbl.readLock();
         try {
+            Map<String, Column> indexColumnMap = Maps.newHashMap();
+            for (Map.Entry<Long, List<Column>> entry : indexSchemaMap.entrySet()) {
+                for (Column column : entry.getValue()) {
+                    indexColumnMap.put(column.getName(), column);
+                }
+            }
+
             Preconditions.checkState(tbl.getState() == OlapTableState.SCHEMA_CHANGE);
 
             for (long partitionId : partitionIndexMap.rowKeySet()) {
                 Partition partition = tbl.getPartition(partitionId);
                 Preconditions.checkNotNull(partition, partitionId);
 
-                // the schema change task will transform the data before visible version(included).
+                // the schema change task will transform the data before visible
+                // version(included).
                 long visibleVersion = partition.getVisibleVersion();
 
                 Map<Long, MaterializedIndex> shadowIndexMap = partitionIndexMap.row(partitionId);
                 for (Map.Entry<Long, MaterializedIndex> entry : shadowIndexMap.entrySet()) {
                     long shadowIdxId = entry.getKey();
                     MaterializedIndex shadowIdx = entry.getValue();
 
+                    Map<String, Expr> defineExprs = Maps.newHashMap();
+
+                    List<Column> fullSchema = tbl.getBaseSchema(true);
+                    DescriptorTable descTable = new DescriptorTable();
+                    for (Column column : fullSchema) {
+                        TupleDescriptor destTupleDesc = descTable.createTupleDescriptor();
+                        SlotDescriptor destSlotDesc = descTable.addSlotDescriptor(destTupleDesc);
+                        destSlotDesc.setIsMaterialized(true);
+                        destSlotDesc.setColumn(column);
+                        if (column.isAllowNull()) {

Review Comment:
   The same problems as `fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java:375`



##########
be/src/olap/schema_change.cpp:
##########
@@ -1119,17 +1080,6 @@ Status SchemaChangeWithSorting::process(RowsetReaderSharedPtr rowset_reader,
     Status res = Status::OK();
     RowsetSharedPtr rowset = rowset_reader->rowset();
 
-    if (rowset->empty() || rowset->num_rows() == 0) {
-        res = new_rowset_writer->flush();

Review Comment:
   Ditto



-- 
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


[GitHub] [incubator-doris] HappenLee merged pull request #10003: [Feature] [Vectorized] Some pre-refactorings or interface additions for schema change part2

Posted by GitBox <gi...@apache.org>.
HappenLee merged PR #10003:
URL: https://github.com/apache/incubator-doris/pull/10003


-- 
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


[GitHub] [incubator-doris] BiteTheDDDDt commented on a diff in pull request #10003: [Feature] [Vectorized] Some pre-refactorings or interface additions for schema change part2

Posted by GitBox <gi...@apache.org>.
BiteTheDDDDt commented on code in PR #10003:
URL: https://github.com/apache/incubator-doris/pull/10003#discussion_r892417931


##########
be/src/olap/schema_change.cpp:
##########
@@ -92,19 +91,20 @@ class RowBlockMerger {
     std::priority_queue<MergeElement> _heap;
 };
 
-RowBlockChanger::RowBlockChanger(const TabletSchema& tablet_schema) {
+RowBlockChanger::RowBlockChanger(const TabletSchema& tablet_schema, DescriptorTbl desc_tbl)
+        : _desc_tbl(desc_tbl) {
     _schema_mapping.resize(tablet_schema.num_columns());
 }
 
 RowBlockChanger::RowBlockChanger(const TabletSchema& tablet_schema,
-                                 const DeleteHandler* delete_handler) {
+                                 const DeleteHandler* delete_handler, DescriptorTbl desc_tbl)
+        : _desc_tbl(desc_tbl) {
     _schema_mapping.resize(tablet_schema.num_columns());
     _delete_handler = delete_handler;
 }
 
 RowBlockChanger::~RowBlockChanger() {
-    SchemaMapping::iterator it = _schema_mapping.begin();
-    for (; it != _schema_mapping.end(); ++it) {
+    for (auto it = _schema_mapping.begin(); it != _schema_mapping.end(); ++it) {
         SAFE_DELETE(it->default_value);

Review Comment:
   `_schema_mapping.default_value` seems be inited by `_parse_request(... RowBlockChanger* rb_changer ...`, so maybe we better delete it by whole vector.



-- 
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


[GitHub] [incubator-doris] BiteTheDDDDt commented on a diff in pull request #10003: [Feature] [Vectorized] Some pre-refactorings or interface additions for schema change part2

Posted by GitBox <gi...@apache.org>.
BiteTheDDDDt commented on code in PR #10003:
URL: https://github.com/apache/incubator-doris/pull/10003#discussion_r892419782


##########
be/src/olap/schema_change.h:
##########
@@ -70,6 +59,8 @@ class RowBlockChanger {
     // delete handler for filtering data which use specified in DELETE_DATA
     const DeleteHandler* _delete_handler = nullptr;
 
+    DescriptorTbl _desc_tbl;

Review Comment:
   It is used to vectorize schema_change,  will be used in later pr.



-- 
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


[GitHub] [incubator-doris] github-actions[bot] commented on pull request #10003: [Feature] [Vectorized] Some pre-refactorings or interface additions for schema change part2

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #10003:
URL: https://github.com/apache/incubator-doris/pull/10003#issuecomment-1156347680

   PR approved by at least one committer and no changes requested.


-- 
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