This is an automated email from the ASF dual-hosted git repository.

zclll pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new d26e4953148 [Feature](func) Support function EXPORT_SET (#56798)
d26e4953148 is described below

commit d26e49531484e71072fe871af6ade5b8a6507504
Author: linrrarity <[email protected]>
AuthorDate: Tue Oct 14 15:34:22 2025 +0800

    [Feature](func) Support function EXPORT_SET (#56798)
    
    `EXPORT_SET` is used to convert each bit of an integer into a specified
    string and concatenate them into a result string. For each bit in `bits`
    that is 1, the corresponding position in the result will display the
    `on` string; for each bit that is 0, the `off` string will be displayed.
    The bits are checked in order from right to left (i.e., from the least
    significant bit to the most significant bit), but concatenated into the
    result string from left to right. The bits are separated by the
    `separator` (default is a comma `,`).
    
    ```text
    mysql> SELECT EXPORT_SET(5, '1', '0');
    
+---------------------------------------------------------------------------------------------------------------------------------+
    | EXPORT_SET(5, '1', '0')                                                   
                                                      |
    
+---------------------------------------------------------------------------------------------------------------------------------+
    | 
1,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0
 |
    
+---------------------------------------------------------------------------------------------------------------------------------+
    
    mysql> SELECT EXPORT_SET(-1, '1', '0', ',');
    
+---------------------------------------------------------------------------------------------------------------------------------+
    | EXPORT_SET(-1, '1', '0', ',')                                             
                                                      |
    
+---------------------------------------------------------------------------------------------------------------------------------+
    | 
1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
 |
    
+---------------------------------------------------------------------------------------------------------------------------------+
    
    mysql> SELECT EXPORT_SET(5, '1', '0', '|', 5);
    +---------------------------------+
    | EXPORT_SET(5, '1', '0', '|', 5) |
    +---------------------------------+
    | 1|0|1|0|0                       |
    +---------------------------------+
    
    mysql> SELECT EXPORT_SET(5, 'apache', 'doris', '123', 5);
    +--------------------------------------------+
    | EXPORT_SET(5, 'apache', 'doris', '123', 5) |
    +--------------------------------------------+
    | apache123doris123apache123doris123doris    |
    +--------------------------------------------+
    ```
---
 be/src/vec/functions/function_string.cpp           |   1 +
 be/src/vec/functions/function_string.h             | 128 ++++++++++++
 .../doris/catalog/BuiltinScalarFunctions.java      |   4 +-
 .../functions/executable/StringArithmetic.java     | 101 ++++++++++
 .../expressions/functions/scalar/ExportSet.java    |  93 +++++++++
 .../expressions/visitor/ScalarFunctionVisitor.java |   5 +
 .../string_functions/test_string_function.out      | 217 +++++++++++++++++++++
 .../string_functions/test_string_function.groovy   |  99 ++++++++++
 8 files changed, 647 insertions(+), 1 deletion(-)

diff --git a/be/src/vec/functions/function_string.cpp 
b/be/src/vec/functions/function_string.cpp
index 8695b0d1b24..f52e90a5bdf 100644
--- a/be/src/vec/functions/function_string.cpp
+++ b/be/src/vec/functions/function_string.cpp
@@ -1428,6 +1428,7 @@ void register_function_string(SimpleFunctionFactory& 
factory) {
     factory.register_function<FunctionXPathString>();
     factory.register_function<FunctionCrc32Internal>();
     factory.register_function<FunctionMakeSet>();
+    factory.register_function<FunctionExportSet>();
 
     factory.register_alias(FunctionLeft::name, "strleft");
     factory.register_alias(FunctionRight::name, "strright");
diff --git a/be/src/vec/functions/function_string.h 
b/be/src/vec/functions/function_string.h
index 3c9310e1eab..8dd22df0c6f 100644
--- a/be/src/vec/functions/function_string.h
+++ b/be/src/vec/functions/function_string.h
@@ -5097,6 +5097,134 @@ private:
     }
 };
 
+class FunctionExportSet : public IFunction {
+public:
+    static constexpr auto name = "export_set";
+    static FunctionPtr create() { return 
std::make_shared<FunctionExportSet>(); }
+    String get_name() const override { return name; }
+    size_t get_number_of_arguments() const override { return 0; }
+    bool is_variadic() const override { return true; }
+    DataTypePtr get_return_type_impl(const DataTypes& arguments) const 
override {
+        return std::make_shared<DataTypeString>();
+    }
+
+    Status execute_impl(FunctionContext* context, Block& block, const 
ColumnNumbers& arguments,
+                        uint32_t result, size_t input_rows_count) const 
override {
+        auto res_col = ColumnString::create();
+
+        const size_t arg_size = arguments.size();
+        bool col_const[5];
+        ColumnPtr arg_cols[5];
+        bool all_const = true;
+        for (int i = 0; i < arg_size; ++i) {
+            col_const[i] = 
is_column_const(*block.get_by_position(arguments[i]).column);
+            all_const = all_const && col_const[i];
+        }
+        std::tie(arg_cols[0], col_const[0]) =
+                unpack_if_const(block.get_by_position(arguments[0]).column);
+        if (arg_size == 3) {
+            default_preprocess_parameter_columns(arg_cols, col_const, {1, 2}, 
block, arguments);
+        } else if (arg_size == 4) {
+            default_preprocess_parameter_columns(arg_cols, col_const, {1, 2, 
3}, block, arguments);
+        } else if (arg_size == 5) {
+            default_preprocess_parameter_columns(arg_cols, col_const, {1, 2, 
3, 4}, block,
+                                                 arguments);
+        }
+
+        const auto* bit_col = assert_cast<const 
ColumnInt128*>(arg_cols[0].get());
+        const auto* on_col = assert_cast<const 
ColumnString*>(arg_cols[1].get());
+        const auto* off_col = assert_cast<const 
ColumnString*>(arg_cols[2].get());
+        const ColumnString* sep_col = nullptr;
+        const ColumnInt32* num_bits_col = nullptr;
+        if (arg_size > 3) {
+            sep_col = assert_cast<const ColumnString*>(arg_cols[3].get());
+            if (arg_size == 5) {
+                num_bits_col = assert_cast<const 
ColumnInt32*>(arg_cols[4].get());
+            }
+        }
+
+        for (size_t i = 0; i < input_rows_count; ++i) {
+            uint64_t bit =
+                    
check_and_get_bit(bit_col->get_element(index_check_const(i, col_const[0])));
+
+            size_t idx_for_args = all_const ? 0 : i;
+            StringRef on = on_col->get_data_at(idx_for_args);
+            StringRef off = off_col->get_data_at(idx_for_args);
+            StringRef separator(",", 1);
+            int8_t num_of_bits = 64;
+
+            if (arg_size > 3) {
+                separator = sep_col->get_data_at(idx_for_args);
+                if (arg_size == 5) {
+                    num_of_bits =
+                            
check_and_get_num_of_bits(num_bits_col->get_element(idx_for_args));
+                }
+            }
+
+            execute_single(bit, on, off, separator, num_of_bits, *res_col);
+        }
+        block.replace_by_position(result, std::move(res_col));
+        return Status::OK();
+    }
+
+private:
+    /* The valid range of the input `bit` parameter should be [-2^63, 2^64 - 1]
+     * If it exceeds this range, the MAX/MIN values of the signed 64-bit 
integer are used for calculation
+     * This behavior is consistent with MySQL.
+     */
+    uint64_t check_and_get_bit(__int128 col_bit_val) const {
+        if (col_bit_val > ULLONG_MAX) {
+            return LLONG_MAX;
+        } else if (col_bit_val < LLONG_MIN) {
+            return LLONG_MIN;
+        }
+        return static_cast<uint64_t>(col_bit_val);
+    }
+
+    // If the input value is not in the range [0, 64], return default value 64
+    int8_t check_and_get_num_of_bits(int32_t col_num_of_bits_val) const {
+        if (col_num_of_bits_val >= 0 && col_num_of_bits_val <= 64) {
+            return static_cast<int8_t>(col_num_of_bits_val);
+        }
+        return 64;
+    }
+
+    void execute_single(uint64_t bit, const StringRef& on, const StringRef& 
off,
+                        const StringRef& separator, int8_t num_of_bits,
+                        ColumnString& res_col) const {
+        ColumnString::Chars data;
+        data.reserve(std::max(on.size, off.size) * num_of_bits +
+                     separator.size * (num_of_bits - 1));
+
+        while (bit && num_of_bits) {
+            if (bit & 1) {
+                data.insert(on.data, on.data + on.size);
+            } else {
+                data.insert(off.data, off.data + off.size);
+            }
+            bit >>= 1;
+            if (--num_of_bits) {
+                data.insert(separator.data, separator.data + separator.size);
+            }
+        }
+
+        if (num_of_bits > 0) {
+            ColumnString::Chars off_sep_combo;
+            off_sep_combo.reserve(separator.size + off.size);
+            off_sep_combo.insert(off_sep_combo.end(), off.data, off.data + 
off.size);
+            off_sep_combo.insert(off_sep_combo.end(), separator.data,
+                                 separator.data + separator.size);
+
+            for (size_t i = 0; i < num_of_bits; ++i) {
+                data.insert(off_sep_combo.data(), off_sep_combo.data() + 
off_sep_combo.size());
+            }
+            data.erase(data.end() - separator.size, data.end());
+        }
+
+        res_col.insert_data(reinterpret_cast<const char*>(data.data()), 
data.size());
+    }
+};
+
 // ATTN: for debug only
 // compute crc32 hash value as the same way in 
`VOlapTablePartitionParam::find_tablets()`
 class FunctionCrc32Internal : public IFunction {
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
index 4c125b8f2f9..c4ef1d6c68c 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
@@ -198,6 +198,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.EndsWith;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.EsQuery;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Even;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Exp;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.ExportSet;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.ExtractUrlParameter;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Field;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.FindInSet;
@@ -1059,7 +1060,8 @@ public class BuiltinScalarFunctions implements 
FunctionHelper {
             scalar(AISimilarity.class, "ai_similarity"),
             scalar(Embed.class, "embed"),
             scalar(Uniform.class, "uniform"),
-            scalar(MakeSet.class, "make_set"));
+            scalar(MakeSet.class, "make_set"),
+            scalar(ExportSet.class, "export_set"));
 
     public static final BuiltinScalarFunctions INSTANCE = new 
BuiltinScalarFunctions();
 
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/StringArithmetic.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/StringArithmetic.java
index e04d4eca2b4..ae0f29806cc 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/StringArithmetic.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/StringArithmetic.java
@@ -45,6 +45,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 import java.io.UnsupportedEncodingException;
+import java.math.BigInteger;
 import java.net.MalformedURLException;
 import java.net.URI;
 import java.net.URISyntaxException;
@@ -1141,4 +1142,104 @@ public class StringArithmetic {
         }
         return castStringLikeLiteral(args[0], sb.toString());
     }
+
+    /**
+     * Executable arithmetic functions export_set in 3 args
+     */
+    @ExecFunction(name = "export_set")
+    public static Expression export_set(LargeIntLiteral bitLiteral, 
StringLikeLiteral on, StringLikeLiteral off) {
+        BigInteger ullongMax = 
BigInteger.ONE.shiftLeft(64).subtract(BigInteger.ONE);
+        BigInteger llongMin = BigInteger.valueOf(Long.MIN_VALUE);
+        BigInteger bitValue = bitLiteral.getValue();
+
+        long finalBitValue;
+        if (bitValue.compareTo(ullongMax) > 0) {
+            finalBitValue = Long.MAX_VALUE;
+        } else if (bitValue.compareTo(llongMin) < 0) {
+            finalBitValue = Long.MIN_VALUE;
+        } else {
+            finalBitValue = bitValue.longValue();
+        }
+
+        return exportSetImpl(finalBitValue, on.getValue(), off.getValue(), 
",", 64);
+    }
+
+    /**
+     * Executable arithmetic functions export_set in 4 args
+     */
+    @ExecFunction(name = "export_set")
+    public static Expression export_set(LargeIntLiteral bitLiteral, 
StringLikeLiteral on, StringLikeLiteral off,
+                                      StringLikeLiteral separator) {
+        BigInteger ullongMax = 
BigInteger.ONE.shiftLeft(64).subtract(BigInteger.ONE);
+        BigInteger llongMin = BigInteger.valueOf(Long.MIN_VALUE);
+        BigInteger bitValue = bitLiteral.getValue();
+
+        long finalBitValue;
+        if (bitValue.compareTo(ullongMax) > 0) {
+            finalBitValue = Long.MAX_VALUE;
+        } else if (bitValue.compareTo(llongMin) < 0) {
+            finalBitValue = Long.MIN_VALUE;
+        } else {
+            finalBitValue = bitValue.longValue();
+        }
+
+        return exportSetImpl(finalBitValue, on.getValue(), off.getValue(), 
separator.getValue(), 64);
+    }
+
+    /**
+     * Executable arithmetic functions export_set in 5 args
+     */
+    @ExecFunction(name = "export_set")
+    public static Expression export_set(LargeIntLiteral bitLiteral, 
StringLikeLiteral on, StringLikeLiteral off,
+                                      StringLikeLiteral separator, 
IntegerLiteral numBits) {
+        BigInteger ullongMax = 
BigInteger.ONE.shiftLeft(64).subtract(BigInteger.ONE);
+        BigInteger llongMin = BigInteger.valueOf(Long.MIN_VALUE);
+        BigInteger bitValue = bitLiteral.getValue();
+
+        long finalBitValue;
+        if (bitValue.compareTo(ullongMax) > 0) {
+            finalBitValue = Long.MAX_VALUE;
+        } else if (bitValue.compareTo(llongMin) < 0) {
+            finalBitValue = Long.MIN_VALUE;
+        } else {
+            finalBitValue = bitValue.longValue();
+        }
+
+        int bits = numBits.getValue();
+        if (bits < 0 || bits > 64) {
+            bits = 64;
+        }
+        return exportSetImpl(finalBitValue, on.getValue(), off.getValue(), 
separator.getValue(), bits);
+    }
+
+    private static Expression exportSetImpl(long bit, String on, String off, 
String separator, int numBits) {
+        StringBuilder result = new StringBuilder();
+        boolean first = true;
+
+        while (bit != 0 && numBits > 0) {
+            if (!first) {
+                result.append(separator);
+            }
+            first = false;
+
+            if ((bit & 1) == 1) {
+                result.append(on);
+            } else {
+                result.append(off);
+            }
+            bit >>>= 1;
+            numBits--;
+        }
+
+        while (numBits > 0) {
+            if (!first) {
+                result.append(separator);
+            }
+            first = false;
+            result.append(off);
+            numBits--;
+        }
+
+        return new VarcharLiteral(result.toString());
+    }
 }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ExportSet.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ExportSet.java
new file mode 100644
index 00000000000..087a7ef9de3
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ExportSet.java
@@ -0,0 +1,93 @@
+// 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.nereids.trees.expressions.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import 
org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.IntegerType;
+import org.apache.doris.nereids.types.LargeIntType;
+import org.apache.doris.nereids.types.StringType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'export_set'.
+ */
+public class ExportSet extends ScalarFunction
+        implements ExplicitlyCastableSignature, PropagateNullable {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(StringType.INSTANCE)
+                .args(LargeIntType.INSTANCE, StringType.INSTANCE, 
StringType.INSTANCE),
+            FunctionSignature.ret(StringType.INSTANCE)
+                .args(LargeIntType.INSTANCE, StringType.INSTANCE, 
StringType.INSTANCE, StringType.INSTANCE),
+            FunctionSignature.ret(StringType.INSTANCE)
+                .args(LargeIntType.INSTANCE, StringType.INSTANCE, 
StringType.INSTANCE, StringType.INSTANCE,
+                        IntegerType.INSTANCE)
+    );
+
+    /**
+     * constructor with 3 arguments.
+     */
+    public ExportSet(Expression arg0, Expression arg1, Expression arg2) {
+        super("export_set", arg0, arg1, arg2);
+    }
+
+    /**
+     * constructor with 4 arguments.
+     */
+    public ExportSet(Expression arg0, Expression arg1, Expression arg2, 
Expression arg3) {
+        super("export_set", arg0, arg1, arg2, arg3);
+    }
+
+    /**
+     * constructor with 5 arguments.
+     */
+    public ExportSet(Expression arg0, Expression arg1, Expression arg2, 
Expression arg3, Expression arg4) {
+        super("export_set", arg0, arg1, arg2, arg3, arg4);
+    }
+
+    /** constructor for withChildren*/
+    private ExportSet(ScalarFunctionParams functionParams) {
+        super(functionParams);
+    }
+
+    /**
+     * withChildren.
+     */
+    @Override
+    public ExportSet withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() >= 3 && children.size() <= 
5);
+        return new ExportSet(getFunctionParams(children));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitExportSet(this, context);
+    }
+}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
index d327bade315..630fcbe1c59 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
@@ -208,6 +208,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.EndsWith;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.EsQuery;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Even;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Exp;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.ExportSet;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.ExtractUrlParameter;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Field;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.FindInSet;
@@ -2595,4 +2596,8 @@ public interface ScalarFunctionVisitor<R, C> {
     default R visitMakeSet(MakeSet makeSet, C context) {
         return visitScalarFunction(makeSet, context);
     }
+
+    default R visitExportSet(ExportSet exportSet, C context) {
+        return visitScalarFunction(exportSet, context);
+    }
 }
diff --git 
a/regression-test/data/query_p0/sql_functions/string_functions/test_string_function.out
 
b/regression-test/data/query_p0/sql_functions/string_functions/test_string_function.out
index 4c3478b764b..2f75d240a02 100644
--- 
a/regression-test/data/query_p0/sql_functions/string_functions/test_string_function.out
+++ 
b/regression-test/data/query_p0/sql_functions/string_functions/test_string_function.out
@@ -816,3 +816,220 @@ x2,x4,x26,x51,x62,x63,x64
 
 -- !mask_set_4 --
 ,y6
+
+-- !export_set_1 --
+1      
1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
+2      0111111111111111111111111111111111111111111111111111111111111111
+3      Y,N,Y,N,N
+4      1010000000000000000000000000000000000000000000000000000000000000
+5      00000000000000000000000000000000000000000000000000000000000000
+6      11
+7      
1,1,0,1,1,0,1,1,0,0,1,1,0,0,1,0,0,1,1,1,0,1,1,0,1,0,1,1,1,1,1,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0
+8      
apache|123|doris|123|doris|123|apache|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|123|doris|12
 [...]
+9      \N
+10     \N
+11     \N
+12     \N
+13     \N
+
+-- !export_set_2 --
+1,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0
+
+-- !export_set_3 --
+你好?你好?你好?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0?0
+
+-- !export_set_4 --
+0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0
+
+-- !export_set_5 --
+1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,0
+
+-- !export_set_6 --
+0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1
+
+-- !export_set_7 --
+0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1
+
+-- !export_set_8 --
+1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
+
+-- !export_set_9 --
+1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
+
+-- !export_set_10 --
+1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,0
+
+-- !export_set_11 --
+0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1
+
+-- !export_set_12 --
+0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0
+
+-- !export_set_13 --
+1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0
+
+-- !export_set_14 --
+0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0
+
+-- !export_set_15 --
+0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,0
+
+-- !export_set_16 --
+0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,1
+
+-- !export_set_17 --
+0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,0,0,0
+
+-- !export_set_18 --
+1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
+
+-- !export_set_19 --
+0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1
+
+-- !export_set_20 --
+1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
+
+-- !export_set_21 --
+
+
+-- !export_set_22 --
+1,1,1,1,1,1,1,1
+
+-- !export_set_23 --
+1,1,1,1,1,1,1,1,1,1
+
+-- !export_set_24 --
+1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,0
+
+-- !export_set_25 --
+0,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
+
+-- !export_set_26 --
+1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,0
+
+-- !export_set_27 --
+1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
+
+-- !export_set_28 --
+0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1
+
+-- !export_set_29 --
+1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,0
+
+-- !export_set_30 --
+0,0,0,0,0,0,1,1,1,0,1,1,1,1,0,1,0,0,0,0,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
+
+-- !export_set_31 --
+0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1
+
+-- !export_set_32 --
+0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1
+
+-- !export_set_33 --
+1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1
+
+-- !export_set_34 --
+1      1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 
1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 
1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1
+2      0 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 
1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 
1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 1 ! 
1 ! 1 ! 1 ! 1 ! 1 ! 1
+3      Y ! N ! Y ! N ! N
+4      1 ! 0 ! 1 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 
0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 
0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 
0 ! 0 ! 0 ! 0 ! 0 ! 0
+5       ! 0 !  ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 
! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 
! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 
! 0 ! 0 ! 0 ! 0 ! 0
+6       ! 1 ! 1 !  !  !  !  !  !  !  !  !  !  !  !  !  !  !  !  !  !  !  !  !  
!  !  !  !  !  !  !  !  !  !  !  !  !  !  !  !  !  !  !  !  !  !  !  !  !  !  ! 
 !  !  !  !  !  !  !  !  !  !  !  ! 
+7      1 ! 1 ! 0 ! 1 ! 1 ! 0 ! 1 ! 1 ! 0 ! 0 ! 1 ! 1 ! 0 ! 0 ! 1 ! 0 ! 0 ! 1 ! 
1 ! 1 ! 0 ! 1 ! 1 ! 0 ! 1 ! 0 ! 1 ! 1 ! 1 ! 1 ! 1 ! 0 ! 0 ! 0 ! 1 ! 0 ! 0 ! 0 ! 
0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 
0 ! 0 ! 0 ! 0 ! 0 ! 0
+8      apache ! doris ! doris ! apache ! doris ! doris ! doris ! doris ! doris 
! doris ! doris ! doris ! doris ! doris ! doris ! doris ! doris ! doris ! doris 
! doris ! doris ! doris ! doris ! doris ! doris ! doris ! doris ! doris ! doris 
! doris ! doris ! doris ! doris ! doris ! doris ! doris ! doris ! doris ! doris 
! doris ! doris ! doris ! doris ! doris ! doris ! doris ! doris ! doris ! doris 
! doris ! doris ! doris ! doris ! doris ! doris ! doris ! doris ! doris ! doris 
! doris ! doris !  [...]
+9      \N
+10     \N
+11     \N
+12     1 ! 0 ! 1 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0 ! 0
+13     \N
+
+-- !export_set_35 --
+1      
1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1
+2      
0|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1|分隔符|1
+3      
Y|分隔符|N|分隔符|Y|分隔符|N|分隔符|N|分隔符|N|分隔符|N|分隔符|N|分隔符|N|分隔符|N|分隔符|N|分隔符|N|分隔符|N|分隔符|N|分隔符|N|分隔符|N|分隔符|N
+4      
1|分隔符|0|分隔符|1|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0
+5      
|分隔符|0|分隔符||分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0
+6      
|分隔符|1|分隔符|1|分隔符||分隔符||分隔符||分隔符||分隔符||分隔符||分隔符||分隔符||分隔符||分隔符||分隔符||分隔符||分隔符||分隔符|
+7      
1|分隔符|1|分隔符|0|分隔符|1|分隔符|1|分隔符|0|分隔符|1|分隔符|1|分隔符|0|分隔符|0|分隔符|1|分隔符|1|分隔符|0|分隔符|0|分隔符|1|分隔符|0|分隔符|0
+8      
apache|分隔符|doris|分隔符|doris|分隔符|apache|分隔符|doris|分隔符|doris|分隔符|doris|分隔符|doris|分隔符|doris|分隔符|doris|分隔符|doris|分隔符|doris|分隔符|doris|分隔符|doris|分隔符|doris|分隔符|doris|分隔符|doris
+9      \N
+10     \N
+11     \N
+12     
1|分隔符|0|分隔符|1|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0
+13     
1|分隔符|0|分隔符|1|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0|分隔符|0
+
+-- !export_set_36 --
+1      1#0#1#0#0
+2      1#0#1#0#0
+3      Y#0#Y#0#0
+4      1#0#1#0#0
+5      #0##0#0
+6      1#0#1#0#0
+7      1#0#1#0#0
+8      apache#0#apache#0#0
+9      1#0#1#0#0
+10     \N
+11     1#0#1#0#0
+12     1#0#1#0#0
+13     1#0#1#0#0
+
+-- !export_set_36 --
+1      
1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
+2      
0,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
+3      
Y,N,Y,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N,N
+4      
1,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0
+5      
,0,,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0
+6      ,1,1,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,
+7      
1,1,0,1,1,0,1,1,0,0,1,1,0,0,1,0,0,1,1,1,0,1,1,0,1,0,1,1,1,1,1,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0
+8      
apache,doris,doris,apache,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris,doris
+9      \N
+10     \N
+11     \N
+12     
1,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0
+13     
1,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0
+
+-- !export_set_37 --
+1      
1,0,0,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
+2      
1,0,0,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
+3      
Y,N,N,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y,Y
+4      
1,0,0,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
+5      ,0,0,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,,
+6      
1,,,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
+7      
1,0,0,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
+8      
apache,doris,doris,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache,apache
+9      
1,0,0,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
+10     \N
+11     \N
+12     
1,0,0,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
+13     
1,0,0,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1
+
+-- !export_set_38 --
+1      
0,1,0,0,1,0,1,0,1,1,1,1,1,1,0,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0
+2      0100101011111101100000000000000000000000000000000000000000000000
+3      
0,1,0,0,1,0,1,0,1,1,1,1,1,1,0,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0
+4      0100101011111101100000000000000000000000000000000000000000000000
+5      0100101011111101100000000000000000000000000000000000000000000000
+6      0100101011111101100000000000000000000000000000000000000000000000
+7      
0,1,0,0,1,0,1,0,1,1,1,1,1,1,0,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0
+8      
0|123|1|123|0|123|0|123|1|123|0|123|1|123|0|123|1|123|1|123|1|123|1|123|1|123|1|123|0|123|1|123|1|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0|123|0
+9      
0,1,0,0,1,0,1,0,1,1,1,1,1,1,0,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0
+10     0100101011111101100000000000000000000000000000000000000000000000
+11     
0,1,0,0,1,0,1,0,1,1,1,1,1,1,0,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0
+12     \N
+13     
0,1,0,0,1,0,1,0,1,1,1,1,1,1,0,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0
+
+-- !export_set_39 --
+1      
1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1
+2      
0世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1
+3      
Y世界!?你好0世界!?你好Y世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0
+4      
1世界!?你好0世界!?你好1世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0
+5      
世界!?你好0世界!?你好世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0
+6      
0世界!?你好1世界!?你好1世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0
+7      
1世界!?你好1世界!?你好0世界!?你好1世界!?你好1世界!?你好0世界!?你好1世界!?你好1世界!?你好0世界!?你好0世界!?你好1世界!?你好1世界!?你好0世界!?你好0世界!?你好1世界!?你好0世界!?你好0世界!?你好1世界!?你好1世界!?你好1世界!?你好0世界!?你好1世界!?你好1世界!?你好0世界!?你好1世界!?你好0世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好1世界!?你好0世界!?你好0世界!?你好0世界!?你好1世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0
+8      
apache世界!?你好0世界!?你好0世界!?你好apache世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0
+9      \N
+10     \N
+11     
1世界!?你好0世界!?你好1世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0
+12     
1世界!?你好0世界!?你好1世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0
+13     
1世界!?你好0世界!?你好1世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0
+
diff --git 
a/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy
 
b/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy
index 07122b80d7e..c5b36c56ec5 100644
--- 
a/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy
+++ 
b/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy
@@ -515,4 +515,103 @@ suite("test_string_function", "arrow_flight_sql") {
 
     sql """DROP TABLE IF EXISTS test_make_set;"""
 
+    // EXPORT_SET
+    sql """DROP TABLE IF EXISTS test_export_set;"""
+    sql """CREATE TABLE `test_export_set` (
+            `id` INT,
+            `bits` BIGINT,
+            `on` VARCHAR(255),
+            `off` VARCHAR(255),
+            `sep` VARCHAR(255),
+            `num_of_b` INT
+        )DUPLICATE KEY(id)
+        DISTRIBUTED BY HASH(id) BUCKETS 1
+        PROPERTIES ( 'replication_num' = '1' );"""
+    sql """INSERT INTO `test_export_set` VALUES
+            (1, -1, '1', '0', ',', 50),
+            (2, -2, '1', '0', '', 64),
+            (3, 5, 'Y', 'N', ',', 5),
+            (4, 5, '1', '0', '', 64),
+            (5, 5, '', '0', '', 65),
+            (6, 6, '1', '', '', 63),
+            (7, 19284249819, '1', '0', ',', 64),
+            (8, 9, 'apache', 'doris', '|123|', 64),
+            (9, NULL, '1', '0', ',', 5),
+            (10, 5, NULL, '0', '', 5),
+            (11, 5, '1', NULL, ',', 10),
+            (12, 5, '1', '0', NULL, 10),
+            (13, 5, '1', '0', ',', NULL);"""
+
+    qt_export_set_1 """SELECT id, EXPORT_SET(`bits`, `on`, `off`, `sep`, 
`num_of_b`) FROM `test_export_set` ORDER BY `id`;"""
+    qt_export_set_2 """SELECT EXPORT_SET(7, '1', '0');"""
+    qt_export_set_3 """SELECT EXPORT_SET(7, '你好', '0', '?');"""
+    qt_export_set_4 """SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 64), '1', '0');"""
+    qt_export_set_5 """SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 63) - 1, '1', 
'0');"""
+    qt_export_set_6 """SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 63), '1', '0');"""
+    qt_export_set_7 """SELECT EXPORT_SET(-BIT_SHIFT_LEFT(1, 63), '1', '0');"""
+    qt_export_set_8 """SELECT EXPORT_SET(-1, '1', '0');"""
+    qt_export_set_9 """SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 64) - 1, '1', 
'0');"""
+    qt_export_set_10 """SELECT EXPORT_SET(99999999999999999999, '1', '0');"""
+    qt_export_set_11 """SELECT EXPORT_SET((BIT_SHIFT_LEFT(1, 63) - 1) + 1, 
'1', '0');"""
+    qt_export_set_12 """SELECT EXPORT_SET(0, '1', '0');"""
+    qt_export_set_13 """SELECT EXPORT_SET(1, '1', '0');"""
+    qt_export_set_14 """SELECT EXPORT_SET(-0, '1', '0');"""
+    qt_export_set_15 """SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 62), '1', '0');"""
+    qt_export_set_16 """SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 62) + 
BIT_SHIFT_LEFT(1, 63), '1', '0');"""
+    qt_export_set_17 """SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 60), '1', '0');"""
+    qt_export_set_18 """SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 63) - 1, '1', '0', 
',', 32);"""
+    qt_export_set_19 """SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 63), '1', '0', 
',', 128);"""
+    qt_export_set_20 """SELECT EXPORT_SET(-1, '1', '0', ',', -5);"""
+    qt_export_set_21 """SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 60), '1', '0', 
',', 0);"""
+    qt_export_set_22 """SELECT EXPORT_SET(255, '1', '0', ',', 8);"""
+    qt_export_set_23 """SELECT EXPORT_SET(1023, '1', '0', ',', 10);"""
+    qt_export_set_24 """SELECT EXPORT_SET((BIT_SHIFT_LEFT(1, 63) - 2) + 1, 
'1', '0');"""
+    qt_export_set_25 """SELECT EXPORT_SET((BIT_SHIFT_LEFT(1, 63) - 1) * 2, 
'1', '0');"""
+    qt_export_set_26 """SELECT EXPORT_SET(18446744073709551616, '1', '0');""" 
// 2^64
+    qt_export_set_27 """SELECT EXPORT_SET(18446744073709551615, '1', '0');""" 
// 2^64 -1
+    qt_export_set_28 """SELECT EXPORT_SET(9223372036854775808, '1', '0');""" 
// 2^63
+    qt_export_set_29 """SELECT EXPORT_SET(1180591620717411303424, '1' 
,'0');""" // 2^70
+    qt_export_set_30 """SELECT EXPORT_SET(18446744073708551616, '1', '0');"""
+    qt_export_set_31 """SELECT EXPORT_SET(-9223372036854775808, '1', '0');"""
+    qt_export_set_32 """SELECT EXPORT_SET(-9223372036854775809, '1', '0');"""
+    qt_export_set_33 """SELECT EXPORT_SET(-9223372036854775807, '1', '0');"""
+    qt_export_set_34 """SELECT id, EXPORT_SET(`bits`, `on`, `off`, ' ! ', 
`num_of_b`) FROM `test_export_set` ORDER BY `id`;"""
+    qt_export_set_35 """SELECT id, EXPORT_SET(`bits`, `on`, `off`, '|分隔符|', 
'17') FROM `test_export_set` ORDER BY `id`;"""
+    qt_export_set_36 """SELECT id, EXPORT_SET(5, `on`, '0', '#', 5) FROM 
`test_export_set` ORDER BY `id`;"""
+    qt_export_set_36 """SELECT id, EXPORT_SET(`bits`, `on`, `off`) FROM 
`test_export_set` ORDER BY `id`;"""
+    qt_export_set_37 """SELECT id, EXPORT_SET(-7, `on`, `off`) FROM 
`test_export_set` ORDER BY `id`;"""
+    qt_export_set_38 """SELECT id, EXPORT_SET(114514, '1', '0', `sep`) FROM 
`test_export_set` ORDER BY `id`;"""
+    qt_export_set_39 """SELECT id, EXPORT_SET(`bits`, `on`, '0', '世界!?你好')FROM 
`test_export_set` ORDER BY `id`;"""
+    testFoldConst("SELECT EXPORT_SET(7, '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(7, '你好', '0', '?');")
+    testFoldConst("SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 64), '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 63) - 1, '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 63), '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(-BIT_SHIFT_LEFT(1, 63), '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(-1, '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 64) - 1, '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(99999999999999999999, '1', '0');")
+    testFoldConst("SELECT EXPORT_SET((BIT_SHIFT_LEFT(1, 63) - 1) + 1, '1', 
'0');")
+    testFoldConst("SELECT EXPORT_SET(0, '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(1, '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(-0, '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 62), '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 62) + BIT_SHIFT_LEFT(1, 
63), '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 60), '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 63) - 1, '1', '0', ',', 
32);")
+    testFoldConst("SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 63), '1', '0', ',', 
128);")
+    testFoldConst("SELECT EXPORT_SET(-1, '1', '0', ',', -5);")
+    testFoldConst("SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 60), '1', '0', ',', 
0);")
+    testFoldConst("SELECT EXPORT_SET(255, '1', '0', ',', 8);")
+    testFoldConst("SELECT EXPORT_SET(1023, '1', '0', ',', 10);")
+    testFoldConst("SELECT EXPORT_SET((BIT_SHIFT_LEFT(1, 63) - 2) + 1, '1', 
'0');")
+    testFoldConst("SELECT EXPORT_SET((BIT_SHIFT_LEFT(1, 63) - 1) * 2, '1', 
'0');")
+    testFoldConst("SELECT EXPORT_SET(18446744073709551616, '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(18446744073709551615, '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(9223372036854775808, '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(1180591620717411303424, '1' ,'0');")
+    testFoldConst("SELECT EXPORT_SET(18446744073708551616, '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(-9223372036854775808, '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(-9223372036854775809, '1', '0');")
+    testFoldConst("SELECT EXPORT_SET(-9223372036854775807, '1', '0');")
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to