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 2020/03/02 19:29:44 UTC

[GitHub] [incubator-doris] liutang123 opened a new pull request #3025: Restructure storage type to support complex types expending

liutang123 opened a new pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025
 
 
   For #2885
   
   Change type to tree structure.
   Add List type.
   This PR is not ready to be review because UT should be added first.
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r393169733
 
 

 ##########
 File path: be/src/olap/column_vector.cpp
 ##########
 @@ -0,0 +1,150 @@
+// 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 "column_vector.h"
+
+namespace doris {
+
+ColumnVectorBatch::~ColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_null_signs);
+}
+
+Status ColumnVectorBatch::resize(size_t new_cap) {
+    if (_nullable) {
+        resize_buff(_capacity, new_cap, reinterpret_cast<uint8_t**>(&_null_signs));
+    }
+    _capacity = new_cap;
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::resize_buff(size_t copy_bytes, size_t new_cap, uint8_t** _buf_ptr) {
+    const uint8_t* old_buf = *_buf_ptr;
+    if (old_buf) {
+        *_buf_ptr = new uint8_t[new_cap];
+        memcpy(*_buf_ptr, old_buf, copy_bytes);
+        delete[] old_buf;
+    } else {
+        *_buf_ptr = new uint8_t[new_cap];
+    }
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::create(size_t init_capacity,
+        bool is_nullable,
+        const TypeInfo* type_info,
+        std::unique_ptr<ColumnVectorBatch>* column_vector_batch) {
+    if (is_scalar_type(type_info->type())) {
+        std::unique_ptr<ColumnVectorBatch> local(
+                new ScalarColumnVectorBatch(type_info, is_nullable, init_capacity));
+        *column_vector_batch = std::move(local);
+        return Status::OK();
+    } else {
+        switch (type_info->type()) {
+        case FieldType::OLAP_FIELD_TYPE_LIST: {
+            std::unique_ptr<ColumnVectorBatch> local(new ListColumnVectorBatch(type_info, is_nullable, init_capacity));
+            *column_vector_batch = std::move(local);
+            return Status::OK();
+        }
+        default:
+            return Status::NotSupported("unsupported type for ColumnVectorBatch: " + std::to_string(type_info->type()));
+        }
+    }
+}
+
+ScalarColumnVectorBatch::ScalarColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity)
+: ColumnVectorBatch(type_info, is_nullable) {
+    resize(init_capacity);
+}
+ScalarColumnVectorBatch::~ScalarColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_data);
+}
+
+Status ScalarColumnVectorBatch::resize(size_t new_cap) {
+    if (get_capacity() < new_cap) { // before first init, _capacity is 0.
+        size_t type_size = type_info()->size();
+        auto copy_bytes = get_capacity() * type_size;
+        size_t new_data_size = new_cap * type_size;
+        resize_buff(copy_bytes, new_data_size, &_data);
+        return ColumnVectorBatch::resize(new_cap);
+    }
+    return Status::OK();
+}
+
+uint8_t* ScalarColumnVectorBatch::data() const { return _data; };
+
+const uint8_t* ScalarColumnVectorBatch::cell_ptr(size_t idx) const { return _data + idx * type_info()->size(); };
+
+uint8_t* ScalarColumnVectorBatch::mutable_cell_ptr(size_t idx) const { return _data + idx * type_info()->size(); };
+
+
+ListColumnVectorBatch::ListColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity)
+: ColumnVectorBatch(type_info, is_nullable) {
+    auto list_type_info = reinterpret_cast<const ListTypeInfo*>(type_info);
+    ColumnVectorBatch::create(init_capacity * 2, true, list_type_info->item_type_info(), &_elements);
+    resize(init_capacity);
+}
+
+ListColumnVectorBatch::~ListColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_data);
+    SAFE_DELETE_ARRAY(_item_offsets);
+}
+
+Status ListColumnVectorBatch::resize(size_t new_cap) {
+    if (get_capacity() < new_cap) {
+        size_t collection_type_size = sizeof(collection);
+        size_t copy_bytes = get_capacity() * collection_type_size;
+        size_t new_data_size = new_cap * collection_type_size;
+        RETURN_IF_ERROR(resize_buff(copy_bytes, new_data_size, reinterpret_cast<uint8_t**>(&_data)));
+
+        size_t offset_type_size = sizeof(size_t);
+        size_t offset_copy_bytes = (get_capacity() + 1) * offset_type_size;
+        size_t new_offset_size = (new_cap + 1) * offset_type_size;
+        RETURN_IF_ERROR(resize_buff(offset_copy_bytes, new_offset_size, reinterpret_cast<uint8_t**>(&_item_offsets)));
+
+        RETURN_IF_ERROR(ColumnVectorBatch::resize(new_cap));
+    }
+    return Status::OK();
+}
+
+uint8_t* ListColumnVectorBatch::data() const { return reinterpret_cast<uint8*>(_data); }
+
+const uint8_t* ListColumnVectorBatch::cell_ptr(size_t idx) const { return reinterpret_cast<uint8*>(_data + idx); }
 
 Review comment:
   ```suggestion
   const uint8_t* ListColumnVectorBatch::cell_ptr(size_t idx) const { return reinterpret_cast<const uint8*>(_data + idx); }
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] liutang123 commented on issue #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
liutang123 commented on issue #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#issuecomment-596892650
 
 
   TODO: performance testing 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r394129988
 
 

 ##########
 File path: be/src/olap/column_vector.h
 ##########
 @@ -0,0 +1,150 @@
+// 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/status.h"
+#include "olap/olap_common.h"
+#include "olap/rowset/segment_v2/common.h" // for ordinal_t
+#include "olap/types.h"
+
+namespace doris {
+
+// struct that contains column data(null bitmap), data array in sub class.
+struct ColumnVectorBatch {
+public:
+    explicit ColumnVectorBatch(const TypeInfo* type_info, bool is_nullable)
+    : _type_info(type_info), _capacity(0), _nullable(is_nullable) {}
+
+    virtual ~ColumnVectorBatch();
+
+    const TypeInfo* type_info() const { return _type_info; }
+
+    size_t get_capacity() { return _capacity; }
+
+    inline bool is_nullable() {return _nullable; }
+
+    inline bool is_null_at(size_t row_idx) const {
+        return _nullable && _null_signs[row_idx];
+    }
+
+    inline void set_is_null(size_t idx, bool is_null) const {
+        if (_nullable) {
+            _null_signs[idx] = is_null;
+        }
+    }
+
+    inline void set_null_bits(size_t offset, size_t num_rows, bool val) const {
+        for (size_t i = 0; i < num_rows; ++i) {
+            set_is_null(offset + i, val);
+        }
+    }
+
+    inline bool* get_null_signs(size_t idx) { return _null_signs + idx; }
+
+
+    /**
+     * Change the number of slots to at least the given capacity.
+     * This function is not recursive into subtypes.
+     * Tips: This function will change `_capacity` attribute.
+     */
+    virtual Status resize(size_t new_cap);
+
+    // Get the start of the data.
+    virtual uint8_t* data() const = 0;
+
+    // Get the idx's cell_ptr
+    virtual const uint8_t* cell_ptr(size_t idx) const = 0;
+
+    // Get thr idx's cell_ptr for write
+    virtual uint8_t* mutable_cell_ptr(size_t idx) const = 0;
+
+
+    static Status create(size_t init_capacity,
+                         bool is_nullable,
+                         const TypeInfo* type_info,
+                         std::unique_ptr<ColumnVectorBatch>* column_vector_batch);
+
+protected:
+    static Status resize_buff(size_t old_cap, size_t new_cap, uint8_t** _buf);
+
+private:
+    const TypeInfo* _type_info;
+    size_t _capacity;
+    const bool _nullable;
+    bool* _null_signs = nullptr;
+};
+
+struct ScalarColumnVectorBatch : public ColumnVectorBatch {
+public:
+    explicit ScalarColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity);
+
+    ~ScalarColumnVectorBatch() override ;
+
+    Status resize(size_t new_cap) override;
+
+    // Get the start of the data.
+    uint8_t* data() const override;
+
+    // Get the idx's cell_ptr
+    const uint8_t* cell_ptr(size_t idx) const override;
+
+    // Get thr idx's cell_ptr for write
+    uint8_t* mutable_cell_ptr(size_t idx) const override;
+
+private:
+    uint8_t* _data = nullptr;
+};
+
+struct ListColumnVectorBatch : public ColumnVectorBatch {
+public:
+    explicit ListColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity);
+    ~ListColumnVectorBatch() override;
+    Status resize(size_t new_cap) override;
+
+    ColumnVectorBatch* get_elements() const { return _elements.get(); }
+
+    // Get the start of the data.
+    uint8_t* data() const override;
+
+    // Get the idx's cell_ptr
+    const uint8_t* cell_ptr(size_t idx) const override;
+
+    // Get thr idx's cell_ptr for write
+    uint8_t* mutable_cell_ptr(size_t idx) const override;
+
+    size_t item_offset(size_t idx) const;
+
+    // 将size个ordinals放入_item_offsets,从start_idx开始覆盖
+    // _item_offsets: 0 3 5 9, ordinals: 100 105 111, size: 3, satart_idx:3
+    // --> _item_offsets: 0 3 5 (9 + 100 - 100) (9 + 105 - 100) (9 + 111 - 100)
+    // 即 _item_offsets变成 0 3 5 9 14 20
+    void put_item_ordinal(segment_v2::ordinal_t* ordinals, size_t start_idx, size_t size);
+
+    // 将_item_offsets从start_idx到end_idx的部分转化成data
+    void transform_offsets_and_elements_to_data(size_t start_idx, size_t end_idx);
+
+private:
+    collection* _data = nullptr;
+
+    std::unique_ptr<ColumnVectorBatch> _elements;
+
+    // 用于存储每个数据在_elements中的位置
+    size_t* _item_offsets = nullptr;
+};
+
+} // namespace doris end
 
 Review comment:
   ```suggestion
   } // namespace doris
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r391552105
 
 

 ##########
 File path: be/src/olap/collection.h
 ##########
 @@ -0,0 +1,50 @@
+// 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 <iostream>
+
+namespace doris {
+
+// cpp type for LIST
+struct collection {
 
 Review comment:
   camel style name

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r392648505
 
 

 ##########
 File path: be/src/olap/rowset/segment_v2/page_decoder.h
 ##########
 @@ -81,13 +81,20 @@ class PageDecoder {
     // allocated in the column_vector_view's mem_pool.
     virtual Status next_batch(size_t* n, ColumnBlockView* dst) = 0;
 
+    // Same as `next_batch` except for not adding index
+    virtual Status peek_next_batch(size_t *n, ColumnBlockView* dst) {
 
 Review comment:
   In ListColumnReader, we want read one next row and do not want to change the cursor of page decoder.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r393160954
 
 

 ##########
 File path: be/src/olap/column_vector.h
 ##########
 @@ -0,0 +1,150 @@
+// 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/status.h"
+#include "olap/olap_common.h"
+#include "olap/rowset/segment_v2/common.h" // for ordinal_t
+#include "olap/types.h"
+
+namespace doris {
+
+// struct that contains column data(null bitmap), data array in sub class.
+struct ColumnVectorBatch {
+public:
+    explicit ColumnVectorBatch(const TypeInfo* type_info, bool is_nullable)
+    : _type_info(type_info), _capacity(0), _nullable(is_nullable) {}
+
+    virtual ~ColumnVectorBatch();
+
+    const TypeInfo* type_info() const { return _type_info; }
+
+    size_t get_capacity() { return _capacity; }
+
+    inline bool is_nullable() {return _nullable; }
+
+    inline bool is_null_at(size_t row_idx) const {
+        return _nullable && _null_signs[row_idx];
+    }
+
+    inline void set_is_null(size_t idx, bool is_null) const {
+        if (_nullable) {
+            _null_signs[idx] = is_null;
+        }
+    }
+
+    inline void set_null_bits(size_t offset, size_t num_rows, bool val) const {
+        for (size_t i = 0; i < num_rows; ++i) {
+            set_is_null(offset + i, val);
+        }
+    }
+
+    inline bool* get_null_signs(size_t idx) { return _null_signs + idx; }
+
+
+    /**
+     * Change the number of slots to at least the given capacity.
+     * This function is not recursive into subtypes.
+     * Tips: This function will change `_capacity` attribute.
+     */
+    virtual Status resize(size_t new_cap);
+
+    // Get the start of the data.
+    virtual uint8_t* data() const = 0;
+
+    // Get the idx's cell_ptr
+    virtual const uint8_t* cell_ptr(size_t idx) const = 0;
+
+    // Get thr idx's cell_ptr for write
+    virtual uint8_t* mutable_cell_ptr(size_t idx) const = 0;
+
+
+    static Status create(size_t init_capacity,
+                         bool is_nullable,
+                         const TypeInfo* type_info,
+                         std::unique_ptr<ColumnVectorBatch>* column_vector_batch);
+
+protected:
+    static Status resize_buff(size_t old_cap, size_t new_cap, uint8_t** _buf);
+
+private:
+    const TypeInfo* _type_info;
+    size_t _capacity;
+    const bool _nullable;
+    bool* _null_signs = nullptr;
+};
+
+struct ScalarColumnVectorBatch : public ColumnVectorBatch {
+public:
+    explicit ScalarColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity);
+
+    ~ScalarColumnVectorBatch() override ;
+
+    Status resize(size_t new_cap) override;
+
+    // Get the start of the data.
+    uint8_t* data() const override;
+
+    // Get the idx's cell_ptr
+    const uint8_t* cell_ptr(size_t idx) const override;
+
+    // Get thr idx's cell_ptr for write
+    uint8_t* mutable_cell_ptr(size_t idx) const override;
+
+private:
+    uint8_t* _data = nullptr;
+};
+
+struct ListColumnVectorBatch : public ColumnVectorBatch {
+public:
+    explicit ListColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity);
+    ~ListColumnVectorBatch() override;
+    Status resize(size_t new_cap) override;
+
+    ColumnVectorBatch* get_elements() const { return _elements.get(); }
+
+    // Get the start of the data.
+    uint8_t* data() const override;
+
+    // Get the idx's cell_ptr
+    const uint8_t* cell_ptr(size_t idx) const override;
+
+    // Get thr idx's cell_ptr for write
+    uint8_t* mutable_cell_ptr(size_t idx) const override;
+
+    size_t item_offset(size_t idx) const;
+
+    // 将size个ordinals放入_item_offsets,从start_idx开始覆盖
+    // _item_offsets: 0 3 5 9, ordinals: 100 105 111, size: 3, satart_idx:3
+    // --> _item_offsets: 0 3 5 (9 + 100 - 100) (9 + 105 - 100) (9 + 111 - 100)
+    // 即 _item_offsets变成 0 3 5 9 14 20
+    void put_item_ordinal(segment_v2::ordinal_t* ordinals, size_t start_idx, size_t size);
+
+    // 将_item_offsets从start_idx到end_idx的部分转化成data
 
 Review comment:
   english

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] liutang123 edited a comment on issue #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
liutang123 edited a comment on issue #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#issuecomment-596221473
 
 
   @gaodayue 
   cc  @imay @Seaven 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r394126344
 
 

 ##########
 File path: be/src/olap/column_vector.cpp
 ##########
 @@ -0,0 +1,150 @@
+// 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 "column_vector.h"
+
+namespace doris {
+
+ColumnVectorBatch::~ColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_null_signs);
+}
+
+Status ColumnVectorBatch::resize(size_t new_cap) {
+    if (_nullable) {
 
 Review comment:
   +1 better to reallocate memory only when new_cap is larger than current cap

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r394094052
 
 

 ##########
 File path: be/src/olap/column_vector.h
 ##########
 @@ -0,0 +1,150 @@
+// 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/status.h"
+#include "olap/olap_common.h"
+#include "olap/rowset/segment_v2/common.h" // for ordinal_t
+#include "olap/types.h"
+
+namespace doris {
+
+// struct that contains column data(null bitmap), data array in sub class.
+struct ColumnVectorBatch {
 
 Review comment:
   ```suggestion
   class ColumnVectorBatch {
   ```
   We use `struct` only when all fields are public

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r393160403
 
 

 ##########
 File path: be/src/olap/column_vector.h
 ##########
 @@ -0,0 +1,150 @@
+// 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/status.h"
+#include "olap/olap_common.h"
+#include "olap/rowset/segment_v2/common.h" // for ordinal_t
+#include "olap/types.h"
+
+namespace doris {
+
+// struct that contains column data(null bitmap), data array in sub class.
+struct ColumnVectorBatch {
+public:
+    explicit ColumnVectorBatch(const TypeInfo* type_info, bool is_nullable)
+    : _type_info(type_info), _capacity(0), _nullable(is_nullable) {}
+
+    virtual ~ColumnVectorBatch();
+
+    const TypeInfo* type_info() const { return _type_info; }
+
+    size_t get_capacity() { return _capacity; }
+
+    inline bool is_nullable() {return _nullable; }
+
+    inline bool is_null_at(size_t row_idx) const {
+        return _nullable && _null_signs[row_idx];
+    }
+
+    inline void set_is_null(size_t idx, bool is_null) const {
+        if (_nullable) {
+            _null_signs[idx] = is_null;
+        }
+    }
+
+    inline void set_null_bits(size_t offset, size_t num_rows, bool val) const {
+        for (size_t i = 0; i < num_rows; ++i) {
+            set_is_null(offset + i, val);
+        }
+    }
+
+    inline bool* get_null_signs(size_t idx) { return _null_signs + idx; }
+
+
+    /**
+     * Change the number of slots to at least the given capacity.
+     * This function is not recursive into subtypes.
+     * Tips: This function will change `_capacity` attribute.
+     */
+    virtual Status resize(size_t new_cap);
+
+    // Get the start of the data.
+    virtual uint8_t* data() const = 0;
+
+    // Get the idx's cell_ptr
+    virtual const uint8_t* cell_ptr(size_t idx) const = 0;
+
+    // Get thr idx's cell_ptr for write
+    virtual uint8_t* mutable_cell_ptr(size_t idx) const = 0;
+
+
+    static Status create(size_t init_capacity,
+                         bool is_nullable,
+                         const TypeInfo* type_info,
+                         std::unique_ptr<ColumnVectorBatch>* column_vector_batch);
+
+protected:
+    static Status resize_buff(size_t old_cap, size_t new_cap, uint8_t** _buf);
+
+private:
+    const TypeInfo* _type_info;
+    size_t _capacity;
+    const bool _nullable;
+    bool* _null_signs = nullptr;
+};
+
+struct ScalarColumnVectorBatch : public ColumnVectorBatch {
+public:
+    explicit ScalarColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity);
+
+    ~ScalarColumnVectorBatch() override ;
+
+    Status resize(size_t new_cap) override;
+
+    // Get the start of the data.
+    uint8_t* data() const override;
+
+    // Get the idx's cell_ptr
+    const uint8_t* cell_ptr(size_t idx) const override;
+
+    // Get thr idx's cell_ptr for write
+    uint8_t* mutable_cell_ptr(size_t idx) const override;
+
+private:
+    uint8_t* _data = nullptr;
+};
+
+struct ListColumnVectorBatch : public ColumnVectorBatch {
+public:
+    explicit ListColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity);
+    ~ListColumnVectorBatch() override;
+    Status resize(size_t new_cap) override;
+
+    ColumnVectorBatch* get_elements() const { return _elements.get(); }
+
+    // Get the start of the data.
+    uint8_t* data() const override;
+
+    // Get the idx's cell_ptr
+    const uint8_t* cell_ptr(size_t idx) const override;
+
+    // Get thr idx's cell_ptr for write
+    uint8_t* mutable_cell_ptr(size_t idx) const override;
+
+    size_t item_offset(size_t idx) const;
+
+    // 将size个ordinals放入_item_offsets,从start_idx开始覆盖
+    // _item_offsets: 0 3 5 9, ordinals: 100 105 111, size: 3, satart_idx:3
 
 Review comment:
   ```suggestion
       // _item_offsets: 0 3 5 9, ordinals: 100 105 111, size: 3, start_idx:3
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r392766603
 
 

 ##########
 File path: be/src/olap/rowset/segment_v2/column_reader.cpp
 ##########
 @@ -290,6 +316,109 @@ Status ColumnReader::seek_at_or_before(ordinal_t ordinal, OrdinalPageIndexIterat
     return Status::OK();
 }
 
+////////////////////////////////////////////////////////////////////////////////
+
+ListColumnReader::~ListColumnReader() = default;
+
+Status ListColumnReader::new_iterator(ColumnIterator** iterator) {
+    ColumnIterator* item_iterator;
+    _item_reader->new_iterator(&item_iterator);
+    *iterator = new ListFileColumnIterator(this, item_iterator);
+    return Status::OK();
+}
+
+TypeInfo* ListColumnReader::get_type_info_for_read() {
+    return get_scalar_type_info(FieldType::OLAP_FIELD_TYPE_BIGINT);
+}
+
+ListFileColumnIterator::ListFileColumnIterator(ColumnReader* offset_reader, ColumnIterator* item_reader)
+: FileColumnIterator(offset_reader) {
+    _item_iterator.reset(item_reader);
+}
+
+ListFileColumnIterator::~ListFileColumnIterator() = default;
+
+Status ListFileColumnIterator::init(const ColumnIteratorOptions& opts) {
+    RETURN_IF_ERROR(FileColumnIterator::init(opts));
+    TypeInfo* bigint_type_info = get_scalar_type_info(FieldType::OLAP_FIELD_TYPE_BIGINT);
+    RETURN_IF_ERROR(ColumnVectorBatch::create(1024, true, bigint_type_info, &_offset_batch));
+    RETURN_IF_ERROR(_item_iterator->init(opts));
+    return Status::OK();
+}
+
+// every invoke this method, _offset_batch will be cover, so this method is not thread safe.
+Status ListFileColumnIterator::next_batch(size_t* n, ColumnBlockView* dst) {
+    // 1. read offsets into  _offset_batch;
+    _offset_batch->resize(*n + 1);
+    ColumnBlock ordinal_block(_offset_batch.get(), nullptr);
+    ColumnBlockView ordinal_view(&ordinal_block);
+    RETURN_IF_ERROR(FileColumnIterator::next_batch(n, &ordinal_view));
+
+    if (*n == 0) {
+        return Status::OK();
+    }
+
+    // 2. 读取最后一个ordinal
+    if (_page->data_decoder->has_remaining()) { // not _page->has_remaining()
+        size_t i = 1;
 
 Review comment:
   OK. Please add more comment.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r394125476
 
 

 ##########
 File path: be/src/olap/column_vector.cpp
 ##########
 @@ -0,0 +1,150 @@
+// 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 "column_vector.h"
+
+namespace doris {
+
+ColumnVectorBatch::~ColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_null_signs);
+}
+
+Status ColumnVectorBatch::resize(size_t new_cap) {
+    if (_nullable) {
+        resize_buff(_capacity, new_cap, reinterpret_cast<uint8_t**>(&_null_signs));
+    }
+    _capacity = new_cap;
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::resize_buff(size_t copy_bytes, size_t new_cap, uint8_t** _buf_ptr) {
+    const uint8_t* old_buf = *_buf_ptr;
+    if (old_buf) {
+        *_buf_ptr = new uint8_t[new_cap];
+        memcpy(*_buf_ptr, old_buf, copy_bytes);
+        delete[] old_buf;
+    } else {
+        *_buf_ptr = new uint8_t[new_cap];
+    }
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::create(size_t init_capacity,
+        bool is_nullable,
+        const TypeInfo* type_info,
+        std::unique_ptr<ColumnVectorBatch>* column_vector_batch) {
+    if (is_scalar_type(type_info->type())) {
+        std::unique_ptr<ColumnVectorBatch> local(
+                new ScalarColumnVectorBatch(type_info, is_nullable, init_capacity));
+        *column_vector_batch = std::move(local);
+        return Status::OK();
+    } else {
+        switch (type_info->type()) {
+        case FieldType::OLAP_FIELD_TYPE_LIST: {
+            std::unique_ptr<ColumnVectorBatch> local(new ListColumnVectorBatch(type_info, is_nullable, init_capacity));
+            *column_vector_batch = std::move(local);
 
 Review comment:
   column_vector_batch->reset(new ListColumnVectorBatch(type_info, is_nullable, init_capacity));

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r395066907
 
 

 ##########
 File path: be/src/olap/row_block2.cpp
 ##########
 @@ -63,7 +54,7 @@ Status RowBlockV2::convert_to_row_block(RowCursor* helper, RowBlock* dst) {
             for (uint16_t i = 0; i < _selected_size; ++i) {
                 uint16_t row_idx = _selection_vector[i];
                 dst->get_row(i, helper);
-                bool is_null = BitmapTest(_column_null_bitmaps[cid], row_idx);
+                bool is_null = column_block(cid).is_null(row_idx);
 
 Review comment:
   You create too many calls of `column_block(cid)` which will hurt performance because each invocation will create a ColumnBlock object. Better to get `ColumnVectorBatch*` before the for loop and use that pointer to do the work.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r394478297
 
 

 ##########
 File path: be/src/olap/column_vector.cpp
 ##########
 @@ -0,0 +1,150 @@
+// 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 "column_vector.h"
+
+namespace doris {
+
+ColumnVectorBatch::~ColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_null_signs);
+}
+
+Status ColumnVectorBatch::resize(size_t new_cap) {
+    if (_nullable) {
 
 Review comment:
   Add this logic in `ScalarColumnVectorBatch` and `ListColumnVectorBatch`.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r395092121
 
 

 ##########
 File path: gensrc/proto/olap_file.proto
 ##########
 @@ -265,6 +265,8 @@ message ColumnPB {
     optional int32 referenced_column_id = 13; //   
     optional string referenced_column = 14; // ColumnMessage.referenced_column?
     optional bool has_bitmap_index = 15 [default=false]; // ColumnMessage.has_bitmap_index
+    repeated ColumnPB children_columns = 16;
+    repeated string children_column_names = 17;
 
 Review comment:
   Why we need `children_column_names`?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r392283832
 
 

 ##########
 File path: be/src/olap/rowset/segment_v2/page_decoder.h
 ##########
 @@ -81,13 +81,20 @@ class PageDecoder {
     // allocated in the column_vector_view's mem_pool.
     virtual Status next_batch(size_t* n, ColumnBlockView* dst) = 0;
 
+    // Same as `next_batch` except for not adding index
+    virtual Status peek_next_batch(size_t *n, ColumnBlockView* dst) {
 
 Review comment:
   Why need this interface?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r394129876
 
 

 ##########
 File path: be/src/olap/column_vector.h
 ##########
 @@ -0,0 +1,150 @@
+// 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/status.h"
+#include "olap/olap_common.h"
+#include "olap/rowset/segment_v2/common.h" // for ordinal_t
+#include "olap/types.h"
+
+namespace doris {
+
+// struct that contains column data(null bitmap), data array in sub class.
+struct ColumnVectorBatch {
+public:
+    explicit ColumnVectorBatch(const TypeInfo* type_info, bool is_nullable)
+    : _type_info(type_info), _capacity(0), _nullable(is_nullable) {}
+
+    virtual ~ColumnVectorBatch();
+
+    const TypeInfo* type_info() const { return _type_info; }
+
+    size_t get_capacity() { return _capacity; }
+
+    inline bool is_nullable() {return _nullable; }
+
+    inline bool is_null_at(size_t row_idx) const {
+        return _nullable && _null_signs[row_idx];
+    }
+
+    inline void set_is_null(size_t idx, bool is_null) const {
+        if (_nullable) {
+            _null_signs[idx] = is_null;
+        }
+    }
+
+    inline void set_null_bits(size_t offset, size_t num_rows, bool val) const {
+        for (size_t i = 0; i < num_rows; ++i) {
+            set_is_null(offset + i, val);
+        }
+    }
+
+    inline bool* get_null_signs(size_t idx) { return _null_signs + idx; }
+
+
+    /**
+     * Change the number of slots to at least the given capacity.
+     * This function is not recursive into subtypes.
+     * Tips: This function will change `_capacity` attribute.
+     */
+    virtual Status resize(size_t new_cap);
+
+    // Get the start of the data.
+    virtual uint8_t* data() const = 0;
+
+    // Get the idx's cell_ptr
+    virtual const uint8_t* cell_ptr(size_t idx) const = 0;
+
+    // Get thr idx's cell_ptr for write
+    virtual uint8_t* mutable_cell_ptr(size_t idx) const = 0;
+
+
+    static Status create(size_t init_capacity,
+                         bool is_nullable,
+                         const TypeInfo* type_info,
+                         std::unique_ptr<ColumnVectorBatch>* column_vector_batch);
+
+protected:
+    static Status resize_buff(size_t old_cap, size_t new_cap, uint8_t** _buf);
+
+private:
+    const TypeInfo* _type_info;
+    size_t _capacity;
+    const bool _nullable;
+    bool* _null_signs = nullptr;
+};
+
+struct ScalarColumnVectorBatch : public ColumnVectorBatch {
+public:
+    explicit ScalarColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity);
+
+    ~ScalarColumnVectorBatch() override ;
+
+    Status resize(size_t new_cap) override;
+
+    // Get the start of the data.
+    uint8_t* data() const override;
+
+    // Get the idx's cell_ptr
+    const uint8_t* cell_ptr(size_t idx) const override;
+
+    // Get thr idx's cell_ptr for write
+    uint8_t* mutable_cell_ptr(size_t idx) const override;
+
+private:
+    uint8_t* _data = nullptr;
+};
+
+struct ListColumnVectorBatch : public ColumnVectorBatch {
+public:
+    explicit ListColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity);
+    ~ListColumnVectorBatch() override;
+    Status resize(size_t new_cap) override;
+
+    ColumnVectorBatch* get_elements() const { return _elements.get(); }
+
+    // Get the start of the data.
+    uint8_t* data() const override;
+
+    // Get the idx's cell_ptr
+    const uint8_t* cell_ptr(size_t idx) const override;
+
+    // Get thr idx's cell_ptr for write
+    uint8_t* mutable_cell_ptr(size_t idx) const override;
+
+    size_t item_offset(size_t idx) const;
+
+    // 将size个ordinals放入_item_offsets,从start_idx开始覆盖
+    // _item_offsets: 0 3 5 9, ordinals: 100 105 111, size: 3, satart_idx:3
+    // --> _item_offsets: 0 3 5 (9 + 100 - 100) (9 + 105 - 100) (9 + 111 - 100)
+    // 即 _item_offsets变成 0 3 5 9 14 20
+    void put_item_ordinal(segment_v2::ordinal_t* ordinals, size_t start_idx, size_t size);
+
+    // 将_item_offsets从start_idx到end_idx的部分转化成data
+    void transform_offsets_and_elements_to_data(size_t start_idx, size_t end_idx);
+
+private:
+    collection* _data = nullptr;
+
+    std::unique_ptr<ColumnVectorBatch> _elements;
+
+    // 用于存储每个数据在_elements中的位置
+    size_t* _item_offsets = nullptr;
 
 Review comment:
   ```suggestion
       int64_t* _item_offsets = nullptr;
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r394095554
 
 

 ##########
 File path: be/src/olap/column_vector.h
 ##########
 @@ -0,0 +1,150 @@
+// 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/status.h"
+#include "olap/olap_common.h"
+#include "olap/rowset/segment_v2/common.h" // for ordinal_t
+#include "olap/types.h"
+
+namespace doris {
+
+// struct that contains column data(null bitmap), data array in sub class.
+struct ColumnVectorBatch {
+public:
+    explicit ColumnVectorBatch(const TypeInfo* type_info, bool is_nullable)
+    : _type_info(type_info), _capacity(0), _nullable(is_nullable) {}
+
+    virtual ~ColumnVectorBatch();
+
+    const TypeInfo* type_info() const { return _type_info; }
+
+    size_t get_capacity() { return _capacity; }
+
+    inline bool is_nullable() {return _nullable; }
 
 Review comment:
   ```suggestion
       bool is_nullable() const { return _nullable; }
   ```
   when method is defined inside class definition, inline can be ignored

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r392648375
 
 

 ##########
 File path: be/src/olap/rowset/segment_v2/column_writer.cpp
 ##########
 @@ -361,5 +430,106 @@ Status ColumnWriter::_finish_current_page() {
     return Status::OK();
 }
 
+////////////////////////////////////////////////////////////////////////////////
+
+ListColumnWriter::ListColumnWriter(const ColumnWriterOptions& opts,
+                         std::unique_ptr<Field> field,
+                         WritableFile* output_file,
+                         std::unique_ptr<ColumnWriter> item_writer):
+        ColumnWriter(opts, std::move(field), output_file), _item_writer(std::move(item_writer)) {}
+
+Status ListColumnWriter::create_page_builder(PageBuilder** page_builder) {
+    TypeInfo* bigint_type_info = get_scalar_type_info(FieldType::OLAP_FIELD_TYPE_BIGINT);
+    RETURN_IF_ERROR(EncodingInfo::get(bigint_type_info, _opts.meta->encoding(), &_encoding_info));
+
+    PageBuilder* local = nullptr;
+    PageBuilderOptions opts;
+    opts.data_page_size = _opts.data_page_size;
+    _opts.meta->set_encoding(_encoding_info->encoding());
+
+    RETURN_IF_ERROR(_encoding_info->create_page_builder(opts, &local));
+    if (local == nullptr) {
+        return Status::NotSupported(
+                Substitute("Failed to create page builder for type LIST encoding $0",
+                           _opts.meta->encoding()));
+    } else {
+        *page_builder = local;
+        return Status::OK();
+    }
+}
+ListColumnWriter::~ListColumnWriter() = default;
+
+Status ListColumnWriter::init() {
+    if (_opts.need_zone_map) {
+        return Status::NotSupported("unsupported zone map for list");
+    }
+
+    if (_opts.need_bitmap_index) {
+        return Status::NotSupported("unsupported bitmap for list");
+    }
+
+    if (_opts.need_bloom_filter) {
+        return Status::NotSupported("unsupported bloom filter for list");
+    }
+
+    RETURN_IF_ERROR(ColumnWriter::init());
+    RETURN_IF_ERROR(_item_writer->init());
+    return Status::OK();
+}
+
+Status ListColumnWriter::put_page_footer_info(DataPageFooterPB* footer) {
+    footer->set_next_array_item_ordinal(_next_item_ordinal);
+    return Status::OK();
 }
+
+// Now we can only write data one by one.
+Status ListColumnWriter::_append_data(const uint8_t** ptr, size_t num_rows) {
+    size_t remaining = num_rows;
+    const auto* col_cursor = reinterpret_cast<const collection*>(*ptr);
+    while (remaining > 0) {
+        size_t num_written = 1;
 
 Review comment:
   Because we do not known the length of each array.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r394128408
 
 

 ##########
 File path: be/src/olap/collection.h
 ##########
 @@ -0,0 +1,50 @@
+// 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 <iostream>
+
+namespace doris {
+
+// cpp type for LIST
+struct collection {
+    size_t length;
 
 Review comment:
   I think we don't need to support 2^64 elements, uint32_t is sufficient

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r393145148
 
 

 ##########
 File path: be/src/olap/collection.h
 ##########
 @@ -0,0 +1,50 @@
+// 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 <iostream>
+
+namespace doris {
+
+// cpp type for LIST
+struct collection {
+    size_t length;
+    // null bitmap
+    bool* null_signs;
+    // 子元素数据数据
+    void* data;
+
+    collection(): length(0), null_signs(nullptr), data(nullptr) {}
+
+    explicit collection(size_t length) : length(length), null_signs(nullptr), data(nullptr) {}
+
+    collection(void* data, size_t length ) : length(length), null_signs(nullptr), data(data) {}
+
+    collection(void* data, size_t length, bool* null_signs)
+    : length(length), null_signs(null_signs), data(data) {}
+
+    bool operator==(const collection& y) const ;
+    bool operator!=(const collection& value) const ;
+    bool operator<(const collection& value) const ;
+    bool operator<=(const collection& value) const ;
+    bool operator>(const collection& value) const ;
 
 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r394126601
 
 

 ##########
 File path: be/src/olap/column_vector.cpp
 ##########
 @@ -0,0 +1,150 @@
+// 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 "column_vector.h"
+
+namespace doris {
+
+ColumnVectorBatch::~ColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_null_signs);
+}
+
+Status ColumnVectorBatch::resize(size_t new_cap) {
+    if (_nullable) {
+        resize_buff(_capacity, new_cap, reinterpret_cast<uint8_t**>(&_null_signs));
+    }
+    _capacity = new_cap;
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::resize_buff(size_t copy_bytes, size_t new_cap, uint8_t** _buf_ptr) {
+    const uint8_t* old_buf = *_buf_ptr;
+    if (old_buf) {
+        *_buf_ptr = new uint8_t[new_cap];
+        memcpy(*_buf_ptr, old_buf, copy_bytes);
+        delete[] old_buf;
+    } else {
+        *_buf_ptr = new uint8_t[new_cap];
+    }
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::create(size_t init_capacity,
+        bool is_nullable,
+        const TypeInfo* type_info,
+        std::unique_ptr<ColumnVectorBatch>* column_vector_batch) {
+    if (is_scalar_type(type_info->type())) {
+        std::unique_ptr<ColumnVectorBatch> local(
+                new ScalarColumnVectorBatch(type_info, is_nullable, init_capacity));
+        *column_vector_batch = std::move(local);
+        return Status::OK();
+    } else {
+        switch (type_info->type()) {
+        case FieldType::OLAP_FIELD_TYPE_LIST: {
+            std::unique_ptr<ColumnVectorBatch> local(new ListColumnVectorBatch(type_info, is_nullable, init_capacity));
+            *column_vector_batch = std::move(local);
+            return Status::OK();
+        }
+        default:
+            return Status::NotSupported("unsupported type for ColumnVectorBatch: " + std::to_string(type_info->type()));
+        }
+    }
+}
+
+ScalarColumnVectorBatch::ScalarColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity)
+: ColumnVectorBatch(type_info, is_nullable) {
+    resize(init_capacity);
 
 Review comment:
   Avoid calling virtual function in constructor/destructor, see https://wiki.sei.cmu.edu/confluence/display/cplusplus/OOP50-CPP.+Do+not+invoke+virtual+functions+from+constructors+or+destructors and https://scc.ustc.edu.cn/zlsc/sugon/intel/ssadiag_docs/pt_reference/references/sc_cpp_virtual_call_in_ctor.htm

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r395127606
 
 

 ##########
 File path: be/src/olap/rowset/segment_v2/column_reader.h
 ##########
 @@ -187,6 +196,25 @@ class ColumnReader {
     std::unique_ptr<BloomFilterIndexReader> _bloom_filter_index;
 };
 
+class ArrayColumnReader : public ColumnReader {
 
 Review comment:
   It's a bad class design to make ArrayColumnReader directly inheriting from ColumnReader. You'd better start with a ColumnReader interface and design ScalarColumnReader and ArrayColumnReader separately.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r394095206
 
 

 ##########
 File path: be/src/olap/column_vector.h
 ##########
 @@ -0,0 +1,150 @@
+// 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/status.h"
+#include "olap/olap_common.h"
+#include "olap/rowset/segment_v2/common.h" // for ordinal_t
+#include "olap/types.h"
+
+namespace doris {
+
+// struct that contains column data(null bitmap), data array in sub class.
+struct ColumnVectorBatch {
+public:
+    explicit ColumnVectorBatch(const TypeInfo* type_info, bool is_nullable)
+    : _type_info(type_info), _capacity(0), _nullable(is_nullable) {}
+
+    virtual ~ColumnVectorBatch();
+
+    const TypeInfo* type_info() const { return _type_info; }
+
+    size_t get_capacity() { return _capacity; }
 
 Review comment:
   ```suggestion
       size_t capacity() const { return _capacity; }
   ```
   For accessor methods, we don't use "get_" in name

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r392648264
 
 

 ##########
 File path: be/src/olap/rowset/segment_v2/column_reader.cpp
 ##########
 @@ -290,6 +316,109 @@ Status ColumnReader::seek_at_or_before(ordinal_t ordinal, OrdinalPageIndexIterat
     return Status::OK();
 }
 
+////////////////////////////////////////////////////////////////////////////////
+
+ListColumnReader::~ListColumnReader() = default;
+
+Status ListColumnReader::new_iterator(ColumnIterator** iterator) {
+    ColumnIterator* item_iterator;
+    _item_reader->new_iterator(&item_iterator);
+    *iterator = new ListFileColumnIterator(this, item_iterator);
+    return Status::OK();
+}
+
+TypeInfo* ListColumnReader::get_type_info_for_read() {
+    return get_scalar_type_info(FieldType::OLAP_FIELD_TYPE_BIGINT);
+}
+
+ListFileColumnIterator::ListFileColumnIterator(ColumnReader* offset_reader, ColumnIterator* item_reader)
+: FileColumnIterator(offset_reader) {
+    _item_iterator.reset(item_reader);
+}
+
+ListFileColumnIterator::~ListFileColumnIterator() = default;
+
+Status ListFileColumnIterator::init(const ColumnIteratorOptions& opts) {
+    RETURN_IF_ERROR(FileColumnIterator::init(opts));
+    TypeInfo* bigint_type_info = get_scalar_type_info(FieldType::OLAP_FIELD_TYPE_BIGINT);
+    RETURN_IF_ERROR(ColumnVectorBatch::create(1024, true, bigint_type_info, &_offset_batch));
+    RETURN_IF_ERROR(_item_iterator->init(opts));
+    return Status::OK();
+}
+
+// every invoke this method, _offset_batch will be cover, so this method is not thread safe.
+Status ListFileColumnIterator::next_batch(size_t* n, ColumnBlockView* dst) {
+    // 1. read offsets into  _offset_batch;
+    _offset_batch->resize(*n + 1);
+    ColumnBlock ordinal_block(_offset_batch.get(), nullptr);
+    ColumnBlockView ordinal_view(&ordinal_block);
+    RETURN_IF_ERROR(FileColumnIterator::next_batch(n, &ordinal_view));
+
+    if (*n == 0) {
+        return Status::OK();
+    }
+
+    // 2. 读取最后一个ordinal
+    if (_page->data_decoder->has_remaining()) { // not _page->has_remaining()
+        size_t i = 1;
 
 Review comment:
   Because in  step 1, we read n rows and we should read n + 1 rows.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r392766691
 
 

 ##########
 File path: be/src/olap/rowset/segment_v2/page_decoder.h
 ##########
 @@ -81,13 +81,20 @@ class PageDecoder {
     // allocated in the column_vector_view's mem_pool.
     virtual Status next_batch(size_t* n, ColumnBlockView* dst) = 0;
 
+    // Same as `next_batch` except for not adding index
+    virtual Status peek_next_batch(size_t *n, ColumnBlockView* dst) {
 
 Review comment:
   OK. Please add more comment.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r392284891
 
 

 ##########
 File path: be/src/olap/rowset/segment_v2/bitshuffle_page.h
 ##########
 @@ -334,6 +334,10 @@ class BitShufflePageDecoder : public PageDecoder {
     }
 
     Status next_batch(size_t* n, ColumnBlockView* dst) override {
+        return next_batch(n, dst, true);
+    }
+
+    Status next_batch(size_t* n, ColumnBlockView* dst, bool forward_index) {
 
 Review comment:
   For performance, we could use function template.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r395092919
 
 

 ##########
 File path: gensrc/proto/segment_v2.proto
 ##########
 @@ -151,6 +151,12 @@ message ColumnMetaPB {
     repeated ColumnIndexMetaPB indexes = 8;
     // pointer to dictionary page when using DICT_ENCODING
     optional PagePointerPB dict_page = 9;
+
+    repeated ColumnMetaPB children_columns = 10;
+    repeated string children_clolumn_names = 11;
 
 Review comment:
   Why we need this? Btw, `clolumn` is a typo

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r395095068
 
 

 ##########
 File path: gensrc/proto/segment_v2.proto
 ##########
 @@ -151,6 +151,12 @@ message ColumnMetaPB {
     repeated ColumnIndexMetaPB indexes = 8;
     // pointer to dictionary page when using DICT_ENCODING
     optional PagePointerPB dict_page = 9;
+
+    repeated ColumnMetaPB children_columns = 10;
+    repeated string children_clolumn_names = 11;
+
+    // required
+    optional uint64 num_rows = 12;
 
 Review comment:
   Add a comment about why we need this. In addition, perhaps num_values is better name?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r392263781
 
 

 ##########
 File path: be/src/olap/rowset/segment_v2/column_reader.cpp
 ##########
 @@ -290,6 +316,109 @@ Status ColumnReader::seek_at_or_before(ordinal_t ordinal, OrdinalPageIndexIterat
     return Status::OK();
 }
 
+////////////////////////////////////////////////////////////////////////////////
+
+ListColumnReader::~ListColumnReader() = default;
+
+Status ListColumnReader::new_iterator(ColumnIterator** iterator) {
+    ColumnIterator* item_iterator;
+    _item_reader->new_iterator(&item_iterator);
+    *iterator = new ListFileColumnIterator(this, item_iterator);
+    return Status::OK();
+}
+
+TypeInfo* ListColumnReader::get_type_info_for_read() {
+    return get_scalar_type_info(FieldType::OLAP_FIELD_TYPE_BIGINT);
+}
+
+ListFileColumnIterator::ListFileColumnIterator(ColumnReader* offset_reader, ColumnIterator* item_reader)
+: FileColumnIterator(offset_reader) {
+    _item_iterator.reset(item_reader);
+}
+
+ListFileColumnIterator::~ListFileColumnIterator() = default;
+
+Status ListFileColumnIterator::init(const ColumnIteratorOptions& opts) {
+    RETURN_IF_ERROR(FileColumnIterator::init(opts));
+    TypeInfo* bigint_type_info = get_scalar_type_info(FieldType::OLAP_FIELD_TYPE_BIGINT);
+    RETURN_IF_ERROR(ColumnVectorBatch::create(1024, true, bigint_type_info, &_offset_batch));
+    RETURN_IF_ERROR(_item_iterator->init(opts));
+    return Status::OK();
+}
+
+// every invoke this method, _offset_batch will be cover, so this method is not thread safe.
+Status ListFileColumnIterator::next_batch(size_t* n, ColumnBlockView* dst) {
+    // 1. read offsets into  _offset_batch;
+    _offset_batch->resize(*n + 1);
+    ColumnBlock ordinal_block(_offset_batch.get(), nullptr);
+    ColumnBlockView ordinal_view(&ordinal_block);
+    RETURN_IF_ERROR(FileColumnIterator::next_batch(n, &ordinal_view));
+
+    if (*n == 0) {
+        return Status::OK();
+    }
+
+    // 2. 读取最后一个ordinal
+    if (_page->data_decoder->has_remaining()) { // not _page->has_remaining()
+        size_t i = 1;
 
 Review comment:
   Why this is 1? 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r393145019
 
 

 ##########
 File path: be/src/olap/collection.h
 ##########
 @@ -0,0 +1,50 @@
+// 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 <iostream>
+
+namespace doris {
+
+// cpp type for LIST
+struct collection {
+    size_t length;
+    // null bitmap
+    bool* null_signs;
+    // 子元素数据数据
+    void* data;
+
+    collection(): length(0), null_signs(nullptr), data(nullptr) {}
+
+    explicit collection(size_t length) : length(length), null_signs(nullptr), data(nullptr) {}
+
+    collection(void* data, size_t length ) : length(length), null_signs(nullptr), data(data) {}
+
+    collection(void* data, size_t length, bool* null_signs)
+    : length(length), null_signs(null_signs), data(data) {}
+
+    bool operator==(const collection& y) const ;
 
 Review comment:
   ```suggestion
       bool operator==(const collection& y) const;
   ```
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r392256175
 
 

 ##########
 File path: be/src/olap/column_vector.cpp
 ##########
 @@ -0,0 +1,150 @@
+// 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 "column_vector.h"
+
+namespace doris {
+
+ColumnVectorBatch::~ColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_null_signs);
+}
+
+Status ColumnVectorBatch::resize(size_t new_cap) {
+    if (_nullable) {
+        resize_buff(_capacity, new_cap, reinterpret_cast<uint8_t**>(&_null_signs));
+    }
+    _capacity = new_cap;
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::resize_buff(size_t copy_bytes, size_t new_cap, uint8_t** _buf_ptr) {
+    const uint8_t* old_buf = *_buf_ptr;
+    if (old_buf) {
+        *_buf_ptr = new uint8_t[new_cap];
+        memcpy(*_buf_ptr, old_buf, copy_bytes);
+        delete[] old_buf;
+    } else {
+        *_buf_ptr = new uint8_t[new_cap];
+    }
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::create(size_t init_capacity,
+        bool is_nullable,
+        const TypeInfo* type_info,
+        std::unique_ptr<ColumnVectorBatch>* column_vector_batch) {
+    if (is_scalar_type(type_info->type())) {
+        std::unique_ptr<ColumnVectorBatch> local(
+                new ScalarColumnVectorBatch(type_info, is_nullable, init_capacity));
+        *column_vector_batch = std::move(local);
+        return Status::OK();
+    } else {
+        switch (type_info->type()) {
+        case FieldType::OLAP_FIELD_TYPE_LIST: {
+            std::unique_ptr<ColumnVectorBatch> local(new ListColumnVectorBatch(type_info, is_nullable, init_capacity));
+            *column_vector_batch = std::move(local);
+            return Status::OK();
+        }
+        default:
+            return Status::NotSupported("unsupported type for ColumnVectorBatch: " + std::to_string(type_info->type()));
+        }
+    }
+}
+
+ScalarColumnVectorBatch::ScalarColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity)
+: ColumnVectorBatch(type_info, is_nullable) {
+    resize(init_capacity);
+}
+ScalarColumnVectorBatch::~ScalarColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_data);
+}
+
+Status ScalarColumnVectorBatch::resize(size_t new_cap) {
+    if (get_capacity() < new_cap) { // before first init, _capacity is 0.
+        size_t type_size = type_info()->size();
+        auto copy_bytes = get_capacity() * type_size;
+        size_t new_data_size = new_cap * type_size;
+        resize_buff(copy_bytes, new_data_size, &_data);
+        return ColumnVectorBatch::resize(new_cap);
 
 Review comment:
   Why `return ColumnVectorBatch::resize(new_cap)`? double `resize_buff`?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r393160170
 
 

 ##########
 File path: be/src/olap/column_vector.h
 ##########
 @@ -0,0 +1,150 @@
+// 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/status.h"
+#include "olap/olap_common.h"
+#include "olap/rowset/segment_v2/common.h" // for ordinal_t
+#include "olap/types.h"
+
+namespace doris {
+
+// struct that contains column data(null bitmap), data array in sub class.
+struct ColumnVectorBatch {
+public:
+    explicit ColumnVectorBatch(const TypeInfo* type_info, bool is_nullable)
+    : _type_info(type_info), _capacity(0), _nullable(is_nullable) {}
+
+    virtual ~ColumnVectorBatch();
+
+    const TypeInfo* type_info() const { return _type_info; }
+
+    size_t get_capacity() { return _capacity; }
+
+    inline bool is_nullable() {return _nullable; }
+
+    inline bool is_null_at(size_t row_idx) const {
+        return _nullable && _null_signs[row_idx];
+    }
+
+    inline void set_is_null(size_t idx, bool is_null) const {
+        if (_nullable) {
+            _null_signs[idx] = is_null;
+        }
+    }
+
+    inline void set_null_bits(size_t offset, size_t num_rows, bool val) const {
+        for (size_t i = 0; i < num_rows; ++i) {
+            set_is_null(offset + i, val);
+        }
+    }
+
+    inline bool* get_null_signs(size_t idx) { return _null_signs + idx; }
+
+
+    /**
+     * Change the number of slots to at least the given capacity.
+     * This function is not recursive into subtypes.
+     * Tips: This function will change `_capacity` attribute.
+     */
+    virtual Status resize(size_t new_cap);
+
+    // Get the start of the data.
+    virtual uint8_t* data() const = 0;
+
+    // Get the idx's cell_ptr
+    virtual const uint8_t* cell_ptr(size_t idx) const = 0;
+
+    // Get thr idx's cell_ptr for write
+    virtual uint8_t* mutable_cell_ptr(size_t idx) const = 0;
+
+
+    static Status create(size_t init_capacity,
+                         bool is_nullable,
+                         const TypeInfo* type_info,
+                         std::unique_ptr<ColumnVectorBatch>* column_vector_batch);
+
+protected:
+    static Status resize_buff(size_t old_cap, size_t new_cap, uint8_t** _buf);
+
+private:
+    const TypeInfo* _type_info;
+    size_t _capacity;
+    const bool _nullable;
+    bool* _null_signs = nullptr;
+};
+
+struct ScalarColumnVectorBatch : public ColumnVectorBatch {
+public:
+    explicit ScalarColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity);
+
+    ~ScalarColumnVectorBatch() override ;
+
+    Status resize(size_t new_cap) override;
+
+    // Get the start of the data.
+    uint8_t* data() const override;
+
+    // Get the idx's cell_ptr
+    const uint8_t* cell_ptr(size_t idx) const override;
+
+    // Get thr idx's cell_ptr for write
+    uint8_t* mutable_cell_ptr(size_t idx) const override;
+
+private:
+    uint8_t* _data = nullptr;
+};
+
+struct ListColumnVectorBatch : public ColumnVectorBatch {
+public:
+    explicit ListColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity);
+    ~ListColumnVectorBatch() override;
+    Status resize(size_t new_cap) override;
+
+    ColumnVectorBatch* get_elements() const { return _elements.get(); }
+
+    // Get the start of the data.
+    uint8_t* data() const override;
+
+    // Get the idx's cell_ptr
+    const uint8_t* cell_ptr(size_t idx) const override;
+
+    // Get thr idx's cell_ptr for write
+    uint8_t* mutable_cell_ptr(size_t idx) const override;
+
+    size_t item_offset(size_t idx) const;
+
+    // 将size个ordinals放入_item_offsets,从start_idx开始覆盖
+    // _item_offsets: 0 3 5 9, ordinals: 100 105 111, size: 3, satart_idx:3
+    // --> _item_offsets: 0 3 5 (9 + 100 - 100) (9 + 105 - 100) (9 + 111 - 100)
+    // 即 _item_offsets变成 0 3 5 9 14 20
 
 Review comment:
   use english to comment

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r392285900
 
 

 ##########
 File path: be/src/olap/types.h
 ##########
 @@ -106,12 +141,186 @@ class TypeInfo {
     const size_t _size;
     const FieldType _field_type;
 
-    friend class TypeInfoResolver;
-    template<typename TypeTraitsClass> TypeInfo(TypeTraitsClass t);
+    friend class ScalarTypeInfoResolver;
+    template<typename TypeTraitsClass> ScalarTypeInfo(TypeTraitsClass t);
+};
+
+class ListTypeInfo : public TypeInfo {
 
 Review comment:
   Please add a UT for `ListTypeInfo`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r393165352
 
 

 ##########
 File path: be/src/olap/column_vector.cpp
 ##########
 @@ -0,0 +1,150 @@
+// 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 "column_vector.h"
+
+namespace doris {
+
+ColumnVectorBatch::~ColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_null_signs);
+}
+
+Status ColumnVectorBatch::resize(size_t new_cap) {
+    if (_nullable) {
+        resize_buff(_capacity, new_cap, reinterpret_cast<uint8_t**>(&_null_signs));
+    }
+    _capacity = new_cap;
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::resize_buff(size_t copy_bytes, size_t new_cap, uint8_t** _buf_ptr) {
+    const uint8_t* old_buf = *_buf_ptr;
+    if (old_buf) {
+        *_buf_ptr = new uint8_t[new_cap];
+        memcpy(*_buf_ptr, old_buf, copy_bytes);
+        delete[] old_buf;
+    } else {
+        *_buf_ptr = new uint8_t[new_cap];
+    }
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::create(size_t init_capacity,
+        bool is_nullable,
+        const TypeInfo* type_info,
+        std::unique_ptr<ColumnVectorBatch>* column_vector_batch) {
+    if (is_scalar_type(type_info->type())) {
+        std::unique_ptr<ColumnVectorBatch> local(
+                new ScalarColumnVectorBatch(type_info, is_nullable, init_capacity));
+        *column_vector_batch = std::move(local);
+        return Status::OK();
+    } else {
+        switch (type_info->type()) {
+        case FieldType::OLAP_FIELD_TYPE_LIST: {
 
 Review comment:
   use ARRAY to replace LIST.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r394478297
 
 

 ##########
 File path: be/src/olap/column_vector.cpp
 ##########
 @@ -0,0 +1,150 @@
+// 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 "column_vector.h"
+
+namespace doris {
+
+ColumnVectorBatch::~ColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_null_signs);
+}
+
+Status ColumnVectorBatch::resize(size_t new_cap) {
+    if (_nullable) {
 
 Review comment:
   Added this logic in `ScalarColumnVectorBatch` and `ListColumnVectorBatch`.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r395097352
 
 

 ##########
 File path: be/src/olap/rowset/segment_v2/column_reader.h
 ##########
 @@ -187,6 +196,25 @@ class ColumnReader {
     std::unique_ptr<BloomFilterIndexReader> _bloom_filter_index;
 };
 
+class ArrayColumnReader : public ColumnReader {
+public:
+    ArrayColumnReader(const ColumnReaderOptions& opts,
+                     const ColumnMetaPB& meta,
 
 Review comment:
   should align with previous parameter

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r394137314
 
 

 ##########
 File path: be/src/olap/rowset/segment_v2/column_writer.h
 ##########
 @@ -94,19 +98,46 @@ class ColumnWriter {
     Status append_nulls(size_t num_rows);
     Status append(const void* data, size_t num_rows);
     Status append_nullable(const uint8_t* nullmap, const void* data, size_t num_rows);
+    Status append_nullable_by_null_signs(const bool* null_signs, const void* data, size_t num_rows);
 
 Review comment:
   There are too many variants of append methods, which is difficult to use and adds maintenance cost. Since SegmentWriter currently only supports single row insert API, can you unify the append methods of ColumnWriter?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r393161845
 
 

 ##########
 File path: be/src/olap/column_vector.h
 ##########
 @@ -0,0 +1,150 @@
+// 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/status.h"
+#include "olap/olap_common.h"
+#include "olap/rowset/segment_v2/common.h" // for ordinal_t
+#include "olap/types.h"
+
+namespace doris {
+
+// struct that contains column data(null bitmap), data array in sub class.
+struct ColumnVectorBatch {
+public:
+    explicit ColumnVectorBatch(const TypeInfo* type_info, bool is_nullable)
+    : _type_info(type_info), _capacity(0), _nullable(is_nullable) {}
+
+    virtual ~ColumnVectorBatch();
+
+    const TypeInfo* type_info() const { return _type_info; }
+
+    size_t get_capacity() { return _capacity; }
+
+    inline bool is_nullable() {return _nullable; }
+
+    inline bool is_null_at(size_t row_idx) const {
+        return _nullable && _null_signs[row_idx];
+    }
+
+    inline void set_is_null(size_t idx, bool is_null) const {
+        if (_nullable) {
+            _null_signs[idx] = is_null;
+        }
+    }
+
+    inline void set_null_bits(size_t offset, size_t num_rows, bool val) const {
+        for (size_t i = 0; i < num_rows; ++i) {
+            set_is_null(offset + i, val);
+        }
+    }
+
+    inline bool* get_null_signs(size_t idx) { return _null_signs + idx; }
+
+
+    /**
+     * Change the number of slots to at least the given capacity.
+     * This function is not recursive into subtypes.
+     * Tips: This function will change `_capacity` attribute.
+     */
+    virtual Status resize(size_t new_cap);
+
+    // Get the start of the data.
+    virtual uint8_t* data() const = 0;
+
+    // Get the idx's cell_ptr
+    virtual const uint8_t* cell_ptr(size_t idx) const = 0;
+
+    // Get thr idx's cell_ptr for write
+    virtual uint8_t* mutable_cell_ptr(size_t idx) const = 0;
+
+
+    static Status create(size_t init_capacity,
+                         bool is_nullable,
+                         const TypeInfo* type_info,
+                         std::unique_ptr<ColumnVectorBatch>* column_vector_batch);
+
+protected:
+    static Status resize_buff(size_t old_cap, size_t new_cap, uint8_t** _buf);
+
+private:
+    const TypeInfo* _type_info;
+    size_t _capacity;
+    const bool _nullable;
+    bool* _null_signs = nullptr;
 
 Review comment:
   ```suggestion
       uint8_t* _null_signs = nullptr;
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r392607943
 
 

 ##########
 File path: be/src/olap/column_vector.cpp
 ##########
 @@ -0,0 +1,150 @@
+// 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 "column_vector.h"
+
+namespace doris {
+
+ColumnVectorBatch::~ColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_null_signs);
+}
+
+Status ColumnVectorBatch::resize(size_t new_cap) {
+    if (_nullable) {
+        resize_buff(_capacity, new_cap, reinterpret_cast<uint8_t**>(&_null_signs));
+    }
+    _capacity = new_cap;
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::resize_buff(size_t copy_bytes, size_t new_cap, uint8_t** _buf_ptr) {
+    const uint8_t* old_buf = *_buf_ptr;
+    if (old_buf) {
+        *_buf_ptr = new uint8_t[new_cap];
+        memcpy(*_buf_ptr, old_buf, copy_bytes);
+        delete[] old_buf;
+    } else {
+        *_buf_ptr = new uint8_t[new_cap];
+    }
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::create(size_t init_capacity,
+        bool is_nullable,
+        const TypeInfo* type_info,
+        std::unique_ptr<ColumnVectorBatch>* column_vector_batch) {
+    if (is_scalar_type(type_info->type())) {
+        std::unique_ptr<ColumnVectorBatch> local(
+                new ScalarColumnVectorBatch(type_info, is_nullable, init_capacity));
+        *column_vector_batch = std::move(local);
+        return Status::OK();
+    } else {
+        switch (type_info->type()) {
+        case FieldType::OLAP_FIELD_TYPE_LIST: {
+            std::unique_ptr<ColumnVectorBatch> local(new ListColumnVectorBatch(type_info, is_nullable, init_capacity));
+            *column_vector_batch = std::move(local);
+            return Status::OK();
+        }
+        default:
+            return Status::NotSupported("unsupported type for ColumnVectorBatch: " + std::to_string(type_info->type()));
+        }
+    }
+}
+
+ScalarColumnVectorBatch::ScalarColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity)
+: ColumnVectorBatch(type_info, is_nullable) {
+    resize(init_capacity);
+}
+ScalarColumnVectorBatch::~ScalarColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_data);
+}
+
+Status ScalarColumnVectorBatch::resize(size_t new_cap) {
+    if (get_capacity() < new_cap) { // before first init, _capacity is 0.
+        size_t type_size = type_info()->size();
+        auto copy_bytes = get_capacity() * type_size;
+        size_t new_data_size = new_cap * type_size;
+        resize_buff(copy_bytes, new_data_size, &_data);
+        return ColumnVectorBatch::resize(new_cap);
 
 Review comment:
   Because `resize` in `ScalarColumnVectorBatch` and `ListColumnVectorBatch` is used to reset data, and `resize` in ColumnVectorBatch is used to resize _null_signs.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r393147724
 
 

 ##########
 File path: be/src/olap/column_vector.cpp
 ##########
 @@ -0,0 +1,150 @@
+// 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 "column_vector.h"
+
+namespace doris {
+
+ColumnVectorBatch::~ColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_null_signs);
+}
+
+Status ColumnVectorBatch::resize(size_t new_cap) {
+    if (_nullable) {
+        resize_buff(_capacity, new_cap, reinterpret_cast<uint8_t**>(&_null_signs));
+    }
+    _capacity = new_cap;
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::resize_buff(size_t copy_bytes, size_t new_cap, uint8_t** _buf_ptr) {
+    const uint8_t* old_buf = *_buf_ptr;
+    if (old_buf) {
+        *_buf_ptr = new uint8_t[new_cap];
+        memcpy(*_buf_ptr, old_buf, copy_bytes);
+        delete[] old_buf;
+    } else {
+        *_buf_ptr = new uint8_t[new_cap];
+    }
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::create(size_t init_capacity,
+        bool is_nullable,
+        const TypeInfo* type_info,
+        std::unique_ptr<ColumnVectorBatch>* column_vector_batch) {
+    if (is_scalar_type(type_info->type())) {
+        std::unique_ptr<ColumnVectorBatch> local(
+                new ScalarColumnVectorBatch(type_info, is_nullable, init_capacity));
+        *column_vector_batch = std::move(local);
+        return Status::OK();
+    } else {
+        switch (type_info->type()) {
+        case FieldType::OLAP_FIELD_TYPE_LIST: {
+            std::unique_ptr<ColumnVectorBatch> local(new ListColumnVectorBatch(type_info, is_nullable, init_capacity));
+            *column_vector_batch = std::move(local);
+            return Status::OK();
+        }
+        default:
+            return Status::NotSupported("unsupported type for ColumnVectorBatch: " + std::to_string(type_info->type()));
+        }
+    }
+}
+
+ScalarColumnVectorBatch::ScalarColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity)
+: ColumnVectorBatch(type_info, is_nullable) {
+    resize(init_capacity);
+}
+ScalarColumnVectorBatch::~ScalarColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_data);
+}
+
+Status ScalarColumnVectorBatch::resize(size_t new_cap) {
+    if (get_capacity() < new_cap) { // before first init, _capacity is 0.
+        size_t type_size = type_info()->size();
+        auto copy_bytes = get_capacity() * type_size;
+        size_t new_data_size = new_cap * type_size;
+        resize_buff(copy_bytes, new_data_size, &_data);
+        return ColumnVectorBatch::resize(new_cap);
+    }
+    return Status::OK();
+}
+
+uint8_t* ScalarColumnVectorBatch::data() const { return _data; };
+
+const uint8_t* ScalarColumnVectorBatch::cell_ptr(size_t idx) const { return _data + idx * type_info()->size(); };
+
+uint8_t* ScalarColumnVectorBatch::mutable_cell_ptr(size_t idx) const { return _data + idx * type_info()->size(); };
+
 
 Review comment:
   additional line

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r392769925
 
 

 ##########
 File path: be/src/olap/rowset/segment_v2/column_writer.cpp
 ##########
 @@ -361,5 +430,106 @@ Status ColumnWriter::_finish_current_page() {
     return Status::OK();
 }
 
+////////////////////////////////////////////////////////////////////////////////
+
+ListColumnWriter::ListColumnWriter(const ColumnWriterOptions& opts,
+                         std::unique_ptr<Field> field,
+                         WritableFile* output_file,
+                         std::unique_ptr<ColumnWriter> item_writer):
+        ColumnWriter(opts, std::move(field), output_file), _item_writer(std::move(item_writer)) {}
+
+Status ListColumnWriter::create_page_builder(PageBuilder** page_builder) {
+    TypeInfo* bigint_type_info = get_scalar_type_info(FieldType::OLAP_FIELD_TYPE_BIGINT);
+    RETURN_IF_ERROR(EncodingInfo::get(bigint_type_info, _opts.meta->encoding(), &_encoding_info));
+
+    PageBuilder* local = nullptr;
+    PageBuilderOptions opts;
+    opts.data_page_size = _opts.data_page_size;
+    _opts.meta->set_encoding(_encoding_info->encoding());
+
+    RETURN_IF_ERROR(_encoding_info->create_page_builder(opts, &local));
+    if (local == nullptr) {
+        return Status::NotSupported(
+                Substitute("Failed to create page builder for type LIST encoding $0",
+                           _opts.meta->encoding()));
+    } else {
+        *page_builder = local;
+        return Status::OK();
+    }
+}
+ListColumnWriter::~ListColumnWriter() = default;
+
+Status ListColumnWriter::init() {
+    if (_opts.need_zone_map) {
+        return Status::NotSupported("unsupported zone map for list");
+    }
+
+    if (_opts.need_bitmap_index) {
+        return Status::NotSupported("unsupported bitmap for list");
+    }
+
+    if (_opts.need_bloom_filter) {
+        return Status::NotSupported("unsupported bloom filter for list");
+    }
+
+    RETURN_IF_ERROR(ColumnWriter::init());
+    RETURN_IF_ERROR(_item_writer->init());
+    return Status::OK();
+}
+
+Status ListColumnWriter::put_page_footer_info(DataPageFooterPB* footer) {
+    footer->set_next_array_item_ordinal(_next_item_ordinal);
+    return Status::OK();
 }
+
+// Now we can only write data one by one.
+Status ListColumnWriter::_append_data(const uint8_t** ptr, size_t num_rows) {
+    size_t remaining = num_rows;
+    const auto* col_cursor = reinterpret_cast<const collection*>(*ptr);
+    while (remaining > 0) {
+        size_t num_written = 1;
 
 Review comment:
   I think maybe offsets and null values could bulk write.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] liutang123 commented on issue #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
liutang123 commented on issue #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#issuecomment-596221473
 
 
   cc @gaodayue @imay @Seaven 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r394093086
 
 

 ##########
 File path: be/src/olap/CMakeLists.txt
 ##########
 @@ -113,4 +113,7 @@ add_library(Olap STATIC
     task/engine_publish_version_task.cpp
     task/engine_alter_tablet_task.cpp
     olap_snapshot_converter.cpp
+    collection.h
+    column_vector.h
 
 Review comment:
   there is no need to put header file here, it's implicitly included by cpp file

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r392268958
 
 

 ##########
 File path: be/src/olap/rowset/segment_v2/column_writer.cpp
 ##########
 @@ -361,5 +430,106 @@ Status ColumnWriter::_finish_current_page() {
     return Status::OK();
 }
 
+////////////////////////////////////////////////////////////////////////////////
+
+ListColumnWriter::ListColumnWriter(const ColumnWriterOptions& opts,
+                         std::unique_ptr<Field> field,
+                         WritableFile* output_file,
+                         std::unique_ptr<ColumnWriter> item_writer):
+        ColumnWriter(opts, std::move(field), output_file), _item_writer(std::move(item_writer)) {}
+
+Status ListColumnWriter::create_page_builder(PageBuilder** page_builder) {
+    TypeInfo* bigint_type_info = get_scalar_type_info(FieldType::OLAP_FIELD_TYPE_BIGINT);
+    RETURN_IF_ERROR(EncodingInfo::get(bigint_type_info, _opts.meta->encoding(), &_encoding_info));
+
+    PageBuilder* local = nullptr;
+    PageBuilderOptions opts;
+    opts.data_page_size = _opts.data_page_size;
+    _opts.meta->set_encoding(_encoding_info->encoding());
+
+    RETURN_IF_ERROR(_encoding_info->create_page_builder(opts, &local));
+    if (local == nullptr) {
+        return Status::NotSupported(
+                Substitute("Failed to create page builder for type LIST encoding $0",
+                           _opts.meta->encoding()));
+    } else {
+        *page_builder = local;
+        return Status::OK();
+    }
+}
+ListColumnWriter::~ListColumnWriter() = default;
+
+Status ListColumnWriter::init() {
+    if (_opts.need_zone_map) {
+        return Status::NotSupported("unsupported zone map for list");
+    }
+
+    if (_opts.need_bitmap_index) {
+        return Status::NotSupported("unsupported bitmap for list");
+    }
+
+    if (_opts.need_bloom_filter) {
+        return Status::NotSupported("unsupported bloom filter for list");
+    }
+
+    RETURN_IF_ERROR(ColumnWriter::init());
+    RETURN_IF_ERROR(_item_writer->init());
+    return Status::OK();
+}
+
+Status ListColumnWriter::put_page_footer_info(DataPageFooterPB* footer) {
+    footer->set_next_array_item_ordinal(_next_item_ordinal);
+    return Status::OK();
 }
+
+// Now we can only write data one by one.
+Status ListColumnWriter::_append_data(const uint8_t** ptr, size_t num_rows) {
+    size_t remaining = num_rows;
+    const auto* col_cursor = reinterpret_cast<const collection*>(*ptr);
+    while (remaining > 0) {
+        size_t num_written = 1;
 
 Review comment:
   Why not add batch?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r393164338
 
 

 ##########
 File path: be/src/olap/column_vector.cpp
 ##########
 @@ -0,0 +1,150 @@
+// 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 "column_vector.h"
+
+namespace doris {
+
+ColumnVectorBatch::~ColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_null_signs);
+}
+
+Status ColumnVectorBatch::resize(size_t new_cap) {
+    if (_nullable) {
+        resize_buff(_capacity, new_cap, reinterpret_cast<uint8_t**>(&_null_signs));
+    }
+    _capacity = new_cap;
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::resize_buff(size_t copy_bytes, size_t new_cap, uint8_t** _buf_ptr) {
 
 Review comment:
   I think reallocate_buffer is a better name.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r392760276
 
 

 ##########
 File path: be/src/olap/column_vector.cpp
 ##########
 @@ -0,0 +1,150 @@
+// 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 "column_vector.h"
+
+namespace doris {
+
+ColumnVectorBatch::~ColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_null_signs);
+}
+
+Status ColumnVectorBatch::resize(size_t new_cap) {
+    if (_nullable) {
+        resize_buff(_capacity, new_cap, reinterpret_cast<uint8_t**>(&_null_signs));
+    }
+    _capacity = new_cap;
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::resize_buff(size_t copy_bytes, size_t new_cap, uint8_t** _buf_ptr) {
+    const uint8_t* old_buf = *_buf_ptr;
+    if (old_buf) {
+        *_buf_ptr = new uint8_t[new_cap];
+        memcpy(*_buf_ptr, old_buf, copy_bytes);
+        delete[] old_buf;
+    } else {
+        *_buf_ptr = new uint8_t[new_cap];
+    }
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::create(size_t init_capacity,
+        bool is_nullable,
+        const TypeInfo* type_info,
+        std::unique_ptr<ColumnVectorBatch>* column_vector_batch) {
+    if (is_scalar_type(type_info->type())) {
+        std::unique_ptr<ColumnVectorBatch> local(
+                new ScalarColumnVectorBatch(type_info, is_nullable, init_capacity));
+        *column_vector_batch = std::move(local);
+        return Status::OK();
+    } else {
+        switch (type_info->type()) {
+        case FieldType::OLAP_FIELD_TYPE_LIST: {
+            std::unique_ptr<ColumnVectorBatch> local(new ListColumnVectorBatch(type_info, is_nullable, init_capacity));
+            *column_vector_batch = std::move(local);
+            return Status::OK();
+        }
+        default:
+            return Status::NotSupported("unsupported type for ColumnVectorBatch: " + std::to_string(type_info->type()));
+        }
+    }
+}
+
+ScalarColumnVectorBatch::ScalarColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity)
+: ColumnVectorBatch(type_info, is_nullable) {
+    resize(init_capacity);
+}
+ScalarColumnVectorBatch::~ScalarColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_data);
+}
+
+Status ScalarColumnVectorBatch::resize(size_t new_cap) {
+    if (get_capacity() < new_cap) { // before first init, _capacity is 0.
+        size_t type_size = type_info()->size();
+        auto copy_bytes = get_capacity() * type_size;
+        size_t new_data_size = new_cap * type_size;
+        resize_buff(copy_bytes, new_data_size, &_data);
+        return ColumnVectorBatch::resize(new_cap);
 
 Review comment:
   Please change the method name and add the comment. Which is confusing.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r394105937
 
 

 ##########
 File path: be/src/olap/column_vector.h
 ##########
 @@ -0,0 +1,150 @@
+// 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/status.h"
+#include "olap/olap_common.h"
+#include "olap/rowset/segment_v2/common.h" // for ordinal_t
+#include "olap/types.h"
+
+namespace doris {
+
+// struct that contains column data(null bitmap), data array in sub class.
+struct ColumnVectorBatch {
+public:
+    explicit ColumnVectorBatch(const TypeInfo* type_info, bool is_nullable)
+    : _type_info(type_info), _capacity(0), _nullable(is_nullable) {}
 
 Review comment:
   ```suggestion
       explicit ColumnVectorBatch(const TypeInfo* type_info, bool is_nullable) :
               _type_info(type_info), _capacity(0), _nullable(is_nullable) {}
   ```
   code style

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r395061622
 
 

 ##########
 File path: be/src/olap/row_block2.cpp
 ##########
 @@ -29,30 +29,21 @@ namespace doris {
 RowBlockV2::RowBlockV2(const Schema& schema, uint16_t capacity)
     : _schema(schema),
       _capacity(capacity),
-      _column_datas(_schema.num_columns(), nullptr),
-      _column_null_bitmaps(_schema.num_columns(), nullptr),
+      _column_vector_batches(_schema.num_columns()),
       _pool(new MemPool(&_tracker)),
       _selection_vector(nullptr) {
-    auto bitmap_size = BitmapSize(capacity);
     for (auto cid : _schema.column_ids()) {
-        size_t data_size = _schema.column(cid)->type_info()->size() * _capacity;
-        _column_datas[cid] = new uint8_t[data_size];
-
-        if (_schema.column(cid)->is_nullable()) {
-            _column_null_bitmaps[cid] = new uint8_t[bitmap_size];;
-        }
+        ColumnVectorBatch::create(
+                _capacity,
+                _schema.column(cid)->is_nullable(),
+                _schema.column(cid)->type_info(),
+                &_column_vector_batches[cid]);
 
 Review comment:
   shoud CHECK the return value so that we can detect BUG early

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r394818744
 
 

 ##########
 File path: be/src/olap/column_vector.h
 ##########
 @@ -0,0 +1,150 @@
+// 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/status.h"
+#include "olap/olap_common.h"
+#include "olap/rowset/segment_v2/common.h" // for ordinal_t
+#include "olap/types.h"
+
+namespace doris {
+
+// struct that contains column data(null bitmap), data array in sub class.
+struct ColumnVectorBatch {
+public:
+    explicit ColumnVectorBatch(const TypeInfo* type_info, bool is_nullable)
+    : _type_info(type_info), _capacity(0), _nullable(is_nullable) {}
+
+    virtual ~ColumnVectorBatch();
+
+    const TypeInfo* type_info() const { return _type_info; }
+
+    size_t get_capacity() { return _capacity; }
+
+    inline bool is_nullable() {return _nullable; }
+
+    inline bool is_null_at(size_t row_idx) const {
+        return _nullable && _null_signs[row_idx];
+    }
+
+    inline void set_is_null(size_t idx, bool is_null) const {
+        if (_nullable) {
+            _null_signs[idx] = is_null;
+        }
+    }
+
+    inline void set_null_bits(size_t offset, size_t num_rows, bool val) const {
+        for (size_t i = 0; i < num_rows; ++i) {
+            set_is_null(offset + i, val);
+        }
+    }
+
+    inline bool* get_null_signs(size_t idx) { return _null_signs + idx; }
+
+
+    /**
+     * Change the number of slots to at least the given capacity.
+     * This function is not recursive into subtypes.
+     * Tips: This function will change `_capacity` attribute.
+     */
+    virtual Status resize(size_t new_cap);
+
+    // Get the start of the data.
+    virtual uint8_t* data() const = 0;
+
+    // Get the idx's cell_ptr
+    virtual const uint8_t* cell_ptr(size_t idx) const = 0;
+
+    // Get thr idx's cell_ptr for write
+    virtual uint8_t* mutable_cell_ptr(size_t idx) const = 0;
+
+
+    static Status create(size_t init_capacity,
+                         bool is_nullable,
+                         const TypeInfo* type_info,
+                         std::unique_ptr<ColumnVectorBatch>* column_vector_batch);
+
+protected:
+    static Status resize_buff(size_t old_cap, size_t new_cap, uint8_t** _buf);
+
+private:
+    const TypeInfo* _type_info;
+    size_t _capacity;
+    const bool _nullable;
+    bool* _null_signs = nullptr;
 
 Review comment:
   It is not a bitmap. Now, use a bool to represents the nullable of a slot. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r393165080
 
 

 ##########
 File path: be/src/olap/column_vector.cpp
 ##########
 @@ -0,0 +1,150 @@
+// 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 "column_vector.h"
+
+namespace doris {
+
+ColumnVectorBatch::~ColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_null_signs);
+}
+
+Status ColumnVectorBatch::resize(size_t new_cap) {
+    if (_nullable) {
+        resize_buff(_capacity, new_cap, reinterpret_cast<uint8_t**>(&_null_signs));
+    }
+    _capacity = new_cap;
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::resize_buff(size_t copy_bytes, size_t new_cap, uint8_t** _buf_ptr) {
+    const uint8_t* old_buf = *_buf_ptr;
+    if (old_buf) {
+        *_buf_ptr = new uint8_t[new_cap];
+        memcpy(*_buf_ptr, old_buf, copy_bytes);
+        delete[] old_buf;
+    } else {
+        *_buf_ptr = new uint8_t[new_cap];
+    }
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::create(size_t init_capacity,
+        bool is_nullable,
+        const TypeInfo* type_info,
+        std::unique_ptr<ColumnVectorBatch>* column_vector_batch) {
+    if (is_scalar_type(type_info->type())) {
+        std::unique_ptr<ColumnVectorBatch> local(
+                new ScalarColumnVectorBatch(type_info, is_nullable, init_capacity));
+        *column_vector_batch = std::move(local);
+        return Status::OK();
+    } else {
+        switch (type_info->type()) {
+        case FieldType::OLAP_FIELD_TYPE_LIST: {
 
 Review comment:
   ```suggestion
           case FieldType::OLAP_FIELD_TYPE_ARRAY: {
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r392281460
 
 

 ##########
 File path: be/src/olap/collection.h
 ##########
 @@ -0,0 +1,50 @@
+// 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 <iostream>
+
+namespace doris {
+
+// cpp type for LIST
+struct collection {
 
 Review comment:
   ```suggestion
   struct Collection {
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r393161255
 
 

 ##########
 File path: be/src/olap/column_vector.h
 ##########
 @@ -0,0 +1,150 @@
+// 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/status.h"
+#include "olap/olap_common.h"
+#include "olap/rowset/segment_v2/common.h" // for ordinal_t
+#include "olap/types.h"
+
+namespace doris {
+
+// struct that contains column data(null bitmap), data array in sub class.
+struct ColumnVectorBatch {
+public:
+    explicit ColumnVectorBatch(const TypeInfo* type_info, bool is_nullable)
+    : _type_info(type_info), _capacity(0), _nullable(is_nullable) {}
+
+    virtual ~ColumnVectorBatch();
+
+    const TypeInfo* type_info() const { return _type_info; }
+
+    size_t get_capacity() { return _capacity; }
+
+    inline bool is_nullable() {return _nullable; }
+
+    inline bool is_null_at(size_t row_idx) const {
+        return _nullable && _null_signs[row_idx];
+    }
+
+    inline void set_is_null(size_t idx, bool is_null) const {
+        if (_nullable) {
+            _null_signs[idx] = is_null;
+        }
+    }
+
+    inline void set_null_bits(size_t offset, size_t num_rows, bool val) const {
+        for (size_t i = 0; i < num_rows; ++i) {
+            set_is_null(offset + i, val);
+        }
+    }
+
+    inline bool* get_null_signs(size_t idx) { return _null_signs + idx; }
+
+
+    /**
+     * Change the number of slots to at least the given capacity.
+     * This function is not recursive into subtypes.
+     * Tips: This function will change `_capacity` attribute.
+     */
+    virtual Status resize(size_t new_cap);
+
+    // Get the start of the data.
+    virtual uint8_t* data() const = 0;
+
+    // Get the idx's cell_ptr
+    virtual const uint8_t* cell_ptr(size_t idx) const = 0;
+
+    // Get thr idx's cell_ptr for write
+    virtual uint8_t* mutable_cell_ptr(size_t idx) const = 0;
+
+
+    static Status create(size_t init_capacity,
+                         bool is_nullable,
+                         const TypeInfo* type_info,
+                         std::unique_ptr<ColumnVectorBatch>* column_vector_batch);
+
+protected:
+    static Status resize_buff(size_t old_cap, size_t new_cap, uint8_t** _buf);
+
+private:
+    const TypeInfo* _type_info;
+    size_t _capacity;
+    const bool _nullable;
+    bool* _null_signs = nullptr;
+};
+
+struct ScalarColumnVectorBatch : public ColumnVectorBatch {
+public:
+    explicit ScalarColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity);
+
+    ~ScalarColumnVectorBatch() override ;
+
+    Status resize(size_t new_cap) override;
+
+    // Get the start of the data.
+    uint8_t* data() const override;
+
+    // Get the idx's cell_ptr
+    const uint8_t* cell_ptr(size_t idx) const override;
+
+    // Get thr idx's cell_ptr for write
+    uint8_t* mutable_cell_ptr(size_t idx) const override;
+
+private:
+    uint8_t* _data = nullptr;
+};
+
+struct ListColumnVectorBatch : public ColumnVectorBatch {
+public:
+    explicit ListColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity);
+    ~ListColumnVectorBatch() override;
+    Status resize(size_t new_cap) override;
+
+    ColumnVectorBatch* get_elements() const { return _elements.get(); }
+
+    // Get the start of the data.
+    uint8_t* data() const override;
+
+    // Get the idx's cell_ptr
+    const uint8_t* cell_ptr(size_t idx) const override;
+
+    // Get thr idx's cell_ptr for write
+    uint8_t* mutable_cell_ptr(size_t idx) const override;
+
+    size_t item_offset(size_t idx) const;
+
+    // 将size个ordinals放入_item_offsets,从start_idx开始覆盖
+    // _item_offsets: 0 3 5 9, ordinals: 100 105 111, size: 3, satart_idx:3
+    // --> _item_offsets: 0 3 5 (9 + 100 - 100) (9 + 105 - 100) (9 + 111 - 100)
+    // 即 _item_offsets变成 0 3 5 9 14 20
+    void put_item_ordinal(segment_v2::ordinal_t* ordinals, size_t start_idx, size_t size);
+
+    // 将_item_offsets从start_idx到end_idx的部分转化成data
+    void transform_offsets_and_elements_to_data(size_t start_idx, size_t end_idx);
+
+private:
+    collection* _data = nullptr;
+
+    std::unique_ptr<ColumnVectorBatch> _elements;
+
+    // 用于存储每个数据在_elements中的位置
 
 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r392281874
 
 

 ##########
 File path: be/src/olap/collection.h
 ##########
 @@ -0,0 +1,50 @@
+// 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 <iostream>
+
+namespace doris {
+
+// cpp type for LIST
+struct collection {
+    size_t length;
+    // null bitmap
+    bool* null_signs;
+    // 子元素数据数据
 
 Review comment:
   comment should use english.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r394101066
 
 

 ##########
 File path: be/src/olap/column_vector.h
 ##########
 @@ -0,0 +1,150 @@
+// 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/status.h"
+#include "olap/olap_common.h"
+#include "olap/rowset/segment_v2/common.h" // for ordinal_t
+#include "olap/types.h"
+
+namespace doris {
+
+// struct that contains column data(null bitmap), data array in sub class.
+struct ColumnVectorBatch {
+public:
+    explicit ColumnVectorBatch(const TypeInfo* type_info, bool is_nullable)
+    : _type_info(type_info), _capacity(0), _nullable(is_nullable) {}
+
+    virtual ~ColumnVectorBatch();
+
+    const TypeInfo* type_info() const { return _type_info; }
+
+    size_t get_capacity() { return _capacity; }
+
+    inline bool is_nullable() {return _nullable; }
+
+    inline bool is_null_at(size_t row_idx) const {
+        return _nullable && _null_signs[row_idx];
+    }
+
+    inline void set_is_null(size_t idx, bool is_null) const {
+        if (_nullable) {
+            _null_signs[idx] = is_null;
+        }
+    }
+
+    inline void set_null_bits(size_t offset, size_t num_rows, bool val) const {
+        for (size_t i = 0; i < num_rows; ++i) {
+            set_is_null(offset + i, val);
+        }
+    }
+
+    inline bool* get_null_signs(size_t idx) { return _null_signs + idx; }
+
+
+    /**
+     * Change the number of slots to at least the given capacity.
+     * This function is not recursive into subtypes.
+     * Tips: This function will change `_capacity` attribute.
+     */
+    virtual Status resize(size_t new_cap);
+
+    // Get the start of the data.
+    virtual uint8_t* data() const = 0;
+
+    // Get the idx's cell_ptr
+    virtual const uint8_t* cell_ptr(size_t idx) const = 0;
+
+    // Get thr idx's cell_ptr for write
+    virtual uint8_t* mutable_cell_ptr(size_t idx) const = 0;
+
+
+    static Status create(size_t init_capacity,
+                         bool is_nullable,
+                         const TypeInfo* type_info,
+                         std::unique_ptr<ColumnVectorBatch>* column_vector_batch);
+
+protected:
+    static Status resize_buff(size_t old_cap, size_t new_cap, uint8_t** _buf);
+
+private:
+    const TypeInfo* _type_info;
+    size_t _capacity;
+    const bool _nullable;
+    bool* _null_signs = nullptr;
+};
+
+struct ScalarColumnVectorBatch : public ColumnVectorBatch {
+public:
+    explicit ScalarColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity);
+
+    ~ScalarColumnVectorBatch() override ;
+
+    Status resize(size_t new_cap) override;
+
+    // Get the start of the data.
+    uint8_t* data() const override;
+
+    // Get the idx's cell_ptr
+    const uint8_t* cell_ptr(size_t idx) const override;
+
+    // Get thr idx's cell_ptr for write
+    uint8_t* mutable_cell_ptr(size_t idx) const override;
+
+private:
+    uint8_t* _data = nullptr;
+};
+
+struct ListColumnVectorBatch : public ColumnVectorBatch {
+public:
+    explicit ListColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity);
+    ~ListColumnVectorBatch() override;
+    Status resize(size_t new_cap) override;
+
+    ColumnVectorBatch* get_elements() const { return _elements.get(); }
 
 Review comment:
   ```suggestion
       ColumnVectorBatch* elements() const { return _elements.get(); }
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on issue #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on issue #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#issuecomment-597494552
 
 
   @liutang123 Could you rebase your branch against master again? There are some merge conflicts

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
gaodayue commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r394125304
 
 

 ##########
 File path: be/src/olap/column_vector.cpp
 ##########
 @@ -0,0 +1,150 @@
+// 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 "column_vector.h"
+
+namespace doris {
+
+ColumnVectorBatch::~ColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_null_signs);
+}
+
+Status ColumnVectorBatch::resize(size_t new_cap) {
+    if (_nullable) {
+        resize_buff(_capacity, new_cap, reinterpret_cast<uint8_t**>(&_null_signs));
+    }
+    _capacity = new_cap;
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::resize_buff(size_t copy_bytes, size_t new_cap, uint8_t** _buf_ptr) {
+    const uint8_t* old_buf = *_buf_ptr;
+    if (old_buf) {
+        *_buf_ptr = new uint8_t[new_cap];
+        memcpy(*_buf_ptr, old_buf, copy_bytes);
+        delete[] old_buf;
+    } else {
+        *_buf_ptr = new uint8_t[new_cap];
+    }
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::create(size_t init_capacity,
+        bool is_nullable,
+        const TypeInfo* type_info,
+        std::unique_ptr<ColumnVectorBatch>* column_vector_batch) {
+    if (is_scalar_type(type_info->type())) {
+        std::unique_ptr<ColumnVectorBatch> local(
+                new ScalarColumnVectorBatch(type_info, is_nullable, init_capacity));
+        *column_vector_batch = std::move(local);
 
 Review comment:
   ```suggestion
           column_vector_batch->reset(new ScalarColumnVectorBatch(type_info, is_nullable, init_capacity));
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r395269663
 
 

 ##########
 File path: gensrc/proto/olap_file.proto
 ##########
 @@ -265,6 +265,8 @@ message ColumnPB {
     optional int32 referenced_column_id = 13; //   
     optional string referenced_column = 14; // ColumnMessage.referenced_column?
     optional bool has_bitmap_index = 15 [default=false]; // ColumnMessage.has_bitmap_index
+    repeated ColumnPB children_columns = 16;
+    repeated string children_column_names = 17;
 
 Review comment:
   For struct type.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r393164018
 
 

 ##########
 File path: be/src/olap/column_vector.cpp
 ##########
 @@ -0,0 +1,150 @@
+// 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 "column_vector.h"
+
+namespace doris {
+
+ColumnVectorBatch::~ColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_null_signs);
+}
+
+Status ColumnVectorBatch::resize(size_t new_cap) {
+    if (_nullable) {
 
 Review comment:
   if new_cap is less than the _capacity, should it avoid to reallocate the memory and copy?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r392265602
 
 

 ##########
 File path: be/src/olap/rowset/segment_v2/column_reader.h
 ##########
 @@ -127,12 +127,26 @@ class ColumnReader {
 
     PagePointer get_dict_page_pointer() const { return _meta.dict_page(); }
 
-private:
+    // do not call this method in classes other than subclasses of ColumnReader and ColumnReader
 
 Review comment:
   ```suggestion
       // do not call this method in classes other than subclasses of ColumnReader
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r392286839
 
 

 ##########
 File path: be/src/olap/column_vector.h
 ##########
 @@ -0,0 +1,150 @@
+// 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/status.h"
+#include "olap/olap_common.h"
+#include "olap/rowset/segment_v2/common.h" // for ordinal_t
+#include "olap/types.h"
+
+namespace doris {
+
+// struct that contains column data(null bitmap), data array in sub class.
+struct ColumnVectorBatch {
 
 Review comment:
   Add a UT for  `ColumnVectorBatch`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r394431823
 
 

 ##########
 File path: be/src/olap/column_vector.cpp
 ##########
 @@ -0,0 +1,150 @@
+// 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 "column_vector.h"
+
+namespace doris {
+
+ColumnVectorBatch::~ColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_null_signs);
+}
+
+Status ColumnVectorBatch::resize(size_t new_cap) {
+    if (_nullable) {
+        resize_buff(_capacity, new_cap, reinterpret_cast<uint8_t**>(&_null_signs));
+    }
+    _capacity = new_cap;
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::resize_buff(size_t copy_bytes, size_t new_cap, uint8_t** _buf_ptr) {
+    const uint8_t* old_buf = *_buf_ptr;
+    if (old_buf) {
+        *_buf_ptr = new uint8_t[new_cap];
+        memcpy(*_buf_ptr, old_buf, copy_bytes);
+        delete[] old_buf;
+    } else {
+        *_buf_ptr = new uint8_t[new_cap];
+    }
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::create(size_t init_capacity,
+        bool is_nullable,
+        const TypeInfo* type_info,
+        std::unique_ptr<ColumnVectorBatch>* column_vector_batch) {
+    if (is_scalar_type(type_info->type())) {
+        std::unique_ptr<ColumnVectorBatch> local(
+                new ScalarColumnVectorBatch(type_info, is_nullable, init_capacity));
+        *column_vector_batch = std::move(local);
+        return Status::OK();
+    } else {
+        switch (type_info->type()) {
+        case FieldType::OLAP_FIELD_TYPE_LIST: {
+            std::unique_ptr<ColumnVectorBatch> local(new ListColumnVectorBatch(type_info, is_nullable, init_capacity));
+            *column_vector_batch = std::move(local);
+            return Status::OK();
+        }
+        default:
+            return Status::NotSupported("unsupported type for ColumnVectorBatch: " + std::to_string(type_info->type()));
+        }
+    }
+}
+
+ScalarColumnVectorBatch::ScalarColumnVectorBatch(const TypeInfo* type_info, bool is_nullable, size_t init_capacity)
+: ColumnVectorBatch(type_info, is_nullable) {
+    resize(init_capacity);
+}
+ScalarColumnVectorBatch::~ScalarColumnVectorBatch() {
+    SAFE_DELETE_ARRAY(_data);
+}
+
+Status ScalarColumnVectorBatch::resize(size_t new_cap) {
+    if (get_capacity() < new_cap) { // before first init, _capacity is 0.
+        size_t type_size = type_info()->size();
+        auto copy_bytes = get_capacity() * type_size;
+        size_t new_data_size = new_cap * type_size;
+        resize_buff(copy_bytes, new_data_size, &_data);
+        return ColumnVectorBatch::resize(new_cap);
 
 Review comment:
   The commont of the ColumnVectorBatch::resize is:
   ```Change the number of slots to at least the given capacity.
   This function is not recursive into subtypes.
   Tips: This function will change `_capacity` attribute.```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
liutang123 commented on a change in pull request #3025:
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r521078102



##########
File path: be/src/olap/rowset/segment_v2/column_reader.cpp
##########
@@ -296,6 +317,115 @@ Status ColumnReader::seek_at_or_before(ordinal_t ordinal, OrdinalPageIndexIterat
     return Status::OK();
 }
 
+Status ColumnReader::new_iterator(ColumnIterator** iterator) {
+    if (is_scalar_type((FieldType)_meta.type())) {
+        *iterator = new FileColumnIterator(this);
+        return Status::OK();
+    } else {
+        auto type = (FieldType)_meta.type();
+        switch(type) {
+            case FieldType::OLAP_FIELD_TYPE_ARRAY: {
+                ColumnIterator* item_iterator;
+                RETURN_IF_ERROR(_sub_readers[0]->new_iterator(&item_iterator));
+                FileColumnIterator* offset_iterator = new FileColumnIterator(this);
+                *iterator = new ArrayFileColumnIterator(offset_iterator, item_iterator);
+                return Status::OK();
+            }
+            default:
+                return Status::NotSupported("unsupported type to create iterator: " + std::to_string(type));
+        }
+    }
+}
+
+////////////////////////////////////////////////////////////////////////////////
+
+ArrayFileColumnIterator::ArrayFileColumnIterator(FileColumnIterator* offset_reader, ColumnIterator* item_iterator) {
+    _offset_iterator.reset(offset_reader);
+    _item_iterator.reset(item_iterator);
+}
+
+Status ArrayFileColumnIterator::init(const ColumnIteratorOptions& opts) {
+    RETURN_IF_ERROR(_offset_iterator->init(opts));
+    RETURN_IF_ERROR(_item_iterator->init(opts));
+    TypeInfo* bigint_type_info = get_scalar_type_info(FieldType::OLAP_FIELD_TYPE_BIGINT);
+    RETURN_IF_ERROR(ColumnVectorBatch::create(1024, _offset_iterator->is_nullable(), bigint_type_info, nullptr, &_offset_batch));
+    return Status::OK();
+}
+
+// every invoke this method, _offset_batch will be cover, so this method is not thread safe.
+Status ArrayFileColumnIterator::next_batch(size_t* n, ColumnBlockView* dst, bool* has_null) {
+    // 1. read n offsets into  _offset_batch;
+    _offset_batch->resize(*n + 1);
+    ColumnBlock ordinal_block(_offset_batch.get(), nullptr);
+    ColumnBlockView ordinal_view(&ordinal_block);
+    RETURN_IF_ERROR(_offset_iterator->next_batch(n, &ordinal_view, has_null));
+
+    if (*n == 0) {
+        return Status::OK();
+    }
+
+    // 2. Because we should read n + 1 offsets, so read one more here.
+    PageDecoder* offset_page_decoder = _offset_iterator->get_current_page()->data_decoder;
+    if (offset_page_decoder->has_remaining()) { // not _page->has_remaining()
+        size_t i = 1;
+        offset_page_decoder->peek_next_batch(&i, &ordinal_view); // not null
+        DCHECK(i == 1);
+    } else {
+        *(reinterpret_cast<ordinal_t*>(ordinal_view.data())) =
+                _offset_iterator->get_current_page()->next_array_item_ordinal;
+    }
+    ordinal_view.set_null_bits(1, false);
+    ordinal_view.advance(1);
+
+    // 3. For nullable data,fill null ordinals from last to start: 0 N N 3 N 5 -> 0 3 3 3 5 5

Review comment:
       The last ordinal can not be null.
   if current page has more data, peek_next_batch will return the next not null ordinal, else next_array_item_ordinal will return then next not null ordinal.




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

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] kangkaisen commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on a change in pull request #3025:
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r516385798



##########
File path: be/src/olap/collection.h
##########
@@ -0,0 +1,50 @@
+// 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 <iostream>
+
+namespace doris {
+
+// cpp type for ARRAY
+struct Collection {
+    size_t length;
+    // null bitmap
+    bool* null_signs;

Review comment:
       You could add a has_null flag. when reading array column, if has_null is false, we could only read the data column.

##########
File path: gensrc/proto/segment_v2.proto
##########
@@ -151,6 +151,12 @@ message ColumnMetaPB {
     repeated ColumnIndexMetaPB indexes = 8;
     // pointer to dictionary page when using DICT_ENCODING
     optional PagePointerPB dict_page = 9;
+
+    repeated ColumnMetaPB children_columns = 10;
+    repeated string children_column_names = 11;

Review comment:
       OK




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

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] wangbo commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
wangbo commented on a change in pull request #3025:
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r520544184



##########
File path: be/src/olap/rowset/segment_v2/column_reader.cpp
##########
@@ -296,6 +317,115 @@ Status ColumnReader::seek_at_or_before(ordinal_t ordinal, OrdinalPageIndexIterat
     return Status::OK();
 }
 
+Status ColumnReader::new_iterator(ColumnIterator** iterator) {
+    if (is_scalar_type((FieldType)_meta.type())) {
+        *iterator = new FileColumnIterator(this);
+        return Status::OK();
+    } else {
+        auto type = (FieldType)_meta.type();
+        switch(type) {
+            case FieldType::OLAP_FIELD_TYPE_ARRAY: {
+                ColumnIterator* item_iterator;
+                RETURN_IF_ERROR(_sub_readers[0]->new_iterator(&item_iterator));
+                FileColumnIterator* offset_iterator = new FileColumnIterator(this);
+                *iterator = new ArrayFileColumnIterator(offset_iterator, item_iterator);
+                return Status::OK();
+            }
+            default:
+                return Status::NotSupported("unsupported type to create iterator: " + std::to_string(type));
+        }
+    }
+}
+
+////////////////////////////////////////////////////////////////////////////////
+
+ArrayFileColumnIterator::ArrayFileColumnIterator(FileColumnIterator* offset_reader, ColumnIterator* item_iterator) {
+    _offset_iterator.reset(offset_reader);
+    _item_iterator.reset(item_iterator);
+}
+
+Status ArrayFileColumnIterator::init(const ColumnIteratorOptions& opts) {
+    RETURN_IF_ERROR(_offset_iterator->init(opts));
+    RETURN_IF_ERROR(_item_iterator->init(opts));
+    TypeInfo* bigint_type_info = get_scalar_type_info(FieldType::OLAP_FIELD_TYPE_BIGINT);
+    RETURN_IF_ERROR(ColumnVectorBatch::create(1024, _offset_iterator->is_nullable(), bigint_type_info, nullptr, &_offset_batch));
+    return Status::OK();
+}
+
+// every invoke this method, _offset_batch will be cover, so this method is not thread safe.
+Status ArrayFileColumnIterator::next_batch(size_t* n, ColumnBlockView* dst, bool* has_null) {
+    // 1. read n offsets into  _offset_batch;
+    _offset_batch->resize(*n + 1);
+    ColumnBlock ordinal_block(_offset_batch.get(), nullptr);
+    ColumnBlockView ordinal_view(&ordinal_block);
+    RETURN_IF_ERROR(_offset_iterator->next_batch(n, &ordinal_view, has_null));
+
+    if (*n == 0) {
+        return Status::OK();
+    }
+
+    // 2. Because we should read n + 1 offsets, so read one more here.
+    PageDecoder* offset_page_decoder = _offset_iterator->get_current_page()->data_decoder;
+    if (offset_page_decoder->has_remaining()) { // not _page->has_remaining()
+        size_t i = 1;
+        offset_page_decoder->peek_next_batch(&i, &ordinal_view); // not null
+        DCHECK(i == 1);
+    } else {
+        *(reinterpret_cast<ordinal_t*>(ordinal_view.data())) =
+                _offset_iterator->get_current_page()->next_array_item_ordinal;
+    }
+    ordinal_view.set_null_bits(1, false);
+    ordinal_view.advance(1);
+
+    // 3. For nullable data,fill null ordinals from last to start: 0 N N 3 N 5 -> 0 3 3 3 5 5

Review comment:
       If the input is ```0 N N 3 N N```
   Then the output is ```0 3 3 3 N N``` ? 
   How to ensure the data in Nth position  is not null?




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

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] wangbo commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
wangbo commented on a change in pull request #3025:
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r522136742



##########
File path: be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp
##########
@@ -215,17 +216,224 @@ void test_nullable_data(uint8_t* src_data, uint8_t* src_is_null, int num_rows, s
     }
 }
 
+template<FieldType item_type, EncodingTypePB item_encoding, EncodingTypePB array_encoding>
+void test_array_nullable_data(Collection* src_data, uint8_t* src_is_null, int num_rows, string test_name) {
+    Collection* src = src_data;
+    ColumnMetaPB meta;
+    TabletColumn list_column(OLAP_FIELD_AGGREGATION_NONE, OLAP_FIELD_TYPE_ARRAY);
+    int32 item_length = 0;
+    if (item_type == OLAP_FIELD_TYPE_CHAR || item_type == OLAP_FIELD_TYPE_VARCHAR) {
+        item_length = 10;
+    }
+    TabletColumn item_column(OLAP_FIELD_AGGREGATION_NONE, item_type, true, 0, item_length);
+    list_column.add_sub_column(item_column);
+    Field* field = FieldFactory::create(list_column);
+
+    // write data
+    string fname = TEST_DIR + "/" + test_name;
+    {
+        std::unique_ptr<fs::WritableBlock> wblock;
+        fs::CreateBlockOptions opts({ fname });
+        Status st = fs::fs_util::block_mgr_for_ut()->create_block(opts, &wblock);
+        ASSERT_TRUE(st.ok()) << st.get_error_msg();
+
+        ColumnWriterOptions writer_opts;
+        writer_opts.meta = &meta;
+        writer_opts.meta->set_column_id(0);
+        writer_opts.meta->set_unique_id(0);
+        writer_opts.meta->set_type(OLAP_FIELD_TYPE_ARRAY);
+        writer_opts.meta->set_length(0);
+        writer_opts.meta->set_encoding(array_encoding);
+        writer_opts.meta->set_compression(segment_v2::CompressionTypePB::LZ4F);
+        writer_opts.meta->set_is_nullable(true);
+        writer_opts.data_page_size = 5 * 8;
+
+        ColumnMetaPB* child_meta = meta.add_children_columns();
+
+        child_meta->set_column_id(1);
+        child_meta->set_unique_id(1);
+        child_meta->set_type(item_type);
+        child_meta->set_length(item_length);
+        child_meta->set_encoding(item_encoding);
+        child_meta->set_compression(segment_v2::CompressionTypePB::LZ4F);
+        child_meta->set_is_nullable(true);
+
+        std::unique_ptr<ColumnWriter> writer;
+        ColumnWriter::create(writer_opts, &list_column, wblock.get(), &writer);
+        st = writer->init();
+        ASSERT_TRUE(st.ok()) << st.to_string();
+
+        for (int i = 0; i < num_rows; ++i) {
+            st = writer->append(BitmapTest(src_is_null, i), src + i);
+            ASSERT_TRUE(st.ok());
+        }
+
+        st = writer->finish();
+        ASSERT_TRUE(st.ok());
+
+        st = writer->write_data();
+        ASSERT_TRUE(st.ok());
+        st = writer->write_ordinal_index();
+        ASSERT_TRUE(st.ok());
+
+        // close the file
+        ASSERT_TRUE(wblock->close().ok());
+    }
+    TypeInfo* type_info = get_type_info(&meta);
+
+    // read and check
+    {
+        ColumnReaderOptions reader_opts;
+        std::unique_ptr<ColumnReader> reader;
+        auto st = ColumnReader::create(reader_opts, meta, num_rows, fname, &reader);
+        ASSERT_TRUE(st.ok());
+
+        ColumnIterator* iter = nullptr;
+        st = reader->new_iterator(&iter);
+        ASSERT_TRUE(st.ok());
+        std::unique_ptr<fs::ReadableBlock> rblock;
+        fs::BlockManager* block_manager = fs::fs_util::block_mgr_for_ut();
+        st = block_manager->open_block(fname, &rblock);
+        ASSERT_TRUE(st.ok());
+        ColumnIteratorOptions iter_opts;
+        OlapReaderStatistics stats;
+        iter_opts.stats = &stats;
+        iter_opts.rblock = rblock.get();
+        st = iter->init(iter_opts);
+        ASSERT_TRUE(st.ok());
+        // sequence read
+        {
+            st = iter->seek_to_first();
+            ASSERT_TRUE(st.ok()) << st.to_string();
+
+            MemTracker tracker;
+            MemPool pool(&tracker);
+            std::unique_ptr<ColumnVectorBatch> cvb;
+            ColumnVectorBatch::create(0, true, type_info, field, &cvb);
+            cvb->resize(1024);
+            ColumnBlock col(cvb.get(), &pool);
+
+            int idx = 0;
+            while (true) {
+                size_t rows_read = 1024;
+                ColumnBlockView dst(&col);
+                st = iter->next_batch(&rows_read, &dst);
+                ASSERT_TRUE(st.ok());
+                for (int j = 0; j < rows_read; ++j) {
+                    ASSERT_EQ(BitmapTest(src_is_null, idx), col.is_null(j));
+                    if (!col.is_null(j)) {
+                        ASSERT_TRUE(type_info->equal(&src[idx], col.cell_ptr(j)));
+                    }
+                    ++idx;
+                }
+                if (rows_read < 1024) {
+                    break;
+                }
+            }
+        }
+        // seek read
+        {
+            MemTracker tracker;
+            MemPool pool(&tracker);
+            std::unique_ptr<ColumnVectorBatch> cvb;
+            ColumnVectorBatch::create(0, true, type_info, field, &cvb);
+            cvb->resize(1024);
+            ColumnBlock col(cvb.get(), &pool);
+
+            for (int rowid = 0; rowid < num_rows; rowid += 4025) {
+                st = iter->seek_to_ordinal(rowid);
+                ASSERT_TRUE(st.ok());
+
+                int idx = rowid;
+                size_t rows_read = 1024;
+                ColumnBlockView dst(&col);
+
+                st = iter->next_batch(&rows_read, &dst);
+                ASSERT_TRUE(st.ok());
+                for (int j = 0; j < rows_read; ++j) {
+                    ASSERT_EQ(BitmapTest(src_is_null, idx), col.is_null(j));
+                    if (!col.is_null(j)) {
+                        ASSERT_TRUE(type_info->equal(&src[idx], col.cell_ptr(j)));
+                    }
+                    ++idx;
+                }
+            }
+        }
+        delete iter;
+    }
+    delete field;
+}
+
+
+TEST_F(ColumnReaderWriterTest, test_array_type) {
+    size_t num_list = 24 * 1024;
+    size_t num_item = num_list * 3;
+
+    uint8_t* array_is_null = new uint8_t[BitmapSize(num_list)];
+    Collection* array_val = new Collection[num_list];
+    bool* item_is_null = new bool[num_item];
+    uint8_t* item_val = new uint8_t[num_item];
+    for (int i = 0; i < num_item; ++i) {
+        item_val[i] = i;
+        item_is_null[i] = (i % 4) == 0;
+        if (i % 3 == 0) {
+            size_t list_index = i / 3;
+            bool is_null = (list_index % 4) == 1;
+            BitmapChange(array_is_null, list_index, is_null);
+            if (is_null) {
+                continue;
+            }
+            array_val[list_index].data = &item_val[i];
+            array_val[list_index].null_signs = &item_is_null[i];
+            array_val[list_index].length = 3;
+        }
+    }
+    test_array_nullable_data<OLAP_FIELD_TYPE_TINYINT, BIT_SHUFFLE, BIT_SHUFFLE>(array_val, array_is_null, num_list, "null_array_bs");
+
+    delete[] array_val;
+    delete[] item_val;
+    delete[] item_is_null;
+
+    array_val = new Collection[num_list];
+    Slice* varchar_vals = new Slice[3];
+    item_is_null = new bool[3];
+    for (int i = 0; i < 3; ++i) {
+        item_is_null[i] = i == 1;
+        if (i != 1) {
+            set_column_value_by_type(OLAP_FIELD_TYPE_VARCHAR, i, (char*)&varchar_vals[i], &_pool);
+        }
+    }
+    for (int i = 0; i <  num_list; ++i) {
+        bool is_null = (i % 4) == 1;
+        BitmapChange(array_is_null, i, is_null);
+        if (is_null) {
+            continue;
+        }
+        array_val[i].data = varchar_vals;
+        array_val[i].null_signs = item_is_null;
+        array_val[i].length = 3;
+    }
+    test_array_nullable_data<OLAP_FIELD_TYPE_VARCHAR, DICT_ENCODING, BIT_SHUFFLE>(array_val, array_is_null, num_list, "null_array_chars");

Review comment:
       I think need a ```Nested Array UT```.
   Such as input one row ```[[1,2],[3,4]]```




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

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] liutang123 closed pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
liutang123 closed pull request #3025:
URL: https://github.com/apache/incubator-doris/pull/3025


   


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

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] wangbo commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
wangbo commented on a change in pull request #3025:
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r520324884



##########
File path: be/src/olap/rowset/segment_v2/column_writer.h
##########
@@ -91,23 +93,95 @@ class ColumnWriter {
         return append_nullable(&nullmap, data, 1);
     }
 
-    Status append_nulls(size_t num_rows);
-    Status append(const void* data, size_t num_rows);
     Status append_nullable(const uint8_t* nullmap, const void* data, size_t num_rows);
 
-    uint64_t estimate_buffer_size();
+    Status append_not_nulls(const void* data, size_t num_rows);
+
+    virtual Status append_nulls(size_t num_rows) = 0;
+
+    virtual Status finish_current_page() = 0;
+
+    virtual uint64_t estimate_buffer_size() = 0;
 
     // finish append data
-    Status finish();
+    virtual Status finish() = 0;
 
     // write all data into file
-    Status write_data();
-    Status write_ordinal_index();
-    Status write_zone_map();
-    Status write_bitmap_index();
-    Status write_bloom_filter_index();
+    virtual Status write_data() = 0;
+
+    virtual Status write_ordinal_index() = 0;
+
+    virtual Status write_zone_map() = 0;
+
+    virtual Status write_bitmap_index() = 0;
+
+    virtual Status write_bloom_filter_index() = 0;
+
+    virtual ordinal_t get_next_rowid() const = 0;
+
+    // used for append not null data.
+    virtual Status append_data(const uint8_t** ptr, size_t num_rows) = 0;
+
+    bool is_nullable() const { return _is_nullable; }
+
+    Field* get_filed() const { return _field.get(); }

Review comment:
       ```suggestion
       Field* get_field() const { return _field.get(); }
   ```




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

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] wangbo commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
wangbo commented on a change in pull request #3025:
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r522136742



##########
File path: be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp
##########
@@ -215,17 +216,224 @@ void test_nullable_data(uint8_t* src_data, uint8_t* src_is_null, int num_rows, s
     }
 }
 
+template<FieldType item_type, EncodingTypePB item_encoding, EncodingTypePB array_encoding>
+void test_array_nullable_data(Collection* src_data, uint8_t* src_is_null, int num_rows, string test_name) {
+    Collection* src = src_data;
+    ColumnMetaPB meta;
+    TabletColumn list_column(OLAP_FIELD_AGGREGATION_NONE, OLAP_FIELD_TYPE_ARRAY);
+    int32 item_length = 0;
+    if (item_type == OLAP_FIELD_TYPE_CHAR || item_type == OLAP_FIELD_TYPE_VARCHAR) {
+        item_length = 10;
+    }
+    TabletColumn item_column(OLAP_FIELD_AGGREGATION_NONE, item_type, true, 0, item_length);
+    list_column.add_sub_column(item_column);
+    Field* field = FieldFactory::create(list_column);
+
+    // write data
+    string fname = TEST_DIR + "/" + test_name;
+    {
+        std::unique_ptr<fs::WritableBlock> wblock;
+        fs::CreateBlockOptions opts({ fname });
+        Status st = fs::fs_util::block_mgr_for_ut()->create_block(opts, &wblock);
+        ASSERT_TRUE(st.ok()) << st.get_error_msg();
+
+        ColumnWriterOptions writer_opts;
+        writer_opts.meta = &meta;
+        writer_opts.meta->set_column_id(0);
+        writer_opts.meta->set_unique_id(0);
+        writer_opts.meta->set_type(OLAP_FIELD_TYPE_ARRAY);
+        writer_opts.meta->set_length(0);
+        writer_opts.meta->set_encoding(array_encoding);
+        writer_opts.meta->set_compression(segment_v2::CompressionTypePB::LZ4F);
+        writer_opts.meta->set_is_nullable(true);
+        writer_opts.data_page_size = 5 * 8;
+
+        ColumnMetaPB* child_meta = meta.add_children_columns();
+
+        child_meta->set_column_id(1);
+        child_meta->set_unique_id(1);
+        child_meta->set_type(item_type);
+        child_meta->set_length(item_length);
+        child_meta->set_encoding(item_encoding);
+        child_meta->set_compression(segment_v2::CompressionTypePB::LZ4F);
+        child_meta->set_is_nullable(true);
+
+        std::unique_ptr<ColumnWriter> writer;
+        ColumnWriter::create(writer_opts, &list_column, wblock.get(), &writer);
+        st = writer->init();
+        ASSERT_TRUE(st.ok()) << st.to_string();
+
+        for (int i = 0; i < num_rows; ++i) {
+            st = writer->append(BitmapTest(src_is_null, i), src + i);
+            ASSERT_TRUE(st.ok());
+        }
+
+        st = writer->finish();
+        ASSERT_TRUE(st.ok());
+
+        st = writer->write_data();
+        ASSERT_TRUE(st.ok());
+        st = writer->write_ordinal_index();
+        ASSERT_TRUE(st.ok());
+
+        // close the file
+        ASSERT_TRUE(wblock->close().ok());
+    }
+    TypeInfo* type_info = get_type_info(&meta);
+
+    // read and check
+    {
+        ColumnReaderOptions reader_opts;
+        std::unique_ptr<ColumnReader> reader;
+        auto st = ColumnReader::create(reader_opts, meta, num_rows, fname, &reader);
+        ASSERT_TRUE(st.ok());
+
+        ColumnIterator* iter = nullptr;
+        st = reader->new_iterator(&iter);
+        ASSERT_TRUE(st.ok());
+        std::unique_ptr<fs::ReadableBlock> rblock;
+        fs::BlockManager* block_manager = fs::fs_util::block_mgr_for_ut();
+        st = block_manager->open_block(fname, &rblock);
+        ASSERT_TRUE(st.ok());
+        ColumnIteratorOptions iter_opts;
+        OlapReaderStatistics stats;
+        iter_opts.stats = &stats;
+        iter_opts.rblock = rblock.get();
+        st = iter->init(iter_opts);
+        ASSERT_TRUE(st.ok());
+        // sequence read
+        {
+            st = iter->seek_to_first();
+            ASSERT_TRUE(st.ok()) << st.to_string();
+
+            MemTracker tracker;
+            MemPool pool(&tracker);
+            std::unique_ptr<ColumnVectorBatch> cvb;
+            ColumnVectorBatch::create(0, true, type_info, field, &cvb);
+            cvb->resize(1024);
+            ColumnBlock col(cvb.get(), &pool);
+
+            int idx = 0;
+            while (true) {
+                size_t rows_read = 1024;
+                ColumnBlockView dst(&col);
+                st = iter->next_batch(&rows_read, &dst);
+                ASSERT_TRUE(st.ok());
+                for (int j = 0; j < rows_read; ++j) {
+                    ASSERT_EQ(BitmapTest(src_is_null, idx), col.is_null(j));
+                    if (!col.is_null(j)) {
+                        ASSERT_TRUE(type_info->equal(&src[idx], col.cell_ptr(j)));
+                    }
+                    ++idx;
+                }
+                if (rows_read < 1024) {
+                    break;
+                }
+            }
+        }
+        // seek read
+        {
+            MemTracker tracker;
+            MemPool pool(&tracker);
+            std::unique_ptr<ColumnVectorBatch> cvb;
+            ColumnVectorBatch::create(0, true, type_info, field, &cvb);
+            cvb->resize(1024);
+            ColumnBlock col(cvb.get(), &pool);
+
+            for (int rowid = 0; rowid < num_rows; rowid += 4025) {
+                st = iter->seek_to_ordinal(rowid);
+                ASSERT_TRUE(st.ok());
+
+                int idx = rowid;
+                size_t rows_read = 1024;
+                ColumnBlockView dst(&col);
+
+                st = iter->next_batch(&rows_read, &dst);
+                ASSERT_TRUE(st.ok());
+                for (int j = 0; j < rows_read; ++j) {
+                    ASSERT_EQ(BitmapTest(src_is_null, idx), col.is_null(j));
+                    if (!col.is_null(j)) {
+                        ASSERT_TRUE(type_info->equal(&src[idx], col.cell_ptr(j)));
+                    }
+                    ++idx;
+                }
+            }
+        }
+        delete iter;
+    }
+    delete field;
+}
+
+
+TEST_F(ColumnReaderWriterTest, test_array_type) {
+    size_t num_list = 24 * 1024;
+    size_t num_item = num_list * 3;
+
+    uint8_t* array_is_null = new uint8_t[BitmapSize(num_list)];
+    Collection* array_val = new Collection[num_list];
+    bool* item_is_null = new bool[num_item];
+    uint8_t* item_val = new uint8_t[num_item];
+    for (int i = 0; i < num_item; ++i) {
+        item_val[i] = i;
+        item_is_null[i] = (i % 4) == 0;
+        if (i % 3 == 0) {
+            size_t list_index = i / 3;
+            bool is_null = (list_index % 4) == 1;
+            BitmapChange(array_is_null, list_index, is_null);
+            if (is_null) {
+                continue;
+            }
+            array_val[list_index].data = &item_val[i];
+            array_val[list_index].null_signs = &item_is_null[i];
+            array_val[list_index].length = 3;
+        }
+    }
+    test_array_nullable_data<OLAP_FIELD_TYPE_TINYINT, BIT_SHUFFLE, BIT_SHUFFLE>(array_val, array_is_null, num_list, "null_array_bs");
+
+    delete[] array_val;
+    delete[] item_val;
+    delete[] item_is_null;
+
+    array_val = new Collection[num_list];
+    Slice* varchar_vals = new Slice[3];
+    item_is_null = new bool[3];
+    for (int i = 0; i < 3; ++i) {
+        item_is_null[i] = i == 1;
+        if (i != 1) {
+            set_column_value_by_type(OLAP_FIELD_TYPE_VARCHAR, i, (char*)&varchar_vals[i], &_pool);
+        }
+    }
+    for (int i = 0; i <  num_list; ++i) {
+        bool is_null = (i % 4) == 1;
+        BitmapChange(array_is_null, i, is_null);
+        if (is_null) {
+            continue;
+        }
+        array_val[i].data = varchar_vals;
+        array_val[i].null_signs = item_is_null;
+        array_val[i].length = 3;
+    }
+    test_array_nullable_data<OLAP_FIELD_TYPE_VARCHAR, DICT_ENCODING, BIT_SHUFFLE>(array_val, array_is_null, num_list, "null_array_chars");

Review comment:
       I think need a ```Nested Array UT``.
   Such as input one row ```[[1,2],[3,4]]```




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

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] wangbo commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
wangbo commented on a change in pull request #3025:
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r520326066



##########
File path: be/src/olap/rowset/segment_v2/column_writer.cpp
##########
@@ -365,5 +415,86 @@ Status ColumnWriter::_finish_current_page() {
     return Status::OK();
 }
 
+////////////////////////////////////////////////////////////////////////////////
+
+ArrayColumnWriter::ArrayColumnWriter(const ColumnWriterOptions& opts,
+                         std::unique_ptr<Field> field,
+                                     ScalarColumnWriter* offset_writer,
+                         std::unique_ptr<ColumnWriter> item_writer):
+        ColumnWriter(std::move(field), opts.meta->is_nullable()), _item_writer(std::move(item_writer)) {
+    _offset_writer.reset(offset_writer);
+}
+
+Status ArrayColumnWriter::init() {
+    RETURN_IF_ERROR(_offset_writer->init());
+    RETURN_IF_ERROR(_item_writer->init());
+    _offset_writer->register_flush_page_callback(this);
+    return Status::OK();
+}
+
+Status ArrayColumnWriter::put_extra_info_in_page(DataPageFooterPB* footer) {
+    footer->set_next_array_item_ordinal( _item_writer->get_next_rowid());
+    return Status::OK();
+}
+
+// Now we can only write data one by one.
+Status ArrayColumnWriter::append_data(const uint8_t** ptr, size_t num_rows) {
+    size_t remaining = num_rows;
+    const auto* col_cursor = reinterpret_cast<const Collection*>(*ptr);
+    while (remaining > 0) {
+        // TODO llj: bulk write
+        size_t num_written = 1;
+        ordinal_t next_item_ordinal = _item_writer->get_next_rowid();
+        ordinal_t* next_item_ordinal_ptr = &next_item_ordinal;
+        RETURN_IF_ERROR(_offset_writer->append_data((const uint8_t**)&next_item_ordinal_ptr, num_written));
+        DCHECK(num_written == 1);

Review comment:
       This ```DCHECK```  for ```num_written``` seems meaningless.
   ```num_written``` 's value never changed.
   




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

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] liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
liutang123 commented on a change in pull request #3025:
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r521096363



##########
File path: be/src/olap/column_vector.cpp
##########
@@ -0,0 +1,212 @@
+// 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 "column_vector.h"
+#include "olap/field.h"
+
+namespace doris {
+
+ColumnVectorBatch::~ColumnVectorBatch() = default;
+
+Status ColumnVectorBatch::resize(size_t new_cap) {
+    if (_nullable) {
+        _null_signs.resize(new_cap);
+    }
+    _capacity = new_cap;
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::create(size_t init_capacity,
+        bool is_nullable,
+        const TypeInfo* type_info,
+        Field* field,
+        std::unique_ptr<ColumnVectorBatch>* column_vector_batch) {
+    if (is_scalar_type(type_info->type())) {
+        std::unique_ptr<ColumnVectorBatch> local;
+        switch (type_info->type()) {
+            case OLAP_FIELD_TYPE_BOOL:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_BOOL>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_TINYINT:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_TINYINT>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_SMALLINT:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_SMALLINT>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_INT:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_INT>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_UNSIGNED_INT:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_UNSIGNED_INT>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_BIGINT:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_BIGINT>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_UNSIGNED_BIGINT:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_UNSIGNED_BIGINT>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_LARGEINT:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_LARGEINT>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_FLOAT:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_FLOAT>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_DOUBLE:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_DOUBLE>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_DECIMAL:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_DECIMAL>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_DATE:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_DATE>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_DATETIME:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_DATETIME>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_CHAR:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_CHAR>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_VARCHAR:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_VARCHAR>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_HLL:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_HLL>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_OBJECT:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_OBJECT>::CppType>(type_info, is_nullable));
+                break;
+            default:
+                return Status::NotSupported("unsupported type for ColumnVectorBatch: " + std::to_string(type_info->type()));
+        }
+        RETURN_IF_ERROR(local->resize(init_capacity));
+        *column_vector_batch = std::move(local);
+        return Status::OK();
+    } else {
+        switch (type_info->type()) {
+        case FieldType::OLAP_FIELD_TYPE_ARRAY: {
+            if (field == nullptr) {
+                return Status::NotSupported("When create ArrayColumnVectorBatch, `Field` is indispensable");
+            }
+            std::unique_ptr<ColumnVectorBatch> local(new ArrayColumnVectorBatch(type_info, is_nullable, init_capacity, field));
+            RETURN_IF_ERROR(local->resize(init_capacity));
+            *column_vector_batch = std::move(local);
+            return Status::OK();
+        }
+        default:
+            return Status::NotSupported("unsupported type for ColumnVectorBatch: " + std::to_string(type_info->type()));
+        }
+    }
+}
+
+template <class ScalarType>
+ScalarColumnVectorBatch<ScalarType>::ScalarColumnVectorBatch(const TypeInfo* type_info, bool is_nullable)
+: ColumnVectorBatch(type_info, is_nullable), _data(0) {
+}
+
+template <class ScalarType>
+ScalarColumnVectorBatch<ScalarType>::~ScalarColumnVectorBatch() = default;
+
+template <class ScalarType>
+Status ScalarColumnVectorBatch<ScalarType>::resize(size_t new_cap) {
+    if (capacity() < new_cap) { // before first init, _capacity is 0.
+        RETURN_IF_ERROR(ColumnVectorBatch::resize(new_cap));
+        _data.resize(new_cap);
+    }
+    return Status::OK();
+}
+
+ArrayColumnVectorBatch::ArrayColumnVectorBatch(
+        const TypeInfo* type_info,
+        bool is_nullable,
+        size_t init_capacity,
+        Field* field) : ColumnVectorBatch(type_info, is_nullable), _data(0), _item_offsets(1) {
+    auto array_type_info = reinterpret_cast<const ArrayTypeInfo*>(type_info);
+    _item_offsets[0] = 0;
+    ColumnVectorBatch::create(
+            init_capacity * 2,

Review comment:
       This is based on item's number is usually bigger than array's.




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

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] liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
liutang123 commented on a change in pull request #3025:
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r523758008



##########
File path: be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp
##########
@@ -215,17 +216,224 @@ void test_nullable_data(uint8_t* src_data, uint8_t* src_is_null, int num_rows, s
     }
 }
 
+template<FieldType item_type, EncodingTypePB item_encoding, EncodingTypePB array_encoding>
+void test_array_nullable_data(Collection* src_data, uint8_t* src_is_null, int num_rows, string test_name) {
+    Collection* src = src_data;
+    ColumnMetaPB meta;
+    TabletColumn list_column(OLAP_FIELD_AGGREGATION_NONE, OLAP_FIELD_TYPE_ARRAY);
+    int32 item_length = 0;
+    if (item_type == OLAP_FIELD_TYPE_CHAR || item_type == OLAP_FIELD_TYPE_VARCHAR) {
+        item_length = 10;
+    }
+    TabletColumn item_column(OLAP_FIELD_AGGREGATION_NONE, item_type, true, 0, item_length);
+    list_column.add_sub_column(item_column);
+    Field* field = FieldFactory::create(list_column);
+
+    // write data
+    string fname = TEST_DIR + "/" + test_name;
+    {
+        std::unique_ptr<fs::WritableBlock> wblock;
+        fs::CreateBlockOptions opts({ fname });
+        Status st = fs::fs_util::block_mgr_for_ut()->create_block(opts, &wblock);
+        ASSERT_TRUE(st.ok()) << st.get_error_msg();
+
+        ColumnWriterOptions writer_opts;
+        writer_opts.meta = &meta;
+        writer_opts.meta->set_column_id(0);
+        writer_opts.meta->set_unique_id(0);
+        writer_opts.meta->set_type(OLAP_FIELD_TYPE_ARRAY);
+        writer_opts.meta->set_length(0);
+        writer_opts.meta->set_encoding(array_encoding);
+        writer_opts.meta->set_compression(segment_v2::CompressionTypePB::LZ4F);
+        writer_opts.meta->set_is_nullable(true);
+        writer_opts.data_page_size = 5 * 8;
+
+        ColumnMetaPB* child_meta = meta.add_children_columns();
+
+        child_meta->set_column_id(1);
+        child_meta->set_unique_id(1);
+        child_meta->set_type(item_type);
+        child_meta->set_length(item_length);
+        child_meta->set_encoding(item_encoding);
+        child_meta->set_compression(segment_v2::CompressionTypePB::LZ4F);
+        child_meta->set_is_nullable(true);
+
+        std::unique_ptr<ColumnWriter> writer;
+        ColumnWriter::create(writer_opts, &list_column, wblock.get(), &writer);
+        st = writer->init();
+        ASSERT_TRUE(st.ok()) << st.to_string();
+
+        for (int i = 0; i < num_rows; ++i) {
+            st = writer->append(BitmapTest(src_is_null, i), src + i);
+            ASSERT_TRUE(st.ok());
+        }
+
+        st = writer->finish();
+        ASSERT_TRUE(st.ok());
+
+        st = writer->write_data();
+        ASSERT_TRUE(st.ok());
+        st = writer->write_ordinal_index();
+        ASSERT_TRUE(st.ok());
+
+        // close the file
+        ASSERT_TRUE(wblock->close().ok());
+    }
+    TypeInfo* type_info = get_type_info(&meta);
+
+    // read and check
+    {
+        ColumnReaderOptions reader_opts;
+        std::unique_ptr<ColumnReader> reader;
+        auto st = ColumnReader::create(reader_opts, meta, num_rows, fname, &reader);
+        ASSERT_TRUE(st.ok());
+
+        ColumnIterator* iter = nullptr;
+        st = reader->new_iterator(&iter);
+        ASSERT_TRUE(st.ok());
+        std::unique_ptr<fs::ReadableBlock> rblock;
+        fs::BlockManager* block_manager = fs::fs_util::block_mgr_for_ut();
+        st = block_manager->open_block(fname, &rblock);
+        ASSERT_TRUE(st.ok());
+        ColumnIteratorOptions iter_opts;
+        OlapReaderStatistics stats;
+        iter_opts.stats = &stats;
+        iter_opts.rblock = rblock.get();
+        st = iter->init(iter_opts);
+        ASSERT_TRUE(st.ok());
+        // sequence read
+        {
+            st = iter->seek_to_first();
+            ASSERT_TRUE(st.ok()) << st.to_string();
+
+            MemTracker tracker;
+            MemPool pool(&tracker);
+            std::unique_ptr<ColumnVectorBatch> cvb;
+            ColumnVectorBatch::create(0, true, type_info, field, &cvb);
+            cvb->resize(1024);
+            ColumnBlock col(cvb.get(), &pool);
+
+            int idx = 0;
+            while (true) {
+                size_t rows_read = 1024;
+                ColumnBlockView dst(&col);
+                st = iter->next_batch(&rows_read, &dst);
+                ASSERT_TRUE(st.ok());
+                for (int j = 0; j < rows_read; ++j) {
+                    ASSERT_EQ(BitmapTest(src_is_null, idx), col.is_null(j));
+                    if (!col.is_null(j)) {
+                        ASSERT_TRUE(type_info->equal(&src[idx], col.cell_ptr(j)));
+                    }
+                    ++idx;
+                }
+                if (rows_read < 1024) {
+                    break;
+                }
+            }
+        }
+        // seek read
+        {
+            MemTracker tracker;
+            MemPool pool(&tracker);
+            std::unique_ptr<ColumnVectorBatch> cvb;
+            ColumnVectorBatch::create(0, true, type_info, field, &cvb);
+            cvb->resize(1024);
+            ColumnBlock col(cvb.get(), &pool);
+
+            for (int rowid = 0; rowid < num_rows; rowid += 4025) {
+                st = iter->seek_to_ordinal(rowid);
+                ASSERT_TRUE(st.ok());
+
+                int idx = rowid;
+                size_t rows_read = 1024;
+                ColumnBlockView dst(&col);
+
+                st = iter->next_batch(&rows_read, &dst);
+                ASSERT_TRUE(st.ok());
+                for (int j = 0; j < rows_read; ++j) {
+                    ASSERT_EQ(BitmapTest(src_is_null, idx), col.is_null(j));
+                    if (!col.is_null(j)) {
+                        ASSERT_TRUE(type_info->equal(&src[idx], col.cell_ptr(j)));
+                    }
+                    ++idx;
+                }
+            }
+        }
+        delete iter;
+    }
+    delete field;
+}
+
+
+TEST_F(ColumnReaderWriterTest, test_array_type) {
+    size_t num_list = 24 * 1024;
+    size_t num_item = num_list * 3;
+
+    uint8_t* array_is_null = new uint8_t[BitmapSize(num_list)];
+    Collection* array_val = new Collection[num_list];
+    bool* item_is_null = new bool[num_item];
+    uint8_t* item_val = new uint8_t[num_item];
+    for (int i = 0; i < num_item; ++i) {
+        item_val[i] = i;
+        item_is_null[i] = (i % 4) == 0;
+        if (i % 3 == 0) {
+            size_t list_index = i / 3;
+            bool is_null = (list_index % 4) == 1;
+            BitmapChange(array_is_null, list_index, is_null);
+            if (is_null) {
+                continue;
+            }
+            array_val[list_index].data = &item_val[i];
+            array_val[list_index].null_signs = &item_is_null[i];
+            array_val[list_index].length = 3;
+        }
+    }
+    test_array_nullable_data<OLAP_FIELD_TYPE_TINYINT, BIT_SHUFFLE, BIT_SHUFFLE>(array_val, array_is_null, num_list, "null_array_bs");
+
+    delete[] array_val;
+    delete[] item_val;
+    delete[] item_is_null;
+
+    array_val = new Collection[num_list];
+    Slice* varchar_vals = new Slice[3];
+    item_is_null = new bool[3];
+    for (int i = 0; i < 3; ++i) {
+        item_is_null[i] = i == 1;
+        if (i != 1) {
+            set_column_value_by_type(OLAP_FIELD_TYPE_VARCHAR, i, (char*)&varchar_vals[i], &_pool);
+        }
+    }
+    for (int i = 0; i <  num_list; ++i) {
+        bool is_null = (i % 4) == 1;
+        BitmapChange(array_is_null, i, is_null);
+        if (is_null) {
+            continue;
+        }
+        array_val[i].data = varchar_vals;
+        array_val[i].null_signs = item_is_null;
+        array_val[i].length = 3;
+    }
+    test_array_nullable_data<OLAP_FIELD_TYPE_VARCHAR, DICT_ENCODING, BIT_SHUFFLE>(array_val, array_is_null, num_list, "null_array_chars");

Review comment:
       Now, `ArrayTypeInfoResolver` just supports scalar array.




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

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] liutang123 commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
liutang123 commented on a change in pull request #3025:
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r516041649



##########
File path: be/src/olap/collection.h
##########
@@ -0,0 +1,50 @@
+// 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 <iostream>
+
+namespace doris {
+
+// cpp type for ARRAY
+struct Collection {
+    size_t length;
+    // null bitmap
+    bool* null_signs;

Review comment:
       Now, all array is nullable aray. Should we design an non-nullable array?
   
   

##########
File path: gensrc/proto/segment_v2.proto
##########
@@ -151,6 +151,12 @@ message ColumnMetaPB {
     repeated ColumnIndexMetaPB indexes = 8;
     // pointer to dictionary page when using DICT_ENCODING
     optional PagePointerPB dict_page = 9;
+
+    repeated ColumnMetaPB children_columns = 10;
+    repeated string children_column_names = 11;

Review comment:
       `children_column_names` can be remove. 
   `num_rows ` is used in `OrdinalIndexReader`




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

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] wangbo commented on a change in pull request #3025: Restructure storage type to support complex types expending

Posted by GitBox <gi...@apache.org>.
wangbo commented on a change in pull request #3025:
URL: https://github.com/apache/incubator-doris/pull/3025#discussion_r520410081



##########
File path: be/src/olap/column_vector.cpp
##########
@@ -0,0 +1,212 @@
+// 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 "column_vector.h"
+#include "olap/field.h"
+
+namespace doris {
+
+ColumnVectorBatch::~ColumnVectorBatch() = default;
+
+Status ColumnVectorBatch::resize(size_t new_cap) {
+    if (_nullable) {
+        _null_signs.resize(new_cap);
+    }
+    _capacity = new_cap;
+    return Status::OK();
+}
+
+Status ColumnVectorBatch::create(size_t init_capacity,
+        bool is_nullable,
+        const TypeInfo* type_info,
+        Field* field,
+        std::unique_ptr<ColumnVectorBatch>* column_vector_batch) {
+    if (is_scalar_type(type_info->type())) {
+        std::unique_ptr<ColumnVectorBatch> local;
+        switch (type_info->type()) {
+            case OLAP_FIELD_TYPE_BOOL:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_BOOL>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_TINYINT:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_TINYINT>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_SMALLINT:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_SMALLINT>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_INT:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_INT>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_UNSIGNED_INT:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_UNSIGNED_INT>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_BIGINT:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_BIGINT>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_UNSIGNED_BIGINT:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_UNSIGNED_BIGINT>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_LARGEINT:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_LARGEINT>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_FLOAT:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_FLOAT>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_DOUBLE:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_DOUBLE>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_DECIMAL:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_DECIMAL>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_DATE:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_DATE>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_DATETIME:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_DATETIME>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_CHAR:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_CHAR>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_VARCHAR:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_VARCHAR>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_HLL:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_HLL>::CppType>(type_info, is_nullable));
+                break;
+            case OLAP_FIELD_TYPE_OBJECT:
+                local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_OBJECT>::CppType>(type_info, is_nullable));
+                break;
+            default:
+                return Status::NotSupported("unsupported type for ColumnVectorBatch: " + std::to_string(type_info->type()));
+        }
+        RETURN_IF_ERROR(local->resize(init_capacity));
+        *column_vector_batch = std::move(local);
+        return Status::OK();
+    } else {
+        switch (type_info->type()) {
+        case FieldType::OLAP_FIELD_TYPE_ARRAY: {
+            if (field == nullptr) {
+                return Status::NotSupported("When create ArrayColumnVectorBatch, `Field` is indispensable");
+            }
+            std::unique_ptr<ColumnVectorBatch> local(new ArrayColumnVectorBatch(type_info, is_nullable, init_capacity, field));
+            RETURN_IF_ERROR(local->resize(init_capacity));
+            *column_vector_batch = std::move(local);
+            return Status::OK();
+        }
+        default:
+            return Status::NotSupported("unsupported type for ColumnVectorBatch: " + std::to_string(type_info->type()));
+        }
+    }
+}
+
+template <class ScalarType>
+ScalarColumnVectorBatch<ScalarType>::ScalarColumnVectorBatch(const TypeInfo* type_info, bool is_nullable)
+: ColumnVectorBatch(type_info, is_nullable), _data(0) {
+}
+
+template <class ScalarType>
+ScalarColumnVectorBatch<ScalarType>::~ScalarColumnVectorBatch() = default;
+
+template <class ScalarType>
+Status ScalarColumnVectorBatch<ScalarType>::resize(size_t new_cap) {
+    if (capacity() < new_cap) { // before first init, _capacity is 0.
+        RETURN_IF_ERROR(ColumnVectorBatch::resize(new_cap));
+        _data.resize(new_cap);
+    }
+    return Status::OK();
+}
+
+ArrayColumnVectorBatch::ArrayColumnVectorBatch(
+        const TypeInfo* type_info,
+        bool is_nullable,
+        size_t init_capacity,
+        Field* field) : ColumnVectorBatch(type_info, is_nullable), _data(0), _item_offsets(1) {
+    auto array_type_info = reinterpret_cast<const ArrayTypeInfo*>(type_info);
+    _item_offsets[0] = 0;
+    ColumnVectorBatch::create(
+            init_capacity * 2,

Review comment:
       why double init_capacity here




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

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