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


##########
be/src/olap/rowset/segment_v2/column_writer.cpp:
##########
@@ -695,5 +808,120 @@ Status ArrayColumnWriter::finish_current_page() {
     return Status::NotSupported("array writer has no data, can not 
finish_current_page");
 }
 
+/// ============================= MapColumnWriter =====================////
+MapColumnWriter::MapColumnWriter(const ColumnWriterOptions& opts, 
std::unique_ptr<Field> field,
+                                     ScalarColumnWriter* null_writer,
+                                     std::unique_ptr<ColumnWriter> key_writer,
+                                     std::unique_ptr<ColumnWriter> 
value_writer)
+        : ColumnWriter(std::move(field), opts.meta->is_nullable()),
+          _key_writer(std::move(key_writer)),
+          _value_writer(std::move(value_writer)),
+          _opts(opts) {
+    if (is_nullable()) {
+        _null_writer.reset(null_writer);
+    }
+}
+
+Status MapColumnWriter::init() {
+    if (is_nullable()) {
+        RETURN_IF_ERROR(_null_writer->init());
+    }
+    RETURN_IF_ERROR(_key_writer->init());
+    RETURN_IF_ERROR(_value_writer->init());
+    return Status::OK();
+}
+
+uint64_t MapColumnWriter::estimate_buffer_size() {
+    size_t estimate =  _key_writer->estimate_buffer_size() +
+                      _value_writer->estimate_buffer_size();
+    if (is_nullable()) {
+        estimate += _null_writer->estimate_buffer_size();
+    }
+    return estimate;
+}
+
+Status MapColumnWriter::finish() {
+    if (is_nullable()) {
+        RETURN_IF_ERROR(_null_writer->finish());
+    }
+    RETURN_IF_ERROR(_key_writer->finish());
+    RETURN_IF_ERROR(_value_writer->finish());
+    return Status::OK();
+}
+
+// todo. make keys and values write
+Status MapColumnWriter::append_data(const uint8_t** ptr, size_t num_rows) {
+    size_t remaining = num_rows;
+    const auto* col_cursor = reinterpret_cast<const MapValue*>(*ptr);
+    while (remaining > 0) {
+        size_t num_written = 1;
+        auto* key_data_ptr = 
const_cast<MapValue*>(col_cursor)->mutable_key_data();
+       const uint8_t* key_ptr = (const uint8_t*)key_data_ptr;
+       RETURN_IF_ERROR(_key_writer->append_data(&key_ptr, 1));
+        auto* val_data_ptr = 
const_cast<MapValue*>(col_cursor)->mutable_value_data();
+       const uint8_t* val_ptr = (const uint8_t*)val_data_ptr;
+       RETURN_IF_ERROR(_value_writer->append_data(&val_ptr, 1));
+        remaining -= num_written;
+        col_cursor += num_written;
+        *ptr += num_written * sizeof(MapValue);
+    }
+
+    if (is_nullable()) {
+        return write_null_column(num_rows, false);

Review Comment:
   warning: use of undeclared identifier 'write_null_column' 
[clang-diagnostic-error]
   ```cpp
           return write_null_column(num_rows, false);
                  ^
   ```
   



##########
be/src/olap/rowset/segment_v2/column_writer.cpp:
##########
@@ -695,5 +808,120 @@
     return Status::NotSupported("array writer has no data, can not 
finish_current_page");
 }
 
+/// ============================= MapColumnWriter =====================////
+MapColumnWriter::MapColumnWriter(const ColumnWriterOptions& opts, 
std::unique_ptr<Field> field,
+                                     ScalarColumnWriter* null_writer,
+                                     std::unique_ptr<ColumnWriter> key_writer,
+                                     std::unique_ptr<ColumnWriter> 
value_writer)
+        : ColumnWriter(std::move(field), opts.meta->is_nullable()),
+          _key_writer(std::move(key_writer)),
+          _value_writer(std::move(value_writer)),
+          _opts(opts) {
+    if (is_nullable()) {
+        _null_writer.reset(null_writer);
+    }
+}
+
+Status MapColumnWriter::init() {
+    if (is_nullable()) {
+        RETURN_IF_ERROR(_null_writer->init());
+    }
+    RETURN_IF_ERROR(_key_writer->init());
+    RETURN_IF_ERROR(_value_writer->init());
+    return Status::OK();
+}
+
+uint64_t MapColumnWriter::estimate_buffer_size() {
+    size_t estimate =  _key_writer->estimate_buffer_size() +
+                      _value_writer->estimate_buffer_size();
+    if (is_nullable()) {
+        estimate += _null_writer->estimate_buffer_size();
+    }
+    return estimate;
+}
+
+Status MapColumnWriter::finish() {
+    if (is_nullable()) {
+        RETURN_IF_ERROR(_null_writer->finish());
+    }
+    RETURN_IF_ERROR(_key_writer->finish());
+    RETURN_IF_ERROR(_value_writer->finish());
+    return Status::OK();
+}
+
+// todo. make keys and values write
+Status MapColumnWriter::append_data(const uint8_t** ptr, size_t num_rows) {
+    size_t remaining = num_rows;
+    const auto* col_cursor = reinterpret_cast<const MapValue*>(*ptr);
+    while (remaining > 0) {
+        size_t num_written = 1;
+        auto* key_data_ptr = 
const_cast<MapValue*>(col_cursor)->mutable_key_data();
+       const uint8_t* key_ptr = (const uint8_t*)key_data_ptr;
+       RETURN_IF_ERROR(_key_writer->append_data(&key_ptr, 1));
+        auto* val_data_ptr = 
const_cast<MapValue*>(col_cursor)->mutable_value_data();
+       const uint8_t* val_ptr = (const uint8_t*)val_data_ptr;
+       RETURN_IF_ERROR(_value_writer->append_data(&val_ptr, 1));
+        remaining -= num_written;
+        col_cursor += num_written;
+        *ptr += num_written * sizeof(MapValue);
+    }
+
+    if (is_nullable()) {
+        return write_null_column(num_rows, false);
+    }
+    return Status::OK();
+}
+
+Status MapColumnWriter::write_data() {
+    if (is_nullable()) {
+        RETURN_IF_ERROR(_null_writer->write_data());
+    }
+    RETURN_IF_ERROR(_offset_writer->write_data());

Review Comment:
   warning: use of undeclared identifier '_offset_writer' 
[clang-diagnostic-error]
   ```cpp
       RETURN_IF_ERROR(_offset_writer->write_data());
                       ^
   ```
   



##########
be/src/udf/udf.h:
##########
@@ -927,6 +952,7 @@ using doris_udf::DateTimeVal;
 using doris_udf::HllVal;
 using doris_udf::FunctionContext;
 using doris_udf::CollectionVal;
+using doris_udf::MapVal;

Review Comment:
   warning: using decl 'MapVal' is unused [misc-unused-using-decls]
   ```cpp
   using doris_udf::MapVal;
                    ^
   ```
   **be/src/udf/udf.h:954:** remove the using
   ```cpp
   using doris_udf::MapVal;
                    ^
   ```
   



##########
be/src/olap/rowset/segment_v2/column_writer.cpp:
##########
@@ -695,5 +808,120 @@
     return Status::NotSupported("array writer has no data, can not 
finish_current_page");
 }
 
+/// ============================= MapColumnWriter =====================////
+MapColumnWriter::MapColumnWriter(const ColumnWriterOptions& opts, 
std::unique_ptr<Field> field,
+                                     ScalarColumnWriter* null_writer,
+                                     std::unique_ptr<ColumnWriter> key_writer,
+                                     std::unique_ptr<ColumnWriter> 
value_writer)
+        : ColumnWriter(std::move(field), opts.meta->is_nullable()),
+          _key_writer(std::move(key_writer)),
+          _value_writer(std::move(value_writer)),
+          _opts(opts) {
+    if (is_nullable()) {
+        _null_writer.reset(null_writer);
+    }
+}
+
+Status MapColumnWriter::init() {
+    if (is_nullable()) {
+        RETURN_IF_ERROR(_null_writer->init());
+    }
+    RETURN_IF_ERROR(_key_writer->init());
+    RETURN_IF_ERROR(_value_writer->init());
+    return Status::OK();
+}
+
+uint64_t MapColumnWriter::estimate_buffer_size() {
+    size_t estimate =  _key_writer->estimate_buffer_size() +
+                      _value_writer->estimate_buffer_size();
+    if (is_nullable()) {
+        estimate += _null_writer->estimate_buffer_size();
+    }
+    return estimate;
+}
+
+Status MapColumnWriter::finish() {
+    if (is_nullable()) {
+        RETURN_IF_ERROR(_null_writer->finish());
+    }
+    RETURN_IF_ERROR(_key_writer->finish());
+    RETURN_IF_ERROR(_value_writer->finish());
+    return Status::OK();
+}
+
+// todo. make keys and values write
+Status MapColumnWriter::append_data(const uint8_t** ptr, size_t num_rows) {
+    size_t remaining = num_rows;
+    const auto* col_cursor = reinterpret_cast<const MapValue*>(*ptr);
+    while (remaining > 0) {
+        size_t num_written = 1;
+        auto* key_data_ptr = 
const_cast<MapValue*>(col_cursor)->mutable_key_data();
+       const uint8_t* key_ptr = (const uint8_t*)key_data_ptr;
+       RETURN_IF_ERROR(_key_writer->append_data(&key_ptr, 1));
+        auto* val_data_ptr = 
const_cast<MapValue*>(col_cursor)->mutable_value_data();
+       const uint8_t* val_ptr = (const uint8_t*)val_data_ptr;
+       RETURN_IF_ERROR(_value_writer->append_data(&val_ptr, 1));
+        remaining -= num_written;
+        col_cursor += num_written;
+        *ptr += num_written * sizeof(MapValue);
+    }
+
+    if (is_nullable()) {
+        return write_null_column(num_rows, false);
+    }
+    return Status::OK();
+}
+
+Status MapColumnWriter::write_data() {
+    if (is_nullable()) {
+        RETURN_IF_ERROR(_null_writer->write_data());
+    }
+    RETURN_IF_ERROR(_offset_writer->write_data());
+    RETURN_IF_ERROR(_key_writer->write_data());
+    RETURN_IF_ERROR(_value_writer->write_data());
+    return Status::OK();
+}
+
+Status MapColumnWriter::write_ordinal_index() {
+    if (is_nullable()) {
+        RETURN_IF_ERROR(_null_writer->write_ordinal_index());
+    }
+    RETURN_IF_ERROR(_key_writer->write_ordinal_index());
+    RETURN_IF_ERROR(_value_writer->write_ordinal_index());
+   
+    return Status::OK();
+}
+
+Status MapColumnWriter::append_nulls(size_t num_rows) {
+    return write_null_column(num_rows, true);
+}
+
+Status MapColumnWriter::write_null_column(size_t num_rows, bool is_null) {

Review Comment:
   warning: out-of-line definition of 'write_null_column' does not match any 
declaration in 'doris::segment_v2::MapColumnWriter' [clang-diagnostic-error]
   ```cpp
   Status MapColumnWriter::write_null_column(size_t num_rows, bool is_null) {
                           ^
   ```
   



##########
be/src/vec/data_types/data_type_map.cpp:
##########
@@ -0,0 +1,157 @@
+// 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 "data_type_map.h"
+
+#include "gen_cpp/data.pb.h"
+#include "vec/data_types/data_type_factory.hpp"
+#include "vec/columns/column_array.h"
+#include "vec/columns/column_map.h"
+#include "vec/common/assert_cast.h"
+
+namespace doris::vectorized {
+
+DataTypeMap::DataTypeMap(const DataTypePtr& keys_, const DataTypePtr& values_)
+{
+    key_type = keys_;
+    value_type = values_;
+
+    keys = std::make_shared<DataTypeArray>(key_type);
+    values = std::make_shared<DataTypeArray>(value_type);
+}
+
+std::string DataTypeMap::to_string(const IColumn& column, size_t row_num) 
const {
+    const ColumnMap & map_column = assert_cast<const ColumnMap &>(column);
+    const ColumnArray::Offsets64& offsets = map_column.get_offsets();
+
+    size_t offset = offsets[row_num - 1];
+    size_t next_offset = offsets[row_num];
+
+    const IColumn & nested_keys = map_column.get_keys();
+    const IColumn & nested_values = map_column.get_values();
+
+    std::stringstream ss;
+    ss << "{";
+    for (size_t i = offset; i < next_offset; ++i)
+    {
+        if (i != offset)
+            ss << ", ";

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
           if (i != offset) {
               ss << ", ";
   }
   ```
   



##########
be/src/olap/rowset/segment_v2/column_writer.cpp:
##########
@@ -695,5 +808,120 @@
     return Status::NotSupported("array writer has no data, can not 
finish_current_page");
 }
 
+/// ============================= MapColumnWriter =====================////
+MapColumnWriter::MapColumnWriter(const ColumnWriterOptions& opts, 
std::unique_ptr<Field> field,
+                                     ScalarColumnWriter* null_writer,
+                                     std::unique_ptr<ColumnWriter> key_writer,
+                                     std::unique_ptr<ColumnWriter> 
value_writer)
+        : ColumnWriter(std::move(field), opts.meta->is_nullable()),
+          _key_writer(std::move(key_writer)),
+          _value_writer(std::move(value_writer)),
+          _opts(opts) {
+    if (is_nullable()) {
+        _null_writer.reset(null_writer);
+    }
+}
+
+Status MapColumnWriter::init() {
+    if (is_nullable()) {
+        RETURN_IF_ERROR(_null_writer->init());
+    }
+    RETURN_IF_ERROR(_key_writer->init());
+    RETURN_IF_ERROR(_value_writer->init());
+    return Status::OK();
+}
+
+uint64_t MapColumnWriter::estimate_buffer_size() {
+    size_t estimate =  _key_writer->estimate_buffer_size() +
+                      _value_writer->estimate_buffer_size();
+    if (is_nullable()) {
+        estimate += _null_writer->estimate_buffer_size();
+    }
+    return estimate;
+}
+
+Status MapColumnWriter::finish() {
+    if (is_nullable()) {
+        RETURN_IF_ERROR(_null_writer->finish());
+    }
+    RETURN_IF_ERROR(_key_writer->finish());
+    RETURN_IF_ERROR(_value_writer->finish());
+    return Status::OK();
+}
+
+// todo. make keys and values write
+Status MapColumnWriter::append_data(const uint8_t** ptr, size_t num_rows) {
+    size_t remaining = num_rows;
+    const auto* col_cursor = reinterpret_cast<const MapValue*>(*ptr);
+    while (remaining > 0) {
+        size_t num_written = 1;
+        auto* key_data_ptr = 
const_cast<MapValue*>(col_cursor)->mutable_key_data();
+       const uint8_t* key_ptr = (const uint8_t*)key_data_ptr;
+       RETURN_IF_ERROR(_key_writer->append_data(&key_ptr, 1));
+        auto* val_data_ptr = 
const_cast<MapValue*>(col_cursor)->mutable_value_data();
+       const uint8_t* val_ptr = (const uint8_t*)val_data_ptr;
+       RETURN_IF_ERROR(_value_writer->append_data(&val_ptr, 1));
+        remaining -= num_written;
+        col_cursor += num_written;
+        *ptr += num_written * sizeof(MapValue);
+    }
+
+    if (is_nullable()) {
+        return write_null_column(num_rows, false);
+    }
+    return Status::OK();
+}
+
+Status MapColumnWriter::write_data() {
+    if (is_nullable()) {
+        RETURN_IF_ERROR(_null_writer->write_data());
+    }
+    RETURN_IF_ERROR(_offset_writer->write_data());
+    RETURN_IF_ERROR(_key_writer->write_data());
+    RETURN_IF_ERROR(_value_writer->write_data());
+    return Status::OK();
+}
+
+Status MapColumnWriter::write_ordinal_index() {
+    if (is_nullable()) {
+        RETURN_IF_ERROR(_null_writer->write_ordinal_index());
+    }
+    RETURN_IF_ERROR(_key_writer->write_ordinal_index());
+    RETURN_IF_ERROR(_value_writer->write_ordinal_index());
+   
+    return Status::OK();
+}
+
+Status MapColumnWriter::append_nulls(size_t num_rows) {
+    return write_null_column(num_rows, true);

Review Comment:
   warning: use of undeclared identifier 'write_null_column' 
[clang-diagnostic-error]
   ```cpp
       return write_null_column(num_rows, true);
              ^
   ```
   



##########
be/src/vec/data_types/data_type_map.cpp:
##########
@@ -0,0 +1,157 @@
+// 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 "data_type_map.h"
+
+#include "gen_cpp/data.pb.h"
+#include "vec/data_types/data_type_factory.hpp"
+#include "vec/columns/column_array.h"
+#include "vec/columns/column_map.h"
+#include "vec/common/assert_cast.h"
+
+namespace doris::vectorized {
+
+DataTypeMap::DataTypeMap(const DataTypePtr& keys_, const DataTypePtr& values_)
+{
+    key_type = keys_;
+    value_type = values_;
+
+    keys = std::make_shared<DataTypeArray>(key_type);
+    values = std::make_shared<DataTypeArray>(value_type);
+}
+
+std::string DataTypeMap::to_string(const IColumn& column, size_t row_num) 
const {
+    const ColumnMap & map_column = assert_cast<const ColumnMap &>(column);
+    const ColumnArray::Offsets64& offsets = map_column.get_offsets();
+
+    size_t offset = offsets[row_num - 1];
+    size_t next_offset = offsets[row_num];
+
+    const IColumn & nested_keys = map_column.get_keys();
+    const IColumn & nested_values = map_column.get_values();
+
+    std::stringstream ss;
+    ss << "{";
+    for (size_t i = offset; i < next_offset; ++i)
+    {
+        if (i != offset)
+            ss << ", ";
+        ss << "'" << keys->to_string(nested_keys, i);
+        ss << ':';
+        ss << "'" << values->to_string(nested_values, i);
+    }
+    ss << "}";
+    return ss.str();
+}
+
+void DataTypeMap::to_string(const class doris::vectorized::IColumn& column, 
size_t row_num,
+                              class doris::vectorized::BufferWritable& ostr) 
const {
+    const ColumnMap & map_column = assert_cast<const ColumnMap &>(column);
+    const ColumnArray::Offsets64& offsets = map_column.get_offsets();
+
+    size_t offset = offsets[row_num - 1];
+    size_t next_offset = offsets[row_num];
+
+    const IColumn & nested_keys = map_column.get_keys();
+    const IColumn & nested_values = map_column.get_values();
+
+    ostr.write("{", 1);
+    for (size_t i = offset; i < next_offset; ++i)
+    {
+        if (i != offset)
+            ostr.write(", ", 2);

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
           if (i != offset) {
               ostr.write(", ", 2);
   }
   ```
   



##########
be/src/vec/data_types/data_type_map.cpp:
##########
@@ -0,0 +1,157 @@
+// 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 "data_type_map.h"
+
+#include "gen_cpp/data.pb.h"
+#include "vec/data_types/data_type_factory.hpp"
+#include "vec/columns/column_array.h"
+#include "vec/columns/column_map.h"
+#include "vec/common/assert_cast.h"
+
+namespace doris::vectorized {
+
+DataTypeMap::DataTypeMap(const DataTypePtr& keys_, const DataTypePtr& values_)
+{
+    key_type = keys_;
+    value_type = values_;
+
+    keys = std::make_shared<DataTypeArray>(key_type);
+    values = std::make_shared<DataTypeArray>(value_type);
+}
+
+std::string DataTypeMap::to_string(const IColumn& column, size_t row_num) 
const {
+    const ColumnMap & map_column = assert_cast<const ColumnMap &>(column);
+    const ColumnArray::Offsets64& offsets = map_column.get_offsets();
+
+    size_t offset = offsets[row_num - 1];
+    size_t next_offset = offsets[row_num];
+
+    const IColumn & nested_keys = map_column.get_keys();
+    const IColumn & nested_values = map_column.get_values();
+
+    std::stringstream ss;
+    ss << "{";
+    for (size_t i = offset; i < next_offset; ++i)
+    {
+        if (i != offset)
+            ss << ", ";
+        ss << "'" << keys->to_string(nested_keys, i);
+        ss << ':';
+        ss << "'" << values->to_string(nested_values, i);
+    }
+    ss << "}";
+    return ss.str();
+}
+
+void DataTypeMap::to_string(const class doris::vectorized::IColumn& column, 
size_t row_num,
+                              class doris::vectorized::BufferWritable& ostr) 
const {
+    const ColumnMap & map_column = assert_cast<const ColumnMap &>(column);
+    const ColumnArray::Offsets64& offsets = map_column.get_offsets();
+
+    size_t offset = offsets[row_num - 1];
+    size_t next_offset = offsets[row_num];
+
+    const IColumn & nested_keys = map_column.get_keys();
+    const IColumn & nested_values = map_column.get_values();
+
+    ostr.write("{", 1);
+    for (size_t i = offset; i < next_offset; ++i)
+    {
+        if (i != offset)
+            ostr.write(", ", 2);
+        keys->to_string(nested_keys, i, ostr);
+        ostr.write(":", 1);
+        values->to_string(nested_values, i, ostr);
+    }
+    ostr.write("}", 1);
+}
+
+Status DataTypeMap::from_string(ReadBuffer& rb, IColumn* column) const {
+    DCHECK(!rb.eof());
+    // only support one level now
+    auto* map_column = assert_cast<ColumnMap*>(column);
+    //    IColumn& nested_column = array_column->get_data();
+    if (*rb.position() != '{') {
+        return Status::InvalidArgument("map does not start with '{' character, 
found '{}'",
+                                       *rb.position());
+    }
+    keys->from_string(rb, &map_column->get_keys());
+    values->from_string(rb, &map_column->get_values());
+    if (*(rb.end() - 1) != '}') {
+        return Status::InvalidArgument("map does not end with '}' character, 
found '{}'",
+                                       *(rb.end() - 1));
+    }
+//    keys->deserializeAsTextQuoted(extractElementColumn(column, 0), istr, 
settings);
+//    assertChar(',', istr);
+//    values->deserializeAsTextQuoted(extractElementColumn(column, 1), istr, 
settings);
+//    assertChar('}', istr);
+    return Status::OK();
+}
+
+MutableColumnPtr DataTypeMap::create_column() const {
+    return ColumnMap::create(keys->create_column(), values->create_column());
+}
+
+void DataTypeMap::to_pb_column_meta(PColumnMeta* col_meta) const {
+    IDataType::to_pb_column_meta(col_meta);
+    auto key_children = col_meta->add_children();
+    auto value_children = col_meta->add_children();
+    keys->to_pb_column_meta(key_children);
+    values->to_pb_column_meta(value_children);
+}
+
+bool DataTypeMap::equals(const IDataType& rhs) const {
+    if (typeid(rhs) != typeid(*this))
+        return false;

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
       if (typeid(rhs) != typeid(*this)) {
           return false;
   }
   ```
   



##########
be/src/runtime/map_value.h:
##########
@@ -0,0 +1,73 @@
+// 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 <type_traits>
+#include "runtime/primitive_type.h"
+
+namespace doris_udf {
+class FunctionContext;
+struct AnyVal;
+} // namespace doris_udf
+
+namespace doris {
+
+using doris_udf::FunctionContext;

Review Comment:
   warning: using decl 'FunctionContext' is unused [misc-unused-using-decls]
   ```cpp
   using doris_udf::FunctionContext;
                    ^
   ```
   **be/src/runtime/map_value.h:29:** remove the using
   ```cpp
   using doris_udf::FunctionContext;
                    ^
   ```
   



##########
be/src/vec/exprs/vmap_literal.cpp:
##########
@@ -0,0 +1,51 @@
+// 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 "vec/exprs/vmap_literal.h"
+
+//insert into table_map values ({'name':'zhangsan', 'gender':'male'}), 
({'name':'lisi', 'gender':'female'});
+namespace doris::vectorized {
+
+Status VMapLiteral::prepare(RuntimeState* state, const RowDescriptor& row_desc,
+                              VExprContext* context) {
+    DCHECK_EQ(type().children.size(), 2) << "map children type not 2";
+
+    RETURN_IF_ERROR_OR_PREPARED(VExpr::prepare(state, row_desc, context));
+    // map-field should contain two vector field for keys and values
+    Field map = Map();
+    Field keys = Array();
+    Field values = Array();
+    // each child is slot with key1, value1, key2, value2...
+    for (int idx = 0; idx < _children.size(); ++idx ) {
+        Field item;
+        ColumnPtrWrapper* const_col_wrapper = nullptr;
+        RETURN_IF_ERROR(_children[idx]->get_const_col(context, 
&const_col_wrapper));
+        const_col_wrapper->column_ptr->get(0, item);
+
+        if ((idx & 1) == 0)
+            keys.get<Array>().push_back(item);
+        else
+            values.get<Array>().push_back(item);

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
           else {
               values.get<Array>().push_back(item);
   }
   ```
   



##########
be/src/runtime/map_value.h:
##########
@@ -0,0 +1,73 @@
+// 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 <type_traits>
+#include "runtime/primitive_type.h"
+
+namespace doris_udf {
+class FunctionContext;
+struct AnyVal;
+} // namespace doris_udf
+
+namespace doris {
+
+using doris_udf::FunctionContext;
+using doris_udf::AnyVal;

Review Comment:
   warning: using decl 'AnyVal' is unused [misc-unused-using-decls]
   ```cpp
   using doris_udf::AnyVal;
                    ^
   ```
   **be/src/runtime/map_value.h:30:** remove the using
   ```cpp
   using doris_udf::AnyVal;
                    ^
   ```
   



##########
be/src/vec/data_types/data_type_map.cpp:
##########
@@ -0,0 +1,157 @@
+// 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 "data_type_map.h"
+
+#include "gen_cpp/data.pb.h"
+#include "vec/data_types/data_type_factory.hpp"
+#include "vec/columns/column_array.h"
+#include "vec/columns/column_map.h"
+#include "vec/common/assert_cast.h"
+
+namespace doris::vectorized {
+
+DataTypeMap::DataTypeMap(const DataTypePtr& keys_, const DataTypePtr& values_)
+{
+    key_type = keys_;
+    value_type = values_;
+
+    keys = std::make_shared<DataTypeArray>(key_type);
+    values = std::make_shared<DataTypeArray>(value_type);
+}
+
+std::string DataTypeMap::to_string(const IColumn& column, size_t row_num) 
const {
+    const ColumnMap & map_column = assert_cast<const ColumnMap &>(column);
+    const ColumnArray::Offsets64& offsets = map_column.get_offsets();
+
+    size_t offset = offsets[row_num - 1];
+    size_t next_offset = offsets[row_num];
+
+    const IColumn & nested_keys = map_column.get_keys();
+    const IColumn & nested_values = map_column.get_values();
+
+    std::stringstream ss;
+    ss << "{";
+    for (size_t i = offset; i < next_offset; ++i)
+    {
+        if (i != offset)
+            ss << ", ";
+        ss << "'" << keys->to_string(nested_keys, i);
+        ss << ':';
+        ss << "'" << values->to_string(nested_values, i);
+    }
+    ss << "}";
+    return ss.str();
+}
+
+void DataTypeMap::to_string(const class doris::vectorized::IColumn& column, 
size_t row_num,
+                              class doris::vectorized::BufferWritable& ostr) 
const {
+    const ColumnMap & map_column = assert_cast<const ColumnMap &>(column);
+    const ColumnArray::Offsets64& offsets = map_column.get_offsets();
+
+    size_t offset = offsets[row_num - 1];
+    size_t next_offset = offsets[row_num];
+
+    const IColumn & nested_keys = map_column.get_keys();
+    const IColumn & nested_values = map_column.get_values();
+
+    ostr.write("{", 1);
+    for (size_t i = offset; i < next_offset; ++i)
+    {
+        if (i != offset)
+            ostr.write(", ", 2);
+        keys->to_string(nested_keys, i, ostr);
+        ostr.write(":", 1);
+        values->to_string(nested_values, i, ostr);
+    }
+    ostr.write("}", 1);
+}
+
+Status DataTypeMap::from_string(ReadBuffer& rb, IColumn* column) const {
+    DCHECK(!rb.eof());
+    // only support one level now
+    auto* map_column = assert_cast<ColumnMap*>(column);
+    //    IColumn& nested_column = array_column->get_data();
+    if (*rb.position() != '{') {
+        return Status::InvalidArgument("map does not start with '{' character, 
found '{}'",
+                                       *rb.position());
+    }
+    keys->from_string(rb, &map_column->get_keys());
+    values->from_string(rb, &map_column->get_values());
+    if (*(rb.end() - 1) != '}') {
+        return Status::InvalidArgument("map does not end with '}' character, 
found '{}'",
+                                       *(rb.end() - 1));
+    }
+//    keys->deserializeAsTextQuoted(extractElementColumn(column, 0), istr, 
settings);
+//    assertChar(',', istr);
+//    values->deserializeAsTextQuoted(extractElementColumn(column, 1), istr, 
settings);
+//    assertChar('}', istr);
+    return Status::OK();
+}
+
+MutableColumnPtr DataTypeMap::create_column() const {
+    return ColumnMap::create(keys->create_column(), values->create_column());
+}
+
+void DataTypeMap::to_pb_column_meta(PColumnMeta* col_meta) const {
+    IDataType::to_pb_column_meta(col_meta);
+    auto key_children = col_meta->add_children();
+    auto value_children = col_meta->add_children();
+    keys->to_pb_column_meta(key_children);
+    values->to_pb_column_meta(value_children);
+}
+
+bool DataTypeMap::equals(const IDataType& rhs) const {
+    if (typeid(rhs) != typeid(*this))
+        return false;
+
+    const DataTypeMap & rhs_map = static_cast<const DataTypeMap &>(rhs);
+
+    if (!keys->equals(*rhs_map.keys))
+        return false;
+
+    if (!values->equals(*rhs_map.values))
+        return false;

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
       if (!values->equals(*rhs_map.values)) {
           return false;
   }
   ```
   



##########
be/src/vec/exprs/vmap_literal.cpp:
##########
@@ -0,0 +1,51 @@
+// 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 "vec/exprs/vmap_literal.h"
+
+//insert into table_map values ({'name':'zhangsan', 'gender':'male'}), 
({'name':'lisi', 'gender':'female'});
+namespace doris::vectorized {
+
+Status VMapLiteral::prepare(RuntimeState* state, const RowDescriptor& row_desc,
+                              VExprContext* context) {
+    DCHECK_EQ(type().children.size(), 2) << "map children type not 2";
+
+    RETURN_IF_ERROR_OR_PREPARED(VExpr::prepare(state, row_desc, context));
+    // map-field should contain two vector field for keys and values
+    Field map = Map();
+    Field keys = Array();
+    Field values = Array();
+    // each child is slot with key1, value1, key2, value2...
+    for (int idx = 0; idx < _children.size(); ++idx ) {
+        Field item;
+        ColumnPtrWrapper* const_col_wrapper = nullptr;
+        RETURN_IF_ERROR(_children[idx]->get_const_col(context, 
&const_col_wrapper));
+        const_col_wrapper->column_ptr->get(0, item);
+
+        if ((idx & 1) == 0)

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
           if ((idx & 1) == 0) {
   ```
   
   be/src/vec/exprs/vmap_literal.cpp:40:
   ```diff
   -         else
   +         } else
   ```
   



##########
be/src/vec/data_types/data_type_map.cpp:
##########
@@ -0,0 +1,157 @@
+// 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 "data_type_map.h"
+
+#include "gen_cpp/data.pb.h"
+#include "vec/data_types/data_type_factory.hpp"
+#include "vec/columns/column_array.h"
+#include "vec/columns/column_map.h"
+#include "vec/common/assert_cast.h"
+
+namespace doris::vectorized {
+
+DataTypeMap::DataTypeMap(const DataTypePtr& keys_, const DataTypePtr& values_)
+{
+    key_type = keys_;
+    value_type = values_;
+
+    keys = std::make_shared<DataTypeArray>(key_type);
+    values = std::make_shared<DataTypeArray>(value_type);
+}
+
+std::string DataTypeMap::to_string(const IColumn& column, size_t row_num) 
const {
+    const ColumnMap & map_column = assert_cast<const ColumnMap &>(column);
+    const ColumnArray::Offsets64& offsets = map_column.get_offsets();
+
+    size_t offset = offsets[row_num - 1];
+    size_t next_offset = offsets[row_num];
+
+    const IColumn & nested_keys = map_column.get_keys();
+    const IColumn & nested_values = map_column.get_values();
+
+    std::stringstream ss;
+    ss << "{";
+    for (size_t i = offset; i < next_offset; ++i)
+    {
+        if (i != offset)
+            ss << ", ";
+        ss << "'" << keys->to_string(nested_keys, i);
+        ss << ':';
+        ss << "'" << values->to_string(nested_values, i);
+    }
+    ss << "}";
+    return ss.str();
+}
+
+void DataTypeMap::to_string(const class doris::vectorized::IColumn& column, 
size_t row_num,
+                              class doris::vectorized::BufferWritable& ostr) 
const {
+    const ColumnMap & map_column = assert_cast<const ColumnMap &>(column);
+    const ColumnArray::Offsets64& offsets = map_column.get_offsets();
+
+    size_t offset = offsets[row_num - 1];
+    size_t next_offset = offsets[row_num];
+
+    const IColumn & nested_keys = map_column.get_keys();
+    const IColumn & nested_values = map_column.get_values();
+
+    ostr.write("{", 1);
+    for (size_t i = offset; i < next_offset; ++i)
+    {
+        if (i != offset)
+            ostr.write(", ", 2);
+        keys->to_string(nested_keys, i, ostr);
+        ostr.write(":", 1);
+        values->to_string(nested_values, i, ostr);
+    }
+    ostr.write("}", 1);
+}
+
+Status DataTypeMap::from_string(ReadBuffer& rb, IColumn* column) const {
+    DCHECK(!rb.eof());
+    // only support one level now
+    auto* map_column = assert_cast<ColumnMap*>(column);
+    //    IColumn& nested_column = array_column->get_data();
+    if (*rb.position() != '{') {
+        return Status::InvalidArgument("map does not start with '{' character, 
found '{}'",
+                                       *rb.position());
+    }
+    keys->from_string(rb, &map_column->get_keys());
+    values->from_string(rb, &map_column->get_values());
+    if (*(rb.end() - 1) != '}') {
+        return Status::InvalidArgument("map does not end with '}' character, 
found '{}'",
+                                       *(rb.end() - 1));
+    }
+//    keys->deserializeAsTextQuoted(extractElementColumn(column, 0), istr, 
settings);
+//    assertChar(',', istr);
+//    values->deserializeAsTextQuoted(extractElementColumn(column, 1), istr, 
settings);
+//    assertChar('}', istr);
+    return Status::OK();
+}
+
+MutableColumnPtr DataTypeMap::create_column() const {
+    return ColumnMap::create(keys->create_column(), values->create_column());
+}
+
+void DataTypeMap::to_pb_column_meta(PColumnMeta* col_meta) const {
+    IDataType::to_pb_column_meta(col_meta);
+    auto key_children = col_meta->add_children();
+    auto value_children = col_meta->add_children();
+    keys->to_pb_column_meta(key_children);
+    values->to_pb_column_meta(value_children);
+}
+
+bool DataTypeMap::equals(const IDataType& rhs) const {
+    if (typeid(rhs) != typeid(*this))
+        return false;
+
+    const DataTypeMap & rhs_map = static_cast<const DataTypeMap &>(rhs);
+
+    if (!keys->equals(*rhs_map.keys))
+        return false;

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
       if (!keys->equals(*rhs_map.keys)) {
           return false;
   }
   ```
   



##########
be/src/vec/functions/array/function_array_element.h:
##########
@@ -81,6 +94,79 @@ class FunctionArrayElement : public IFunction {
     }
 
 private:
+    //=========================== map element===========================//
+    ColumnPtr _get_mapped_idx(const ColumnArray& key_column,
+                           const ColumnWithTypeAndName& argument) {
+        if (key_column.get_data().is_column_string()) {
+            return _mapped_key_string(key_column, argument);
+        }
+        return nullptr;
+    }
+
+    ColumnPtr _get_mapped_value(const ColumnArray& val_column,
+                                const IColumn& matched_indices,
+                                const UInt8* src_null_map,
+                                UInt8* dst_null_map) {
+        const UInt8* nested_null_map = nullptr;
+        ColumnPtr nested_column = nullptr;
+        if (is_column_nullable(val_column.get_data())) {
+            const auto& nested_null_column =
+                    reinterpret_cast<const 
ColumnNullable&>(val_column.get_data());
+            nested_null_map = 
nested_null_column.get_null_map_column().get_data().data();
+            nested_column = nested_null_column.get_nested_column_ptr();
+        } else {
+            nested_column = val_column.get_data_ptr();
+        }
+        if (check_column<ColumnInt8>(nested_column)) {
+            return _execute_number<ColumnInt8>(val_column.get_offsets(), 
*nested_column,
+                                                src_null_map, matched_indices,
+                                                nested_null_map, dst_null_map);
+        } else if (check_column<ColumnInt32>(nested_column)) {
+            _execute_number<ColumnInt32>(val_column.get_offsets(), 
*nested_column,
+                                        src_null_map, matched_indices,
+                                        nested_null_map, dst_null_map);
+        }
+        return nullptr;
+    }
+
+    ColumnPtr _mapped_key_string(const ColumnArray& column,
+                            const ColumnWithTypeAndName& argument) {
+        auto right_column = argument.column->convert_to_full_column_if_const();
+        const ColumnString& match_key = reinterpret_cast<const 
ColumnString&>(*right_column);
+        const ColumnArray::Offsets64& offsets = column.get_offsets();
+        ColumnPtr nested_ptr = nullptr;
+        if (is_column_nullable(column.get_data())) {
+            nested_ptr = reinterpret_cast<const 
ColumnNullable&>(column.get_data()).get_nested_column_ptr();
+        } else {
+            nested_ptr = column.get_data_ptr();
+        }
+        const ColumnString& nested_key = reinterpret_cast<const 
ColumnString&>(*nested_ptr);
+        size_t rows = offsets.size();
+        // prepare return data
+        auto matched_indices = ColumnVector<Int8>::create();
+        matched_indices->reserve(rows);
+
+        for (size_t i = 0; i < rows; i++)
+        {
+            bool matched = false;
+            size_t begin = offsets[i - 1];
+            size_t end = offsets[i];
+            for (size_t j = begin; j < end; j++) {
+                if (nested_key.get_data_at(j) == match_key.get_data_at(i)) {
+                    matched_indices->insert_value(j-begin+1);
+                    matched = true;
+                    break;
+                }
+            }
+
+            if (!matched)
+                matched_indices->insert_value(end-begin+1); // make indices 
for null

Review Comment:
   warning: statement should be inside braces 
[readability-braces-around-statements]
   
   ```suggestion
               if (!matched) {
                   matched_indices->insert_value(end-begin+1); // make indices 
for 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.

To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org

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


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


Reply via email to