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/02/24 08:11:03 UTC

[GitHub] [incubator-doris] Seaven opened a new pull request #2979: Incomplete support Array type (#2871)

Seaven opened a new pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979
 
 
   The sub-element type only support INT/VARCHAR.
   
   Main include:
   
   * FE SQL syntax (Create table) support array type column, like 'name Array<int> not null'
   * FE SQL syntax support ArrayLiteral, like 'select array(1, 2, 3, 4)'
   * FE update meta: Columns meta support array type, include meta serialized and deserialized, and add flags in FEMetaVersions for compatibility
   * FE support ArrayLiteral: Mainly support array related function, and the array constucted function support sub-element type auto-cast.
   
   * BE update result buffer, support dynamic mode for output variable length data type.
   * BE support ArrayLiteral and array(*) functions.
   
   And other todo:
   * BE meta udpate
   * BE transfer data from storage to tuple
   * BE support array type related function(hash, equal.....)
   * BE support array type data import
   * other....(many related code, such as print value, value 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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386434207
 
 

 ##########
 File path: fe/src/test/java/org/apache/doris/catalog/ColumnTest.java
 ##########
 @@ -44,6 +45,7 @@ public void setUp() {
 
         FakeCatalog.setCatalog(catalog);
         FakeCatalog.setMetaVersion(FeConstants.meta_version);
+        FakeCatalog.setMetaVersion(FeMetaVersion.VERSION_74);
 
 Review comment:
   no need, but you need to set VERSION_CURRENT

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386763011
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/common/FeMetaVersion.java
 ##########
 @@ -157,6 +157,8 @@
     public static final int VERSION_72 = 72;
     // broker persist isAlive
     public static final int VERSION_73 = 73;
+    // array type
+    public static final int VERSION_74 = 74;
 
 Review comment:
   But array type support is incomplete(lack of storage), change the VERSION_CURRENT will take some problems, special for meta compatibility

----------------------------------------------------------------
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 #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386994292
 
 

 ##########
 File path: be/src/util/mysql_row_buffer.cpp
 ##########
 @@ -261,21 +321,28 @@ int MysqlRowBuffer::push_string(const char* str, int length) {
         return ret;
     }
 
-    _pos =  pack_vlen(_pos, length);
+    if (NOT_DYNAMIC_MODE) {
+        _pos =  pack_vlen(_pos, length);
+    }
     memcpy(_pos, str, length);
     _pos += length;
     return 0;
 }
 
 int MysqlRowBuffer::push_null() {
+    if (!NOT_DYNAMIC_MODE) {
+        // dynamic mode not write 
+        return 0;
+    }
+        
     int ret = reserve(1);
 
     if (0 != ret) {
         LOG(ERROR) << "mysql row buffer reserver failed.";
         return ret;
     }
 
-    int1store(_pos, 251);
+    int1store(_pos, 251);    
 
 Review comment:
   ```suggestion
       int1store(_pos, 251);
   ```

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386441583
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/analysis/ArrayLiteral.java
 ##########
 @@ -0,0 +1,114 @@
+// 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.
+
+package org.apache.doris.analysis;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.doris.catalog.ArrayType;
+import org.apache.doris.catalog.Type;
+import org.apache.doris.thrift.TExpr;
+import org.apache.doris.thrift.TExprNode;
+import org.apache.doris.thrift.TExprNodeType;
+
+public class ArrayLiteral extends LiteralExpr {
+
+    public ArrayLiteral() {
+        this.type = new ArrayType(Type.NULL);
+        children = new ArrayList<>();
+    }
+
+    public ArrayLiteral(LiteralExpr... v) {
+        if (v.length < 1) {
+            this.type = new ArrayType(Type.NULL);
+            return;
+        }
+
+        this.type = new ArrayType(v[0].type);
+        children = new ArrayList<>(v.length);
+        children.addAll(Arrays.asList(v));
+    }
+
+    protected ArrayLiteral(ArrayLiteral other) {
+        super(other);
+        this.type = other.type;
+        this.children.addAll(other.children);
+    }
+
+    @Override
+    public boolean isMinValue() {
+        return false;
+    }
+
+    @Override
+    public int compareLiteral(LiteralExpr expr) {
+        return 0;
+    }
+
+    @Override
+    protected String toSqlImpl() {
+        List<String> list = new ArrayList<>(children.size());
+        children.forEach(v -> list.add(v.toSqlImpl()));
+
+        return "ARRAY(" + StringUtils.join(list, ", ") + ")";
+    }
+
+    @Override
+    public String getStringValue() {
+        List<String> list = new ArrayList<>(children.size());
+        children.forEach(v -> list.add(((LiteralExpr) v).getStringValue()));
+
+        return "ARRAY[" + StringUtils.join(list, ", ") + "]";
+    }
+
+    @Override
+    protected void treeToThriftHelper(TExpr container) {
+        super.treeToThriftHelper(container);
+
+//        for (LiteralExpr expr: values) {
+//            expr.treeToThriftHelper(container);
+//        }
+    }
+
+    @Override
+    protected void toThrift(TExprNode msg) {
+        msg.node_type = TExprNodeType.ARRAY_LITERAL;
+//        msg.num_children = values.size();
+        msg.setChild_type(((ArrayType) type).getItemType().getPrimitiveType().toThrift());
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        super.write(out);
 
 Review comment:
   Not implemented?

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386773407
 
 

 ##########
 File path: be/src/runtime/raw_value.cpp
 ##########
 @@ -300,7 +314,34 @@ void RawValue::write(const void* value, void* dst, const TypeDescriptor& type, M
         } else {
             dest->ptr = src->ptr;
         }
+        break;
+    }
+    case TYPE_ARRAY: {
+        DCHECK_EQ(type.children.size(), 1);
+        
+        const CollectionValue* src = reinterpret_cast<const CollectionValue*>(value);
+        CollectionValue* val = reinterpret_cast<CollectionValue*>(dst);
 
+        if (pool != NULL) {
+            auto children_type = type.children.at(0).type;
+            CollectionValue::init_collection(pool, src->size(), children_type, val);
+            CollectionIterator src_iter = src->iterator(children_type);
+            CollectionIterator val_iter = val->iterator(children_type);
+
+            val->copy_null_signs(src);
+            
+            while (src_iter.has_next() && val_iter.has_next()) {
+                if (!src_iter.is_null()) {
+                    // write children 
+                    write(src_iter.value(), val_iter.value(), children_type, pool);    
+                }
 
 Review comment:
   Do we need to set the nullbits in `val_iter`?

----------------------------------------------------------------
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 #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386995575
 
 

 ##########
 File path: be/src/util/mysql_row_buffer.h
 ##########
 @@ -57,13 +57,44 @@ class MysqlRowBuffer {
     int length() const {
         return _pos - _buf;
     }
+
+    /**
+     * Why?
+     * Because the Nested-Type's data need pushed multiple times, but mysql protocal don't support nested type and each 
+     * push will decide a column data.  
+     * 
+     * How?
+     * Dynamic mode allow we can push data in a column multiple times, and allow recursive push. 
+     * NOTE:Need to ensure the open() and close() appear in pairs
+     * 
+     * the code:
+     *     mrb.push_smallint(120);
+     *     mrb.push_int(-30000);
+     *     mrb.push_bigint(900000);
+     * 
+     * In normal mode, the buffer contains three column:
+     *  1-'5'-3-'120'-6-'-30000' 
 
 Review comment:
   Any documents for this example?
   I do not quite understand.

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386759459
 
 

 ##########
 File path: be/src/util/mysql_row_buffer.cpp
 ##########
 @@ -109,15 +132,24 @@ int MysqlRowBuffer::push_tinyint(int8_t data) {
         return ret;
     }
 
-    int length = snprintf(_pos + 1, MAX_TINYINT_WIDTH + 2, "%d", data);
+    int size_bytes = 0;
+    if (NOT_DYNAMIC_MODE) {
 
 Review comment:
   In most case, NOT_DYNAMIC_MODE is true, so here we have to set size_bytes twice.
   How about change it to
   ```
   int size_bytes = 1;
   if (DYNAMIC_MODE) {
       size_bytes = 0;
   }
   ```

----------------------------------------------------------------
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] imay commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386078219
 
 

 ##########
 File path: be/src/util/mysql_row_buffer.cpp
 ##########
 @@ -26,6 +26,8 @@
 
 namespace doris {
 
+#define NOT_DYNAMIC_MODE  (_dynamic_mode == 0)           
 
 Review comment:
   Can you give more comments for this option to help others?

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r387476798
 
 

 ##########
 File path: be/src/util/mysql_row_buffer.h
 ##########
 @@ -57,13 +57,44 @@ class MysqlRowBuffer {
     int length() const {
         return _pos - _buf;
     }
+
+    /**
+     * Why?
+     * Because the Nested-Type's data need pushed multiple times, but mysql protocal don't support nested type and each 
+     * push will decide a column data.  
+     * 
+     * How?
+     * Dynamic mode allow we can push data in a column multiple times, and allow recursive push. 
+     * NOTE:Need to ensure the open() and close() appear in pairs
+     * 
+     * the code:
+     *     mrb.push_smallint(120);
+     *     mrb.push_int(-30000);
+     *     mrb.push_bigint(900000);
+     * 
+     * In normal mode, the buffer contains three column:
+     *  1-'5'-3-'120'-6-'-30000' 
 
 Review comment:
   Em......Mysql protocol like this:
   
   ```
   | flag | (length) | value | flag | (length) | value | ......
   ^-------A column-------^
   
   The flag means value's length or null value:
   If value is NULL, flag is 251
   If value's length < 250, flag is the value's length
   If 250 < value's length < 65536, flag is 252 and the next two byte is length
   If 65536 < value's length < 16777216 , flag is 253 and the next three byte is length
   If 16777216 < value's length, flag is 254 and the next eighth byte is length
   ```
   
   I will update the 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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r387468284
 
 

 ##########
 File path: be/src/exprs/anyval_util.cpp
 ##########
 @@ -164,10 +166,64 @@ FunctionContext::TypeDesc AnyValUtil::column_type_to_type_desc(const TypeDescrip
     case TYPE_NULL:
         out.type = FunctionContext::TYPE_NULL;
         break;
+    case TYPE_ARRAY: {
+        out.type = FunctionContext::TYPE_ARRAY;
+        for (auto&& t : type.children) {
 
 Review comment:
   Em,the better way is `const auto &`, will modify it.

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386766810
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/Type.java
 ##########
 @@ -43,11 +43,8 @@
 public abstract class Type {
     private static final Logger LOG = LogManager.getLogger(Type.class);
 
-    // Maximum nesting depth of a type. This limit was determined experimentally by
-    // generating and scanning deeply nested Parquet and Avro files. In those experiments,
-    // we exceeded the stack space in the scanner (which uses recursion for dealing with
-    // nested types) at a nesting depth between 200 and 300 (200 worked, 300 crashed).
-    public static int MAX_NESTING_DEPTH = 100;
+    // Maximum nesting depth of a type.
 
 Review comment:
   Just limit the depth of nested types, deeper nested type require a lot of extra development, and I don't think we need to support more than 2 depths, like array<map<array&lt;int&gt;>>......

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386224277
 
 

 ##########
 File path: be/src/util/mysql_row_buffer.cpp
 ##########
 @@ -26,6 +26,8 @@
 
 namespace doris {
 
+#define NOT_DYNAMIC_MODE  (_dynamic_mode == 0)           
 
 Review comment:
   OK. Will add comments in .h

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386450281
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/analysis/CreateTableStmt.java
 ##########
 @@ -350,6 +353,17 @@ public void analyze(Analyzer analyzer) throws UserException {
                 }
             }
 
+            if (columnDef.getType().isArrayType()) {
+                ArrayType tp =  (ArrayType) columnDef.getType();
+                if (!PrimitiveType.INT.equals(tp.getItemType().getPrimitiveType()) &&
 
 Review comment:
   PrimitiveType is enum class, so you can just using `==`
   like:
   ```
   tp.getItemType().getPrimitiveType() == PrimitiveType.INT
   ```

----------------------------------------------------------------
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] imay commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386077242
 
 

 ##########
 File path: be/src/exprs/collection_functions.cpp
 ##########
 @@ -0,0 +1,43 @@
+// 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 "exprs/collection_functions.h"
+#include "runtime/collection_value.h"
+
+#include "common/logging.h"
+
+namespace doris {
+
+void CollectionFunctions::init() { }
+
+#define ARRAY_FUNCTION(TYPE, PRIMARY_TYPE)    \
 
 Review comment:
   no need to align  \.

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386784395
 
 

 ##########
 File path: be/src/runtime/collection_value.cpp
 ##########
 @@ -0,0 +1,204 @@
+// 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 "runtime/collection_value.h"
+
+#include "util/bitmap.h"
+
+#include "common/logging.h"
+#include "exprs/anyval_util.h"
+
+
+namespace doris {
+
+int sizeof_type(const PrimitiveType& type) {
+    switch (type) {
+        case TYPE_INT:
 
 Review comment:
   Just support INT/String in current version, but FE will auto cast to int if sub-element type is   tiny/small,  maybe will support other types in future.

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386772649
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/PrimitiveType.java
 ##########
 @@ -56,6 +56,7 @@
     TIME("TIME", 8, TPrimitiveType.TIME),
     // we use OBJECT type represent BITMAP type in Backend
     BITMAP("BITMAP", 16, TPrimitiveType.OBJECT),
+    ARRAY("ARRAY", 16, TPrimitiveType.ARRAY),
 
 Review comment:
   The slot_size is used for Tuple to compute memory layout, why need change to 17?
   
   

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386768258
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/rewrite/FEFunctions.java
 ##########
 @@ -421,4 +422,21 @@ public static LiteralExpr ifNullBigInt(LiteralExpr first, LiteralExpr second) th
     public static LiteralExpr ifNullDateTime(LiteralExpr first, LiteralExpr second) throws AnalysisException {
         return first instanceof NullLiteral ? second : first;
     }
+
+    /**
+     ------------------------------------------------------------------------------
+     */
+
+    /**
+     * Nest type function
+     */
+    @FEFunction(name = "array", argTypes = {"INT"}, returnType = "ARRAY")
+    public static LiteralExpr arrayInt(LiteralExpr... exprs) throws AnalysisException {
+        return new ArrayLiteral(exprs);
+    }
+
+    @FEFunction(name = "array", argTypes = {"VARCHAR"}, returnType = "ARRAY")
+    public static LiteralExpr array(StringLiteral... exprs) throws AnalysisException {
 
 Review comment:
   Em...the method is same with array(IntLiteral.....) in fact, maybe will same with array(BigintLiteral.....)、array(BoolLiteral.....) in future......I think better way is refactor the annotation for support a list of types

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386466068
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/Column.java
 ##########
 @@ -243,9 +272,36 @@ public TColumn toThrift() {
         tColumn.setIs_key(this.isKey);
         tColumn.setIs_allow_null(this.isAllowNull);
         tColumn.setDefault_value(this.defaultValue);
+
+        toChildrenThrift(this, tColumn);
+
         return tColumn;
     }
 
+    private void toChildrenThrift(Column column, TColumn tColumn) {
+        if (column.type.isArrayType()) {
+            Column children = column.getChildren().get(0);
 
 Review comment:
   Why not calling `children.toThrift()`?

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386467420
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/PrimitiveType.java
 ##########
 @@ -56,6 +56,7 @@
     TIME("TIME", 8, TPrimitiveType.TIME),
     // we use OBJECT type represent BITMAP type in Backend
     BITMAP("BITMAP", 16, TPrimitiveType.OBJECT),
+    ARRAY("ARRAY", 16, TPrimitiveType.ARRAY),
 
 Review comment:
   ```suggestion
       ARRAY("ARRAY", 17, TPrimitiveType.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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386765053
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/ScalarType.java
 ##########
 @@ -141,6 +141,9 @@ public static ScalarType createType(PrimitiveType type) {
                 return DEFAULT_DECIMALV2;
             case LARGEINT:
                 return LARGEINT;
+            case ARRAY:
+                // can't static, because the same array may cause circular dependency
 
 Review comment:
   Nested type need decide  and set sub-element type in runtime, return a static object will take repeatedly set the sub-element type of same object in many places......

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386436319
 
 

 ##########
 File path: fe/src/main/cup/sql_parser.cup
 ##########
 @@ -3697,6 +3700,17 @@ function_call_expr ::=
   :}
   ;
 
+array_expr ::=
+  KW_ARRAY LPAREN function_params:params RPAREN
+  {:
+    RESULT = new FunctionCallExpr("array", params);
+  :}
+  | KW_ARRAY LPAREN RPAREN
+  {:
+    RESULT = new ArrayLiteral();
 
 Review comment:
   what does `array()` mean?

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386433329
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/common/FeMetaVersion.java
 ##########
 @@ -157,6 +157,8 @@
     public static final int VERSION_72 = 72;
     // broker persist isAlive
     public static final int VERSION_73 = 73;
+    // array type
+    public static final int VERSION_74 = 74;
 
 Review comment:
   you should also change the VERSION_CURRENT below.

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386750847
 
 

 ##########
 File path: be/src/util/mysql_row_buffer.cpp
 ##########
 @@ -67,6 +71,25 @@ MysqlRowBuffer::~MysqlRowBuffer() {
     }
 }
 
+void MysqlRowBuffer::open_dynamic_mode() {
+    if (_dynamic_mode == 0) {
+        *_pos++ = 254;
+        // write length when dynamic mode close
+        _len_pos = _pos;
+        _pos = _pos + 8;
+    }
+    _dynamic_mode++;
+}
+
+void MysqlRowBuffer::close_dynamic_mode() {
+    _dynamic_mode--;
+
+    if (_dynamic_mode == 0) {
 
 Review comment:
   ```suggestion
       if (NOT_DYNAMIC_MODE) {
   ```

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r387475124
 
 

 ##########
 File path: be/src/util/mysql_row_buffer.h
 ##########
 @@ -57,13 +57,44 @@ class MysqlRowBuffer {
     int length() const {
         return _pos - _buf;
     }
+
+    /**
+     * Why?
+     * Because the Nested-Type's data need pushed multiple times, but mysql protocal don't support nested type and each 
+     * push will decide a column data.  
+     * 
+     * How?
+     * Dynamic mode allow we can push data in a column multiple times, and allow recursive push. 
+     * NOTE:Need to ensure the open() and close() appear in pairs
+     * 
+     * the code:
+     *     mrb.push_smallint(120);
+     *     mrb.push_int(-30000);
+     *     mrb.push_bigint(900000);
+     * 
+     * In normal mode, the buffer contains three column:
+     *  1-'5'-3-'120'-6-'-30000' 
 
 Review comment:
   Em......Mysql protocol like this:
   
   ```
   | flag | (size) | value | flag | (size) | value | ......
   ^-------A column------^
   
   The flag means value's length or null value:
   If value is NULL, flag is 251
   If value's length < 250, flag is the value's length
   If 250 < value's length < 65536, flag is 252 and the next two byte is length
   If 65536 < value's length < 16777216 , flag is 253 and the next three byte is length
   If 16777216 < value's length, flag is 254 and the next eighth byte is length
   ```
   
   I will update the 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] imay commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386077430
 
 

 ##########
 File path: be/src/runtime/result_writer.cpp
 ##########
 @@ -57,20 +57,15 @@ Status ResultWriter::init(RuntimeState* state) {
     return Status::OK();
 }
 
-Status ResultWriter::add_one_row(TupleRow* row) {
-    _row_buffer->reset();
-    int num_columns = _output_expr_ctxs.size();
+int ResultWriter::add_row_value(int index, const TypeDescriptor& type, void* item) {
     int buf_ret = 0;
 
-    for (int i = 0; 0 == buf_ret && i < num_columns; ++i) {
-        void* item = _output_expr_ctxs[i]->get_value(row);
+    if (NULL == item) {
 
 Review comment:
   ```suggestion
       if (item == 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] imay commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386077261
 
 

 ##########
 File path: be/src/exprs/collection_functions.h
 ##########
 @@ -0,0 +1,38 @@
+// 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.
+
+#ifndef DORIS_BE_SRC_QUERY_EXPRS_COLLECTION_FUNCTIONS_H
+#define DORIS_BE_SRC_QUERY_EXPRS_COLLECTION_FUNCTIONS_H
+
+
+#include "anyval_util.h"
+
+namespace doris {
+
+class CollectionFunctions {
+    
+public:
+    static void init();
+
+    static CollectionVal array(FunctionContext* context, int num_children, const IntVal* values);
 
 Review comment:
   Give some comments for these functions

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386767550
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/ScalarType.java
 ##########
 @@ -141,6 +141,9 @@ public static ScalarType createType(PrimitiveType type) {
                 return DEFAULT_DECIMALV2;
             case LARGEINT:
                 return LARGEINT;
+            case ARRAY:
+                // can't static, because the same array may cause circular dependency
 
 Review comment:
   Nested type need decide sub-element type in runtime, return a static object will take repeatedly set the sub-element type of same object in many places......

----------------------------------------------------------------
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] imay commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386077191
 
 

 ##########
 File path: be/src/common/object_pool.h
 ##########
 @@ -47,6 +47,15 @@ class ObjectPool {
         _objects.push_back(obj);
         return t;
     }
+    
+    template <class T>
+    T* add_array(T* t) {
+        ArrayElement<T>* obj = new ArrayElement<T>(t);
+        DCHECK(obj != NULL);
+        boost::lock_guard<SpinLock> l(_lock);
 
 Review comment:
   use std::lock_guard instead

----------------------------------------------------------------
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] imay commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386078055
 
 

 ##########
 File path: be/src/udf/udf.h
 ##########
 @@ -786,6 +792,31 @@ struct HllVal : public StringVal {
     void agg_merge(const HllVal &other);
 };
 
+struct CollectionVal : public AnyVal {
 
 Review comment:
   Actually, if this struct is defined, the struct can not be changed, because some UDF function will use this struct. If this is only used for array, we can make it ArrayVal?

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386469025
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/rewrite/FEFunctions.java
 ##########
 @@ -421,4 +422,21 @@ public static LiteralExpr ifNullBigInt(LiteralExpr first, LiteralExpr second) th
     public static LiteralExpr ifNullDateTime(LiteralExpr first, LiteralExpr second) throws AnalysisException {
         return first instanceof NullLiteral ? second : first;
     }
+
+    /**
+     ------------------------------------------------------------------------------
+     */
+
+    /**
+     * Nest type function
+     */
+    @FEFunction(name = "array", argTypes = {"INT"}, returnType = "ARRAY")
+    public static LiteralExpr arrayInt(LiteralExpr... exprs) throws AnalysisException {
+        return new ArrayLiteral(exprs);
+    }
+
+    @FEFunction(name = "array", argTypes = {"VARCHAR"}, returnType = "ARRAY")
+    public static LiteralExpr array(StringLiteral... exprs) throws AnalysisException {
 
 Review comment:
   better to call arrayString()?

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386782383
 
 

 ##########
 File path: be/src/runtime/raw_value.cpp
 ##########
 @@ -300,7 +314,34 @@ void RawValue::write(const void* value, void* dst, const TypeDescriptor& type, M
         } else {
             dest->ptr = src->ptr;
         }
+        break;
+    }
+    case TYPE_ARRAY: {
+        DCHECK_EQ(type.children.size(), 1);
+        
+        const CollectionValue* src = reinterpret_cast<const CollectionValue*>(value);
+        CollectionValue* val = reinterpret_cast<CollectionValue*>(dst);
 
+        if (pool != NULL) {
+            auto children_type = type.children.at(0).type;
+            CollectionValue::init_collection(pool, src->size(), children_type, val);
+            CollectionIterator src_iter = src->iterator(children_type);
+            CollectionIterator val_iter = val->iterator(children_type);
+
+            val->copy_null_signs(src);
+            
+            while (src_iter.has_next() && val_iter.has_next()) {
+                if (!src_iter.is_null()) {
+                    // write children 
+                    write(src_iter.value(), val_iter.value(), children_type, pool);    
+                }
 
 Review comment:
   Need,  has resolve it through copy null_signs in line 331

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386764936
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/ScalarType.java
 ##########
 @@ -141,6 +141,9 @@ public static ScalarType createType(PrimitiveType type) {
                 return DEFAULT_DECIMALV2;
             case LARGEINT:
                 return LARGEINT;
+            case ARRAY:
+                // can't static, because the same array may cause circular dependency
 
 Review comment:
   Nested type need decide sub-element type in runtime, return a static object will take repeatedly set the sub-element type of same object in many places......

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386764936
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/ScalarType.java
 ##########
 @@ -141,6 +141,9 @@ public static ScalarType createType(PrimitiveType type) {
                 return DEFAULT_DECIMALV2;
             case LARGEINT:
                 return LARGEINT;
+            case ARRAY:
+                // can't static, because the same array may cause circular dependency
 
 Review comment:
   Nested type need decide sub-element type in runtime, return a static object will take repeatedly set the sub-element type of same object in many places......

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386223835
 
 

 ##########
 File path: be/src/udf/udf.h
 ##########
 @@ -786,6 +792,31 @@ struct HllVal : public StringVal {
     void agg_merge(const HllVal &other);
 };
 
+struct CollectionVal : public AnyVal {
 
 Review comment:
   I referenced the implementation of Impala,  nested type(array, map)  is used same struct to describe, the different is the data in struct, so I think we can also use same struct. Personally, I also think called ArrayVal and ArrayValue is good idea

----------------------------------------------------------------
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 #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386983752
 
 

 ##########
 File path: be/src/exprs/anyval_util.cpp
 ##########
 @@ -164,10 +166,64 @@ FunctionContext::TypeDesc AnyValUtil::column_type_to_type_desc(const TypeDescrip
     case TYPE_NULL:
         out.type = FunctionContext::TYPE_NULL;
         break;
+    case TYPE_ARRAY: {
+        out.type = FunctionContext::TYPE_ARRAY;
+        for (auto&& t : type.children) {
 
 Review comment:
   why use right value reference 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


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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386777205
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/PrimitiveType.java
 ##########
 @@ -56,6 +56,7 @@
     TIME("TIME", 8, TPrimitiveType.TIME),
     // we use OBJECT type represent BITMAP type in Backend
     BITMAP("BITMAP", 16, TPrimitiveType.OBJECT),
+    ARRAY("ARRAY", 16, TPrimitiveType.ARRAY),
 
 Review comment:
   Oh, I misunderstood..

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386468089
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/ScalarType.java
 ##########
 @@ -141,6 +141,9 @@ public static ScalarType createType(PrimitiveType type) {
                 return DEFAULT_DECIMALV2;
             case LARGEINT:
                 return LARGEINT;
+            case ARRAY:
+                // can't static, because the same array may cause circular dependency
 
 Review comment:
   What does this mean?

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r387475124
 
 

 ##########
 File path: be/src/util/mysql_row_buffer.h
 ##########
 @@ -57,13 +57,44 @@ class MysqlRowBuffer {
     int length() const {
         return _pos - _buf;
     }
+
+    /**
+     * Why?
+     * Because the Nested-Type's data need pushed multiple times, but mysql protocal don't support nested type and each 
+     * push will decide a column data.  
+     * 
+     * How?
+     * Dynamic mode allow we can push data in a column multiple times, and allow recursive push. 
+     * NOTE:Need to ensure the open() and close() appear in pairs
+     * 
+     * the code:
+     *     mrb.push_smallint(120);
+     *     mrb.push_int(-30000);
+     *     mrb.push_bigint(900000);
+     * 
+     * In normal mode, the buffer contains three column:
+     *  1-'5'-3-'120'-6-'-30000' 
 
 Review comment:
   Em......Mysql protocol like this:
   
   ```
   | flag | (size) | value | flag | (size) | value | ......
   ^-----A column-----^
   
   The flag means value's length or null value:
   If value is NULL, flag is 251
   If value's length < 250, flag is the value's length
   If 250 < value's length < 65536, flag is 252 and the next two byte is length
   If 65536 < value's length < 16777216 , flag is 253 and the next three byte is length
   If 16777216 < value's length, flag is 254 and the next eighth byte is length
   ```
   
   I will update the 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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r387475124
 
 

 ##########
 File path: be/src/util/mysql_row_buffer.h
 ##########
 @@ -57,13 +57,44 @@ class MysqlRowBuffer {
     int length() const {
         return _pos - _buf;
     }
+
+    /**
+     * Why?
+     * Because the Nested-Type's data need pushed multiple times, but mysql protocal don't support nested type and each 
+     * push will decide a column data.  
+     * 
+     * How?
+     * Dynamic mode allow we can push data in a column multiple times, and allow recursive push. 
+     * NOTE:Need to ensure the open() and close() appear in pairs
+     * 
+     * the code:
+     *     mrb.push_smallint(120);
+     *     mrb.push_int(-30000);
+     *     mrb.push_bigint(900000);
+     * 
+     * In normal mode, the buffer contains three column:
+     *  1-'5'-3-'120'-6-'-30000' 
 
 Review comment:
   Em......Mysql protocol like this:
   
   ```
   | flag | (length) | value | flag | (length) | value | ......
   ^-------A column------^
   
   The flag means value's length or null value:
   If value is NULL, flag is 251
   If value's length < 250, flag is the value's length
   If 250 < value's length < 65536, flag is 252 and the next two byte is length
   If 65536 < value's length < 16777216 , flag is 253 and the next three byte is length
   If 16777216 < value's length, flag is 254 and the next eighth byte is length
   ```
   
   I will update the 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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386766810
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/Type.java
 ##########
 @@ -43,11 +43,8 @@
 public abstract class Type {
     private static final Logger LOG = LogManager.getLogger(Type.class);
 
-    // Maximum nesting depth of a type. This limit was determined experimentally by
-    // generating and scanning deeply nested Parquet and Avro files. In those experiments,
-    // we exceeded the stack space in the scanner (which uses recursion for dealing with
-    // nested types) at a nesting depth between 200 and 300 (200 worked, 300 crashed).
-    public static int MAX_NESTING_DEPTH = 100;
+    // Maximum nesting depth of a type.
 
 Review comment:
   Just limit the depth of nested types, deeper nested type require a lot of extra development, and I don't think we need to support more than 2 depths, like array<map<array<int>>>......

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386773935
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/Column.java
 ##########
 @@ -419,6 +485,14 @@ public void write(DataOutput out) throws IOException {
         stats.write(out);
 
         Text.writeString(out, comment);
+
+        if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_74) {
 
 Review comment:
   OK, we can refactor the serialized and deserialized method of Column in other issuses? 

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386224277
 
 

 ##########
 File path: be/src/util/mysql_row_buffer.cpp
 ##########
 @@ -26,6 +26,8 @@
 
 namespace doris {
 
+#define NOT_DYNAMIC_MODE  (_dynamic_mode == 0)           
 
 Review comment:
   OK. Will add comment in .h

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r387466203
 
 

 ##########
 File path: be/src/exprs/anyval_util.h
 ##########
 @@ -296,6 +302,8 @@ class AnyValUtil {
 
     static FunctionContext::TypeDesc column_type_to_type_desc(const TypeDescriptor& type);
 
+    static PrimitiveType function_type_to_primitive_type(const FunctionContext::Type& type);
 
 Review comment:
   useless function, will delete it.

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386771623
 
 

 ##########
 File path: be/src/runtime/collection_value.cpp
 ##########
 @@ -0,0 +1,204 @@
+// 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 "runtime/collection_value.h"
+
+#include "util/bitmap.h"
+
+#include "common/logging.h"
+#include "exprs/anyval_util.h"
+
+
+namespace doris {
+
+int sizeof_type(const PrimitiveType& type) {
+    switch (type) {
+        case TYPE_INT:
 
 Review comment:
   What about tiny/small/bigint?

----------------------------------------------------------------
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] imay commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386077355
 
 

 ##########
 File path: be/src/runtime/collection_value.cpp
 ##########
 @@ -0,0 +1,203 @@
+// 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 "runtime/collection_value.h"
+
+#include "util/bitmap.h"
+
+#include "common/logging.h"
+#include "exprs/anyval_util.h"
+
+
+namespace doris {
+
+int sizeof_type(const PrimitiveType &type) {
 
 Review comment:
   ```suggestion
   int sizeof_type(const PrimitiveType& 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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386763223
 
 

 ##########
 File path: fe/src/main/cup/sql_parser.cup
 ##########
 @@ -3697,6 +3700,17 @@ function_call_expr ::=
   :}
   ;
 
+array_expr ::=
+  KW_ARRAY LPAREN function_params:params RPAREN
+  {:
+    RESULT = new FunctionCallExpr("array", params);
+  :}
+  | KW_ARRAY LPAREN RPAREN
+  {:
+    RESULT = new ArrayLiteral();
 
 Review comment:
   For create a array without any sub-element

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386466888
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/Column.java
 ##########
 @@ -419,6 +485,14 @@ public void write(DataOutput out) throws IOException {
         stats.write(out);
 
         Text.writeString(out, comment);
+
+        if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_74) {
 
 Review comment:
   Since Column is already support GSON serde, we can change the write method to GSON.
   And write method does not need to judge the FeMetaVersion

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r387475124
 
 

 ##########
 File path: be/src/util/mysql_row_buffer.h
 ##########
 @@ -57,13 +57,44 @@ class MysqlRowBuffer {
     int length() const {
         return _pos - _buf;
     }
+
+    /**
+     * Why?
+     * Because the Nested-Type's data need pushed multiple times, but mysql protocal don't support nested type and each 
+     * push will decide a column data.  
+     * 
+     * How?
+     * Dynamic mode allow we can push data in a column multiple times, and allow recursive push. 
+     * NOTE:Need to ensure the open() and close() appear in pairs
+     * 
+     * the code:
+     *     mrb.push_smallint(120);
+     *     mrb.push_int(-30000);
+     *     mrb.push_bigint(900000);
+     * 
+     * In normal mode, the buffer contains three column:
+     *  1-'5'-3-'120'-6-'-30000' 
 
 Review comment:
   Em......Mysql protocol like this:
   
   ```
   | flag | (length) | value | flag | (length) | value | ......
   ^-------A column------^
   
   The flag means value's length or null value:
   If value is NULL, flag is 251
   If value's length < 250, flag is the value's length
   If 250 < value's length < 65536, flag is 252 and the next two byte is length
   If 65536 < value's length < 16777216 , flag is 253 and the next three byte is length
   If 16777216 < value's length, flag is 254 and the next eighth byte is length
   ```
   
   I will update the 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] kangpinghuang commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386984292
 
 

 ##########
 File path: be/src/exprs/anyval_util.h
 ##########
 @@ -296,6 +302,8 @@ class AnyValUtil {
 
     static FunctionContext::TypeDesc column_type_to_type_desc(const TypeDescriptor& type);
 
+    static PrimitiveType function_type_to_primitive_type(const FunctionContext::Type& type);
 
 Review comment:
   what is this api for?

----------------------------------------------------------------
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] imay commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386077404
 
 

 ##########
 File path: be/src/runtime/collection_value.h
 ##########
 @@ -0,0 +1,123 @@
+// 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.
+
+#ifndef DORIS_BE_RUNTIME_COLLECTION_VALUE_H
+#define DORIS_BE_RUNTIME_COLLECTION_VALUE_H
+
+#include "udf/udf.h"
+#include "common/object_pool.h"
+#include "common/status.h"
+#include "runtime/mem_pool.h"
+#include "runtime/types.h"
+
+namespace doris {
+
+using doris_udf::AnyVal;
+
+class CollectionIterator;
+
+class CollectionValue {
 
 Review comment:
   comment for this class

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386766810
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/Type.java
 ##########
 @@ -43,11 +43,8 @@
 public abstract class Type {
     private static final Logger LOG = LogManager.getLogger(Type.class);
 
-    // Maximum nesting depth of a type. This limit was determined experimentally by
-    // generating and scanning deeply nested Parquet and Avro files. In those experiments,
-    // we exceeded the stack space in the scanner (which uses recursion for dealing with
-    // nested types) at a nesting depth between 200 and 300 (200 worked, 300 crashed).
-    public static int MAX_NESTING_DEPTH = 100;
+    // Maximum nesting depth of a type.
 
 Review comment:
   Just limit the depth of nested types, deeper nested type require a lot of extra development, and I don't think we need to support more than 2 depths, like array<array<array&lt;int&gt;>>......

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386763223
 
 

 ##########
 File path: fe/src/main/cup/sql_parser.cup
 ##########
 @@ -3697,6 +3700,17 @@ function_call_expr ::=
   :}
   ;
 
+array_expr ::=
+  KW_ARRAY LPAREN function_params:params RPAREN
+  {:
+    RESULT = new FunctionCallExpr("array", params);
+  :}
+  | KW_ARRAY LPAREN RPAREN
+  {:
+    RESULT = new ArrayLiteral();
 
 Review comment:
   For create an array without any sub-element

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r387475124
 
 

 ##########
 File path: be/src/util/mysql_row_buffer.h
 ##########
 @@ -57,13 +57,44 @@ class MysqlRowBuffer {
     int length() const {
         return _pos - _buf;
     }
+
+    /**
+     * Why?
+     * Because the Nested-Type's data need pushed multiple times, but mysql protocal don't support nested type and each 
+     * push will decide a column data.  
+     * 
+     * How?
+     * Dynamic mode allow we can push data in a column multiple times, and allow recursive push. 
+     * NOTE:Need to ensure the open() and close() appear in pairs
+     * 
+     * the code:
+     *     mrb.push_smallint(120);
+     *     mrb.push_int(-30000);
+     *     mrb.push_bigint(900000);
+     * 
+     * In normal mode, the buffer contains three column:
+     *  1-'5'-3-'120'-6-'-30000' 
 
 Review comment:
   Em......Mysql protocol like this:
   
   ```
   | flag | (size) | value | flag | (size) | value | ......
   
   The flag means value's length or null value:
   If value is NULL, flag is 251
   If value's length < 250, flag is the value's length
   If 250 < value's length < 65536, flag is 252 and the next two byte is length
   If 65536 < value's length < 16777216 , flag is 253 and the next three byte is length
   If 16777216 < value's length, flag is 254 and the next eighth byte is length
   ```
   
   I will update the 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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386768258
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/rewrite/FEFunctions.java
 ##########
 @@ -421,4 +422,21 @@ public static LiteralExpr ifNullBigInt(LiteralExpr first, LiteralExpr second) th
     public static LiteralExpr ifNullDateTime(LiteralExpr first, LiteralExpr second) throws AnalysisException {
         return first instanceof NullLiteral ? second : first;
     }
+
+    /**
+     ------------------------------------------------------------------------------
+     */
+
+    /**
+     * Nest type function
+     */
+    @FEFunction(name = "array", argTypes = {"INT"}, returnType = "ARRAY")
+    public static LiteralExpr arrayInt(LiteralExpr... exprs) throws AnalysisException {
+        return new ArrayLiteral(exprs);
+    }
+
+    @FEFunction(name = "array", argTypes = {"VARCHAR"}, returnType = "ARRAY")
+    public static LiteralExpr array(StringLiteral... exprs) throws AnalysisException {
 
 Review comment:
   Em...the method is same with array(IntLiteral.....) in fact, and maybe will same with array(BigintLiteral.....)、array(BoolLiteral.....) in future......I think better way is refactor the annotation for support a list of types

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386784395
 
 

 ##########
 File path: be/src/runtime/collection_value.cpp
 ##########
 @@ -0,0 +1,204 @@
+// 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 "runtime/collection_value.h"
+
+#include "util/bitmap.h"
+
+#include "common/logging.h"
+#include "exprs/anyval_util.h"
+
+
+namespace doris {
+
+int sizeof_type(const PrimitiveType& type) {
+    switch (type) {
+        case TYPE_INT:
 
 Review comment:
   Just support INT/String in current version, but FE will auto cast to int if sub-element type is   tiny/small/bigint,  maybe will support other types in future.

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386459677
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/analysis/TypeDef.java
 ##########
 @@ -67,6 +69,20 @@ private void analyze(Type type) throws AnalysisException {
     if (!type.isSupported()) {
       throw new AnalysisException("Unsupported data type: " + type.toSql());
     }
+
+    if (type.isArrayType()) {
+      if (FeMetaVersion.VERSION_CURRENT < FeMetaVersion.VERSION_74) {
 
 Review comment:
   This is not a valid way to forbid a feature...

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386772519
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/PrimitiveType.java
 ##########
 @@ -56,6 +56,7 @@
     TIME("TIME", 8, TPrimitiveType.TIME),
     // we use OBJECT type represent BITMAP type in Backend
     BITMAP("BITMAP", 16, TPrimitiveType.OBJECT),
+    ARRAY("ARRAY", 16, TPrimitiveType.ARRAY),
 
 Review comment:
   The slot_size is used for Tuple to compute memory layout, why need change to 17?

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386465055
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/Column.java
 ##########
 @@ -243,9 +272,36 @@ public TColumn toThrift() {
         tColumn.setIs_key(this.isKey);
         tColumn.setIs_allow_null(this.isAllowNull);
         tColumn.setDefault_value(this.defaultValue);
+
+        toChildrenThrift(this, tColumn);
 
 Review comment:
   It is strange to pass `this` to a private method

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386782639
 
 

 ##########
 File path: be/src/util/mysql_row_buffer.cpp
 ##########
 @@ -109,15 +132,24 @@ int MysqlRowBuffer::push_tinyint(int8_t data) {
         return ret;
     }
 
-    int length = snprintf(_pos + 1, MAX_TINYINT_WIDTH + 2, "%d", data);
+    int size_bytes = 0;
+    if (NOT_DYNAMIC_MODE) {
 
 Review comment:
   Good idea, will modify it.

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386772519
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/PrimitiveType.java
 ##########
 @@ -56,6 +56,7 @@
     TIME("TIME", 8, TPrimitiveType.TIME),
     // we use OBJECT type represent BITMAP type in Backend
     BITMAP("BITMAP", 16, TPrimitiveType.OBJECT),
+    ARRAY("ARRAY", 16, TPrimitiveType.ARRAY),
 
 Review comment:
   The slot_size is used for Tuple to compute memory layout, why need change to 17?

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r387475124
 
 

 ##########
 File path: be/src/util/mysql_row_buffer.h
 ##########
 @@ -57,13 +57,44 @@ class MysqlRowBuffer {
     int length() const {
         return _pos - _buf;
     }
+
+    /**
+     * Why?
+     * Because the Nested-Type's data need pushed multiple times, but mysql protocal don't support nested type and each 
+     * push will decide a column data.  
+     * 
+     * How?
+     * Dynamic mode allow we can push data in a column multiple times, and allow recursive push. 
+     * NOTE:Need to ensure the open() and close() appear in pairs
+     * 
+     * the code:
+     *     mrb.push_smallint(120);
+     *     mrb.push_int(-30000);
+     *     mrb.push_bigint(900000);
+     * 
+     * In normal mode, the buffer contains three column:
+     *  1-'5'-3-'120'-6-'-30000' 
 
 Review comment:
   Em......Mysql protocol like this:
   
   ```
   | flag | (size) | value | flag | (size) | value | ......
   ^-----A colum------^
   
   The flag means value's length or null value:
   If value is NULL, flag is 251
   If value's length < 250, flag is the value's length
   If 250 < value's length < 65536, flag is 252 and the next two byte is length
   If 65536 < value's length < 16777216 , flag is 253 and the next three byte is length
   If 16777216 < value's length, flag is 254 and the next eighth byte is length
   ```
   
   I will update the 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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r387465139
 
 

 ##########
 File path: be/src/common/object_pool.h
 ##########
 @@ -18,6 +18,7 @@
 #ifndef DORIS_BE_SRC_COMMON_COMMON_OBJECT_POOL_H
 #define DORIS_BE_SRC_COMMON_COMMON_OBJECT_POOL_H
 
+#include <mutex>
 
 Review comment:
   use for std::lock_guard in line 56

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386461175
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/analysis/TypeDef.java
 ##########
 @@ -67,6 +69,20 @@ private void analyze(Type type) throws AnalysisException {
     if (!type.isSupported()) {
       throw new AnalysisException("Unsupported data type: " + type.toSql());
     }
+
+    if (type.isArrayType()) {
+      if (FeMetaVersion.VERSION_CURRENT < FeMetaVersion.VERSION_74) {
+        throw new AnalysisException("Unsupported data type: " + type.toSql());
+      }
+
+      Type itemType = ((ArrayType) type).getItemType();
+      if (itemType.isNull()) {
 
 Review comment:
   I see that you just support INT 和 VARCHAR as the item type of array? Why just checking null 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


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 #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
kangpinghuang commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386982741
 
 

 ##########
 File path: be/src/common/object_pool.h
 ##########
 @@ -18,6 +18,7 @@
 #ifndef DORIS_BE_SRC_COMMON_COMMON_OBJECT_POOL_H
 #define DORIS_BE_SRC_COMMON_COMMON_OBJECT_POOL_H
 
+#include <mutex>
 
 Review comment:
   useless?

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386468185
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/Type.java
 ##########
 @@ -43,11 +43,8 @@
 public abstract class Type {
     private static final Logger LOG = LogManager.getLogger(Type.class);
 
-    // Maximum nesting depth of a type. This limit was determined experimentally by
-    // generating and scanning deeply nested Parquet and Avro files. In those experiments,
-    // we exceeded the stack space in the scanner (which uses recursion for dealing with
-    // nested types) at a nesting depth between 200 and 300 (200 worked, 300 crashed).
-    public static int MAX_NESTING_DEPTH = 100;
+    // Maximum nesting depth of a type.
 
 Review comment:
   Why we just support 2?

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386750772
 
 

 ##########
 File path: be/src/util/mysql_row_buffer.cpp
 ##########
 @@ -67,6 +71,25 @@ MysqlRowBuffer::~MysqlRowBuffer() {
     }
 }
 
+void MysqlRowBuffer::open_dynamic_mode() {
+    if (_dynamic_mode == 0) {
 
 Review comment:
   ```suggestion
       if (NOT_DYNAMIC_MODE) {
   ```

----------------------------------------------------------------
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] Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
Seaven commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386767550
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/catalog/ScalarType.java
 ##########
 @@ -141,6 +141,9 @@ public static ScalarType createType(PrimitiveType type) {
                 return DEFAULT_DECIMALV2;
             case LARGEINT:
                 return LARGEINT;
+            case ARRAY:
+                // can't static, because the same array may cause circular dependency
 
 Review comment:
   Nested type need decide sub-element type in runtime, return a static object will take repeatedly set the sub-element type of same object in many places......

----------------------------------------------------------------
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] morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2979: Incomplete support Array type (#2871)
URL: https://github.com/apache/incubator-doris/pull/2979#discussion_r386457002
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/analysis/ExpressionFunctions.java
 ##########
 @@ -221,6 +221,12 @@ public LiteralExpr invoke(List<Expr> args) throws AnalysisException {
             } else {
                 throw new IllegalArgumentException("Doris does't support type:" + argType);
             }
+
+            // if args all is NullLiteral
 
 Review comment:
   Why not using `argType.isNull()` and put this logic into above `if else` block?

----------------------------------------------------------------
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] yangzhg closed pull request #2979: Incomplete support Array type (#2871)

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


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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



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