Skip to content

Conversation

@Tan-JiaLiang
Copy link
Contributor

@Tan-JiaLiang Tan-JiaLiang commented Jan 2, 2025

Purpose

Currently, the bitmap index result has been supported pushdown in Parquet, We can merge the deletion vector and push down filtering.

Tests

org.apache.paimon.table.PrimaryKeyFileStoreTableTest#testDeletionVectorsCombineWithFileIndexPushDownParquet

API and Format

Documentation

@Tan-JiaLiang Tan-JiaLiang marked this pull request as draft January 2, 2025 02:33
@Tan-JiaLiang Tan-JiaLiang force-pushed the feat/parquet_dv_push_down branch from 0043055 to 3ba1d4b Compare January 2, 2025 03:08
@Tan-JiaLiang Tan-JiaLiang changed the title [WIP][parquet] Support deletion vector result pushdown [parquet] Support deletion vector result pushdown Jan 2, 2025
@Tan-JiaLiang Tan-JiaLiang marked this pull request as ready for review January 2, 2025 06:29
@JingsongLi
Copy link
Contributor

Can you share some benchmark on this? I'm a little worried about whether this optimization is useful for DV and whether it has any side effects.

@Tan-JiaLiang
Copy link
Contributor Author

Tan-JiaLiang commented Jan 3, 2025

@JingsongLi Sorry late. Your worried is reasonable, when DV result filters nothing, it incurs some performance overhead (the bitmap operation), maybe index have the same problem, because I only test the EQ predicate currently.

This is the benchmark code: https://github.com/Tan-JiaLiang/paimon/tree/benchmark/dv_push_down

I test in a 20 fields table, generate 1000000 rows per commit, commit 10 times, using the BSI index and DV to test the EQ predicate.

  • normal: dv and index disable push down.
  • dv-push-down: dv enable push down, and disable the index.
  • index-push-down: dv disable push down, and index enable.
  • dv-and-index-push-down: dv and index enable push down.

generate random value bound in [0, 10000000) for Primary key field, [0, 50000) for BSI index field, using a random value 27024 to test EQ predicate.

read Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative filter row groups filter row ranges
normal-10000000-50000-27024 18105 / 18197 165.7 6034.9 1.0X 0 0
dv-push-down-10000000-50000-27024 18577 / 18744 161.5 6192.4 1.0X 0 0
index-push-down-10000000-50000-27024 5272 / 5440 569.0 1757.3 3.4X 0 246
dv-and-index-push-down-10000000-50000-27024 5024 / 5052 597.1 1674.8 3.6X 0 260

generate random value bound in [0, 10000000) for Primary key field, [0, 100000) for BSI index field, using a random value 7748 to test EQ predicate.

read Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative filter row groups filter row ranges
normal-10000000-100000-7748 15763 / 15845 190.3 5254.5 1.0X 0 0
dv-push-down-10000000-100000-7748 16123 / 16160 186.1 5374.2 1.0X 0 0
index-push-down-10000000-100000-7748 2413 / 2443 1243.3 804.3 6.5X 0 310
dv-and-index-push-down-10000000-100000-7748 2163 / 2190 1386.9 721.0 7.3X 0 318

generate random value bound in [0, 10000000) for Primary key field, [0, 300000) for BSI index field, using a random value 288225 to test EQ predicate.

read Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative filter row groups filter row ranges
normal-10000000-300000-288225 15781 / 15836 190.1 5260.3 1.0X 0 0
dv-push-down-10000000-300000-288225 16102 / 16199 186.3 5367.5 1.0X 0 0
index-push-down-10000000-300000-288225 1623 / 1627 1848.8 540.9 9.7X 0 329
dv-and-index-push-down-10000000-300000-288225 1418 / 1484 2116.3 472.5 11.1X 0 334

generate random value bound in [0, 10000000) for Primary key field, [0, 800000) for BSI index field, using a random value 788897 to test EQ predicate.

read Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative filter row groups filter row ranges
normal-10000000-800000-788897 15685 / 15760 191.3 5228.3 1.0X 0 0
dv-push-down-10000000-800000-788897 16049 / 16413 186.9 5349.8 1.0X 0 0
index-push-down-10000000-800000-788897 632 / 676 4744.8 210.8 24.8X 0 263
dv-and-index-push-down-10000000-800000-788897 679 / 696 4420.2 226.2 23.1X 0 263

@Tan-JiaLiang
Copy link
Contributor Author

Maybe add RoaringBitmap#contains(long, long) to do fast skip is better, I will benchmark it again.

@Tan-JiaLiang
Copy link
Contributor Author

I used the same data as above for testing after upgrade the RoaringBitmap version to 1.2.1.

generate random value bound in [0, 10000000) for Primary key field, [0, 50000) for BSI index field, using a random value 27024 to test EQ predicate.

read Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative filter row groups filter row ranges
normal-10000000-50000-27024 16717 / 17656 179.5 5572.4 1.0X 0 0
dv-push-down-10000000-50000-27024 17083 / 17500 175.6 5694.2 1.0X 0 0
index-push-down-10000000-50000-27024 4623 / 4771 649.0 1540.9 3.6X 0 246
dv-and-index-push-down-10000000-50000-27024 4227 / 4353 709.8 1408.9 4.0X 0 260

generate random value bound in [0, 10000000) for Primary key field, [0, 100000) for BSI index field, using a random value 7748 to test EQ predicate.

read Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative filter row groups filter row ranges
normal-10000000-100000-7748 16024 / 16530 187.2 5341.3 1.0X 0 0
dv-push-down-10000000-100000-7748 15695 / 16090 191.1 5231.7 1.0X 0 0
index-push-down-10000000-100000-7748 2958 / 3298 1014.1 986.1 5.4X 0 310
dv-and-index-push-down-10000000-100000-7748 2738 / 2756 1095.8 912.6 5.9X 0 318

generate random value bound in [0, 10000000) for Primary key field, [0, 300000) for BSI index field, using a random value 288225 to test EQ predicate.

read Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative filter row groups filter row ranges
normal-10000000-300000-288225 16367 / 16460 183.3 5455.8 1.0X 0 0
dv-push-down-10000000-300000-288225 16547 / 16852 181.3 5515.7 1.0X 0 0
index-push-down-10000000-300000-288225 2068 / 2092 1450.7 689.3 7.9X 0 329
dv-and-index-push-down-10000000-300000-288225 1726 / 1773 1737.9 575.4 9.5X 0 334

generate random value bound in [0, 10000000) for Primary key field, [0, 800000) for BSI index field, using a random value 788897 to test EQ predicate.

read Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative filter row groups filter row ranges
normal-10000000-800000-788897 16168 / 16287 185.6 5389.3 1.0X 0 0
dv-push-down-10000000-800000-788897 16020 / 16591 187.3 5340.0 1.0X 0 0
index-push-down-10000000-800000-788897 819 / 857 3662.2 273.1 19.7X 0 263
dv-and-index-push-down-10000000-800000-788897 788 / 1123 3806.2 262.7 20.5X 0 263

@JingsongLi
Copy link
Contributor

Hi @JingsongLi ,I found some bugs in RoaringBitmap 1.0.5 version when I'm benchmark. Will you consider upgrade the roaringbitmap version?

It is OK to upgrade it.

@Tan-JiaLiang
Copy link
Contributor Author

Tan-JiaLiang commented Jan 3, 2025

I did another benchmark based on the code submitted so far. I came to some conclusions: (Please correct me if something wrong)

  1. DV push down itself is hard to filtering some row ranges because it is sparse and level compaction may do when writing.
  2. Based on the current benchmark, support the DV push down should not cause much performance penalty.
  3. DV push down combine with index push down can give better performance.

generate random value bound in [0, 10000000) for Primary key field, [0, 50000) for BSI index field, using a random value 27024 to test EQ predicate.

read Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative filter row groups filter row ranges
normal-10000000-50000-27024 15584 / 15684 192.5 5194.7 1.0X 0 0
dv-push-down-10000000-50000-27024 15577 / 15746 192.6 5192.3 1.0X 0 0
index-push-down-10000000-50000-27024 4662 / 4745 643.5 1554.0 3.3X 0 246
dv-and-index-push-down-10000000-50000-27024 4088 / 4129 733.9 1362.7 3.8X 0 260

generate random value bound in [0, 10000000) for Primary key field, [0, 100000) for BSI index field, using a random value 7748 to test EQ predicate.

read Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative filter row groups filter row ranges
normal-10000000-100000-7748 16040 / 16864 187.0 5346.7 1.0X 0 0
dv-push-down-10000000-100000-7748 16389 / 16792 183.1 5462.9 1.0X 0 0
index-push-down-10000000-100000-7748 2564 / 2616 1170.0 854.7 6.3X 0 310
dv-and-index-push-down-10000000-100000-7748 2234 / 2281 1343.0 744.6 7.2X 0 318

generate random value bound in [0, 10000000) for Primary key field, [0, 300000) for BSI index field, using a random value 288225 to test EQ predicate.

read Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative filter row groups filter row ranges
normal-10000000-300000-288225 15347 / 15494 195.5 5115.8 1.0X 0 0
dv-push-down-10000000-300000-288225 15265 / 15435 196.5 5088.2 1.0X 0 0
index-push-down-10000000-300000-288225 1681 / 1702 1784.3 560.5 9.1X 0 329
dv-and-index-push-down-10000000-300000-288225 1533 / 1551 1956.4 511.1 10.0X 0 334

generate random value bound in [0, 10000000) for Primary key field, [0, 800000) for BSI index field, using a random value 788897 to test EQ predicate.

read Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative filter row groups filter row ranges
normal-10000000-800000-788897 15562 / 15574 192.8 5187.2 1.0X 0 0
dv-push-down-10000000-800000-788897 15309 / 15542 196.0 5103.0 1.0X 0 0
index-push-down-10000000-800000-788897 745 / 758 4026.9 248.3 20.9X 0 263
dv-and-index-push-down-10000000-800000-788897 742 / 750 4043.2 247.3 21.0X 0 263

@ranxianglei
Copy link
Contributor

Hi,can you help find which file the benchmark code: https://github.com/Tan-JiaLiang/paimon/tree/benchmark/dv_push_down ? I have review that results, but maybe slow than expexct ,could you check this issue #4586 ?

chinese: 我不知道你是不是中国人,会不会中文,但是我英文非常差,打上面英文字怕你看不懂,特别中文描述下。
是否有benchmark具体的代码位置?在#4586 中,修复过一些问题已经把查询性能提升了很多了,我理解在1千万行数据查询主键,即时查询1000次也仅仅花费4s左右,平均每一次都是10ms以下的。你给你这个测试和我们之前测试的结果差太大了,可以给下测试代码吗?我帮你看下

@Tan-JiaLiang

@Tan-JiaLiang
Copy link
Contributor Author

Hi, @ranxianglei I'm glad you're willing to help.

You can find the benchmark code here and If something wrong, please feel free to ask.
https://github.com/Tan-JiaLiang/paimon/blob/benchmark/dv_push_down/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/deletionvectors/DeletionVectorsWithIndexResultPushDownBenchmark.java

Actually I did not test the EQ predicate with the primary key, I was testing the EQ predicate with another field, test benchmark with or without index and deletion vector push down.

In order to minimise errors, the benchmark will run several iterations, and every single iteration will run query in many times, so you can see that it takes a long time.

@ranxianglei
Copy link
Contributor

ranxianglei commented Jan 5, 2025

I understand that if the filter condition is pushed down to orc or Parquet, good performance can be obtained without using index push down. It happens that the dv table can do push-down of non-primary keys. I tested Orc's dv table pushdown. The size of the data generated was similar to yours. On average, a non-primary key query took about 100 ms. I tested it locally and the hard disk was SSD. Although it is much slower than the gradual query which takes 10ms, the performance is not bad.

我理解如果把filter条件下推到orc或者Parquet,不需要使用index的下推也可以获得很好的性能。恰巧dv表可以做非主键的下推。我测试了orc的dv表下推,生成的数据规模和你的类似,平均一个非主键查询的耗时大约100多ms,本机测试,硬盘是ssd。虽然比逐渐查询耗时10ms慢很多,但是性能还算可以。当然性能还可以优化,这个case应该也可以做到10ms左右。

如果你想复现需要暂时把 && !deletionVectorsEnabled
&& !(fileIndexResult instanceof BitmapIndexResult 删去,因为这里一个bug尚未修复,但是不影响测试性能。

The reason why this result occurs is that before non-primary key pushdown is enabled in the dv table, the filter condition cannot be applied to the orc index. After the index takes effect, the number of filtered row groups is naturally reduced. I don’t know much about Parquet. Maybe there is still something wrong with Parquet implementation. You can try it yourself.

之所以会出现这个结果是因为dv表在未开启非主键下推之前,filter条件无法作用到orc的索引上,索引起作用后,筛选后的row group数量自然少了。所以实际上不需要用index下推,仅仅用orc或者Parquet自带的稀疏索引或者bloom过滤器可以获得很好的性能,这里要优化的点或许是每次扫描group 元数据过多的问题。

Parquet我不太懂,或许Parquet实现还是有问题,你可以自己试试。

schema

{
    "version": 2,
    "id": 0,
    "fields": [
        {
            "id": 0,
            "name": "f0",
            "type": "INT NOT NULL"
        },
        {
            "id": 1,
            "name": "f1",
            "type": "INT NOT NULL"
        },
        {
            "id": 2,
            "name": "f2",
            "type": "STRING"
        },
        {
            "id": 3,
            "name": "f3",
            "type": "STRING"
        },
        {
            "id": 4,
            "name": "f4",
            "type": "STRING"
        },
        {
            "id": 5,
            "name": "f5",
            "type": "STRING"
        },
        {
            "id": 6,
            "name": "f6",
            "type": "STRING"
        },
        {
            "id": 7,
            "name": "f7",
            "type": "STRING"
        },
        {
            "id": 8,
            "name": "f8",
            "type": "STRING"
        },
        {
            "id": 9,
            "name": "f9",
            "type": "STRING"
        },
        {
            "id": 10,
            "name": "f10",
            "type": "STRING"
        },
        {
            "id": 11,
            "name": "f11",
            "type": "STRING"
        },
        {
            "id": 12,
            "name": "f12",
            "type": "STRING"
        },
        {
            "id": 13,
            "name": "f13",
            "type": "STRING"
        },
        {
            "id": 14,
            "name": "f14",
            "type": "STRING"
        },
        {
            "id": 15,
            "name": "f15",
            "type": "STRING"
        },
        {
            "id": 16,
            "name": "f16",
            "type": "STRING"
        },
        {
            "id": 17,
            "name": "f17",
            "type": "STRING"
        },
        {
            "id": 18,
            "name": "f18",
            "type": "STRING"
        },
        {
            "id": 19,
            "name": "f19",
            "type": "STRING"
        },
        {
            "id": 20,
            "name": "f20",
            "type": "STRING"
        }
    ],
    "highestFieldId": 2,
    "partitionKeys": [],
    "primaryKeys": [
        "f0"
    ],
    "options": {
        "bucket": "1",
        "file.format": "orc",
        "orc.filter.use.selected": "true",
        "orc.sarg.to.filter": "true",
        "target-file-size": "1024 MB",
        "file.block-size": "8 MB",
        "deletion-vectors.enabled": "true"
    },
    "timeMillis": 1731654078602
}

java code

import org.apache.paimon.data.BinaryString;
import org.apache.paimon.data.GenericRow;
import org.apache.paimon.data.InternalRow;
import org.apache.paimon.disk.IOManagerImpl;
import org.apache.paimon.predicate.Predicate;
import org.apache.paimon.predicate.PredicateBuilder;
import org.apache.paimon.reader.RecordReader;
import org.apache.paimon.table.Table;
import org.apache.paimon.table.sink.BatchTableCommit;
import org.apache.paimon.table.sink.BatchTableWrite;
import org.apache.paimon.table.sink.BatchWriteBuilder;
import org.apache.paimon.table.sink.CommitMessage;
import org.apache.paimon.table.source.InnerTableRead;
import org.apache.paimon.table.source.InnerTableScan;
import org.apache.paimon.table.source.ReadBuilder;
import org.apache.paimon.table.source.Split;
import org.apache.paimon.table.source.TableScan;
import org.apache.paimon.types.DataTypes;
import org.apache.paimon.types.RowType;
import org.junit.Test;

import java.util.List;
import java.util.Random;
import java.util.UUID;

public class SpeedOrcLookupNoPrimaryKey {
    @Test
    public void testWriter() throws Exception {
        writer();
    }
    @Test
    public void testReader() throws Exception {
        reader();
    }
    private void reader() throws Exception {
        Table table =  // getTable();
        PredicateBuilder builder = new PredicateBuilder(
                RowType.of(DataTypes.INT(),DataTypes.INT(),
                        DataTypes.STRING(),
                        DataTypes.STRING()));

        int[] projection = new int[] {0, 1, 2,3};

        ReadBuilder readBuilder = table.newReadBuilder()
                .withProjection(projection);

        Random random = new Random();
        for(int i = 0 ; i < 30 ; i ++){
            InnerTableRead read = (InnerTableRead)readBuilder.newRead();
            int key = random.nextInt(4000000);

            Predicate keyFilter = builder.equal(1, 2903308);

            InnerTableScan tableScan = (InnerTableScan) readBuilder
                    .withFilter(keyFilter)
                    .newScan();
            InnerTableScan innerTableScan = tableScan.withFilter(keyFilter);
            TableScan.Plan plan = innerTableScan.plan();
            List<Split> splits = plan.splits();

            read.withFilter(keyFilter);//.executeFilter();
            RecordReader<InternalRow> reader = read.createReader(splits);

            reader.forEachRemaining(internalRow -> {

                int f0 = internalRow.getInt(0);
                int f1 = internalRow.getInt(1);
                String f2 = internalRow.getString(2).toString();
                String f3 = internalRow.getString(3).toString();
            });
        }
        long startTime = System.currentTimeMillis();

        for(int i = 0 ; i < 1000 ; i ++){
            InnerTableRead read = (InnerTableRead)readBuilder.newRead();
            int key = random.nextInt(4000000);

            Predicate keyFilter = builder.equal(1, key);

            InnerTableScan tableScan = (InnerTableScan) readBuilder
                    .withFilter(keyFilter)
                    .newScan();
            InnerTableScan innerTableScan = tableScan.withFilter(keyFilter);
            TableScan.Plan plan = innerTableScan.plan();
            List<Split> splits = plan.splits();

            read.withFilter(keyFilter);//.executeFilter();
            RecordReader<InternalRow> reader = read.createReader(splits);

            reader.forEachRemaining(internalRow -> {

                int f0 = internalRow.getInt(0);
                int f1 = internalRow.getInt(1);
                String f2 = internalRow.getString(2).toString();
                String f3 = internalRow.getString(3).toString();
            });
        }
        long stopTime = System.currentTimeMillis();
        System.out.println("time : " + (stopTime - startTime));
    }
    private void writer() throws Exception {
        Table table =  // getTable();

        BatchWriteBuilder writeBuilder = table.newBatchWriteBuilder()
                ;
        BatchTableWrite write = writeBuilder
                .newWrite() .withIOManager(new IOManagerImpl("./tmp")).asBatchTableWrite();  // TableWriteImpl

        Random random = new Random();

        long startTime = System.currentTimeMillis();

        for(int i = 0; i < 1000_0000; i++){

            GenericRow genericRow = GenericRow.of(
                    i,
                    random.nextInt(1000_0000),
                    BinaryString.fromString(String.valueOf(random.nextInt())),
                    BinaryString.fromString(UUID.randomUUID().toString())
                    ,BinaryString.fromString(UUID.randomUUID().toString())
                    ,BinaryString.fromString(UUID.randomUUID().toString())
                    ,BinaryString.fromString(UUID.randomUUID().toString())
                    ,BinaryString.fromString(UUID.randomUUID().toString())
                    ,BinaryString.fromString(UUID.randomUUID().toString())
                    ,BinaryString.fromString(UUID.randomUUID().toString())
                    ,BinaryString.fromString(UUID.randomUUID().toString())
                    ,BinaryString.fromString(UUID.randomUUID().toString())
                    ,BinaryString.fromString(UUID.randomUUID().toString())
                    ,BinaryString.fromString(UUID.randomUUID().toString())
                    ,BinaryString.fromString(UUID.randomUUID().toString())
                    ,BinaryString.fromString(UUID.randomUUID().toString())
                    ,BinaryString.fromString(UUID.randomUUID().toString())
                    ,BinaryString.fromString(UUID.randomUUID().toString())
                    ,BinaryString.fromString(UUID.randomUUID().toString())
                    ,BinaryString.fromString(UUID.randomUUID().toString())
                    ,BinaryString.fromString(UUID.randomUUID().toString())
            );
            write.write(genericRow);
        }

        List<CommitMessage> messages = write.prepareCommit();

        BatchTableCommit commit = writeBuilder.newCommit();

        commit.commit(messages);

        long stopTime = System.currentTimeMillis();

        System.out.println("time : " + (stopTime - startTime));
    }
}

@Tan-JiaLiang

@JingsongLi
Copy link
Contributor

@Tan-JiaLiang Very thanks for your detailed benchmark!

@Tan-JiaLiang
Copy link
Contributor Author

Hi, @ranxianglei.
The min/max index combined with the page-grained bloom-filter is very useful in the case of EQ predicates. But it is only useful in the EQ predicate case.
More importantly, I wanted to explore the ability to use some indices to optimise other predicates, such as the Range and the TopK search.

Using EQ as the benchmark predicate is just because of I think it is simple and it can more clearly compare the effect and side effect after push down.

@ranxianglei
Copy link
Contributor

min max can also be used for ranges, not just eq. In addition, queries outside the sorting optimization range may not achieve good results. It is recommended to first check whether there is still a problem with Parquet's pushdown implementation. The benefits of fixing a Parquet pushdown problem may be much greater than adding an index. @Tan-JiaLiang

@Tan-JiaLiang
Copy link
Contributor Author

Hi, @ranxianglei

The benefits of fixing a Parquet pushdown problem may be much greater than adding an index.

I quite agree with this view. But I do not see of any room for improvement temporarily, since it's already optimised for page-level filtering with the zonemap index. But if there is, I'd love to look into it.

min max can also be used for ranges, not just eq.

Yes, min/max can do filtering in range case, but the bloom-filter can not. The min/max index is always has the false-positive and it is hard to get good results without sort compact. And the sort compact can not sort on too many fields.

More: the difference between the bitmap index and the zone map index (min/max index)

The bitmap category index is the position index and it does not have the false-positive problem and can do more in other predicate case. However, it is larger than the zonemap index, and loading and evaluation is slower than the zonemap index.

The main of this PR is to combine the selection and the deletion result and push it down and do the page filtering for better performance.

@Tan-JiaLiang
Copy link
Contributor Author

Because of only push down the deletion vector result does not work in the benchmark, this PR will change to merge the file index and deletion vector and push down. cc @JingsongLi

@Tan-JiaLiang Tan-JiaLiang changed the title [parquet] Support deletion vector result pushdown [parquet] Merge the file index and the deletion vector and push down filtering Jan 6, 2025
Copy link
Contributor Author

@Tan-JiaLiang Tan-JiaLiang left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@JingsongLi Can you please take a look again? If something wrong please feel free to say.

This reverts commit 918a377.
This reverts commit 9444900.
@JingsongLi
Copy link
Contributor

+1

@JingsongLi JingsongLi merged commit 3dab1b8 into apache:master Jan 6, 2025
12 checks passed
@Tan-JiaLiang Tan-JiaLiang deleted the feat/parquet_dv_push_down branch June 25, 2025 02:46
tub pushed a commit to tub/paimon that referenced this pull request Jan 13, 2026
* [core] Add parquet write page limit parameter (apache#4632)

* [flink] Fix that 'values-time' partition expire might throw NPE (apache#4646)

* [spark] Support show create and describe view (apache#4645)

* [test][spark] Add insert with column list test case (apache#4654)

* [core] Expire partiitons add default delete num (apache#4652)

* [flink] Optimizing parallelism for fixed bucekt and non-partitioned table (apache#4643)

* [flink] support flink sourceIdleTime metric in ReadOperator (apache#4644)

* [parquet] Fix minor format codes in parquet readers

* [core] Optimize memory usage for expiring snapshots and tags (apache#4655)

This closes apache#4655.

* [spark] Paimon parser only resolve own supported procedures (apache#4662)

* [core] Optimize overwrite commit to use CommitResult to retry (apache#4661)

This closes apache#4661.

* [core] Optimize drop partitions to avoid stack overflow (apache#4663)

This closes apache#4663.

* [hive] Make HiveMetastoreClient.addPartition thread safe (apache#4669)

* [core] Support auth in REST Catalog (apache#4648)

* [flink-cdc] kafka_sync_database supports table name mapping when prefix and postfix could not fit the need. (apache#4660)

* [minor] Fix RESTCatalog.close should close client and refreshExecutor

* [core] Introduce timeout for commit retry avoid long time loop (apache#4668)

* [core] Introduce data-file.path-directory (apache#4672)

* [core] Trim key field in reading, map it to value field (apache#4651)

This closes apache#4651.

---------

Co-authored-by: tsreaper <tsreaper96@gmail.com>

* [hive][spark] Support creating external table without schema when the table already exists (apache#4638)

* [hotfix] Fix flaky test of orc tests in ArrowBatchConverterTest (apache#4673)

* [orc] Row group filter push down cannot work with bitmap index

* [docs] add glue metastore repo for iceberg compability (apache#4674)

* [core] Improve fault tolerance for data spill to disk. (apache#4675)

* [doc] updated url links in documentation (apache#4679)

* [hotfix] CloneAction throw more clear exception when no table in source catalog (apache#4682)

* [core] Check file size after write bundle of records (apache#4685)

* [doc] Fix links in sql-write

* [core] Introduce RollbackToWatermarkProcedure for rollback (apache#4687)

* [core] Add database API implementation in RESTCatalog (apache#4676)

* [spark] Fix the build of read type in binlog table (apache#4689)

* [core] Clean up invalid branch cache and not cache system table in caching catalog (apache#4681)

* [core] Remove useless codes in CachingCatalog

* [core] Rename BulkFormatMapping to FormatReaderMapping

* [core] Rename BulkFormatMappingBuilder to FormatReaderMapping.Builder

* [spark] Avoid explicitly creating catalog in PaimonTableValuedFunctions (apache#4690)

* [core] Update drop Database API and remove api in URL (apache#4691)

* [doc] update totalRecordCount and deltaRecordCount in understand-files.md (apache#4694)

* [core] fix the issue where streaming reading of overwrite data would fail when retract type data appeared. (apache#4697)

* [core] Introduce data-file.thin-mode in primary key table write (apache#4666)

This closes apache#4666.

* [core] Retry if snapshot commit hint failed. (apache#4701)

* [flink] Replace legacy SourceFunction with v2 Source (apache#4614)

This closes apache#4614.

* [core] Store `cardinality` in the deletion vector meta (apache#4699)

* [flink] kafka_sync_database supports different prefix and suffix for different db (apache#4704)

* [core] Tolerate the NoSuchObjectException when report the partition statistic (apache#4708)

* [core][spark] Enable limit pushdown and count optimization for dv table (apache#4709)

* [flink] avoid using 'newHashMapWithExpectedSize' which is internal in flink (apache#4713)

* [flink] Enable limit pushdown and count optimization for dv table (apache#4712)

* [doc] Specific true column names for manifest files

* [spark] Fix relativeBucketPath with data-file.path-directory (apache#4714)

* [core] Introduce PurgeFilesProcedure to purge table (apache#4717)

* [core] Remove Catalog.getTableLocation interface (apache#4718)

* [core][spark] Fix create external table with schema evolution (apache#4719)

* [orc] Optimize configuration creating in orc file format (apache#4716)

* [core] Clean constants, caseSensitive, loader in Catalog (apache#4721)

* [pom] prefer central repo for releases; limit apache-snapshots to snapshots (apache#4707)

* [core] Refactor MetastoreClient methods to simplify catalog (apache#4726)

* [core] Support alter database (apache#4700)

* [core] Introduce Variant Data (apache#4729)

* [flink] kafka_sync_database supports db whitelist and blacklist (apache#4732)

* [core] Introduce CacheStats and expose ScanStats (apache#4678)

* [core] Minor refactor for cache metrics

* [cdc] Correct equalsIgnoreFieldId in UpdatedDataFieldsProcessFunctionBase

* [cdc] add exception message for CdcRecordStoreMultiWriteOperator (apache#4734)

* [core] Fix predicate literals cast in filter pushdown after schema evolution (apache#4705)

* [hive] Fix sync hms partition with data-file.path-directory (apache#4735)

* [flink] make warehouse in Flink action optional (apache#4728)

* [core] Skip case checking in catalog (apache#4730)

* [hive] Batch list tables and skip checking table exists in filesystem with hive catalog (apache#4737)

* [hotfix] Remove unused SchemaEvolutionUtil methods (apache#4739)

* [doc] Add doc for spark drop column with hive catalog

* [dependency] Upgrade paimon shade version (apache#4740)

* [doc] Document hive.metastore.disallow.incompatible.col.type.changes for Flink and Spark client

* [spark] Fix load function with SparkGenericCatalog (apache#4741)

* [flink] Support nested projection pushdown (apache#4667)

This closes apache#4667.

* [spark] Fix delete with partial non-convertible partition filter (apache#4738)

* [hive] Ignore path comparison when creating external table (apache#4747)

* [Parquet] Revert parquet patch apache#3883 which tries to construct column vectors like orc (apache#4745)

* [parquet] parquet reader should not retrun VectorizedRowIterator for nested schema (apache#4749)

* [core] Add _EXTERNAL_PATH in DataFileMeta

This closes apache#4751

* [hotfix] Fix typo in NestedColumnReader and NestedPrimitiveColumnReader (apache#4752)

Co-authored-by: yuzelin <zelin.yzl@alibaba-inc.com>

* [core] fix parquet can not read row with last column is array. (apache#4755)

* [core] Fix serializer error in 'Add _EXTERNAL_PATH in DataFileMeta'

* [test] Add current version compatibility tests for manifest committable and split

* [flink][cdc] Add support for retry cnt instead of busy wait and additional support to skip corrupt records in cdc writer (apache#4295)

* [spark] Make show table extended compatible with spark3.4- (apache#4754)

* [hotfix] Rename cdc schema change options in CdcRecordStoreWriteOperator (apache#4756)

* [format] support parquet reader reading field with 'FIXED_LEN_BYTE_ARRAY' type (apache#4759)

* [release] Update version to 1.1-SNAPSHOT

* [core] Introduce VariantType (apache#4757)

* [spark] Integrate Variant with Spark4 (apache#4764)

* [core] Support read external path in DataFileMeta (apache#4761)

* [core] External Path in DataFileMeta should be the file path (apache#4766)

* [hotfix] Add spark3 profile in deploy_staging_jars (apache#4768)

* [core] Support Table API in RESTCatalog (apache#4736)

* [core] optimize the binlog table read performance (apache#4773)

* [rest] Fix GetTableResponse should return fields (apache#4769)

* [flink] Introduce max_two_pt for Flink lookup join (apache#4772)

* [doc] Add max_two_pt to 'lookup.dynamic-partition' option

* [docs] Fix typo in docs/content/pk-table&append-table (apache#4776)

* [doc] Add EXTERNAL_PATH to manifest doc

* [core] SortLookupStoreWriter should support empty record (apache#4777)

* [core] Support alter table API (apache#4775)

* [core] Optimize fileFormat discovery and avoid creating fileFormat (apache#4782)

* [spark] Fix writing null struct col (apache#4787)

* [docs] Use `config.yaml` for flink version >= 1.19 (apache#4784)

* [core] Add check of older_than when RemoveOrphanFiles (apache#4779)

* [core] Support customized tag name in Batch TagCreationMode (apache#4778)

* [hotfix] Fix invalid link in Partial Update doc (apache#4789)

* [core] Support to query indexes (apache#4788)

* [common] Using a faster deserialization method in RoaringBitmap32 (apache#4765)

* [parquet] Support using file index result to filter row ranges (apache#4780)

* [doc] Fix incorrect header level in manage-branches.md

* [core] Add schema validation for record-level.time-field (apache#4758)

* [test] Remove the loggingTimer to fix the CDC build error (apache#4799)

* [hive] Fix listTablesImpl possible timeout issue (apache#4800)

* [core] Support partition API (apache#4786)

* [rest] Partition methods should check table first in RESTCatalog

* [rest] Remove useless fetchOptionsFromServer in RESTCatalog

* [doc] Add a note to the delete branch operation to explain its behavior (apache#4803)

* [parquet] Fix file index result filter the row ranges missing rowgroup offset problem (apache#4806)

* [core] Introduce Partition to Catalog.listPartitions (apache#4807)

* [spark] Purge file need refresh table avoid FileNotFound (apache#4809)

* [iceberg] Add Iceberg database / table options as alias when generating Iceberg metadata (apache#4811)

* [iceberg] Add iceberg options to docs

* [spark] Bump Spark 3.5.4 (apache#4819)

* [hotfix] Fix flaky test testCleanWithBranch (apache#4826)

* [spark] Remove unreasonable test output (apache#4825)

* [core] Refactor TagManager to remove unnecessary tag existence check (apache#4820)

* [test] Disable PrimaryKeyFileStoreTableITCase.testFullCompactionChangelogProducerStreamingRandom

* [hotfix] Fix typo of create_tag procedure doc (apache#4829)

* [doc] Add release 1.0 doc

* [common] Upgrade the RoaringBitmap version from 1.0.5 to 1.2.1 (apache#4832)

* [doc] Modify master to 1.1-SNAPSHOT

* [spark] SparkGenericCatalog support tag DDL (apache#4833)

* [hotfix] Refactor RemoveOrphanFilesActionITCase in flink-1.18 to avoid flaky test (apache#4840)

* [core] overwrite should be true when commit change log (apache#4838)

* [core] remove nullable check for record-level.time-field (apache#4837)

* [core] support write to the external path (apache#4770)

* [core] Refactor ExternalPathProvider to generate file in DataFilePathFactory

* [parquet] Merge the file index and the deletion vector and push down filtering (apache#4812)

* [core] Support customize action for partition mark done. (apache#4817)

* [doc] Improving the mysql-cdc.md usage Documentation (apache#4792)

* [core] Introduce incremental-to-auto-tag for reading changes between auto tags and let incremental-between be tag-first (apache#4808)

* [core] support null value for record-level.time-field (apache#4839)

* [flink][action] add '`' to the fields of merge into action to avoid exceptions when the field name is an SQL keyword. (apache#4846)

* [core] SnapshotReaderImpl.toChangesPlan should use snapshot for tags (apache#4841)

* [doc] Fix names in fileindex spec

* [doc] Add 2025 on NOTICE (apache#4850)

* [core] Introduce nested type cast to string (apache#4848)

* [core] Use cast to string to show partition in system table (apache#4852)

* [flink] Fix the refresh executor not work after reopen (apache#4851)

* [core] add SuccessFileTagCallback for tag creation (apache#4847)

* [spark] Introduce paimon_incremental_between_timestamp and paimon_incremental_to_auto_tag tvf (apache#4855)

* [refactor] Refactor LookupJoin: add logs for troubleshoot and polish DynamicPartitionLoader (apache#4828)

* [hotfix][doc] Fix some typo and add a three-level catalogue (apache#4858)

* [core] Support auto create tag with custom duration (apache#4857)

* [flink] Flink batch delete supports partial-update.remove-record-on-sequence-group option (apache#4861)

* [core] Introduce batch partition methods in catalog (apache#4866)

* [core] Fix that incremental-to-auto-tag return wrong result if snapshots are expired (apache#4869)

* [core] Fix retract behavior for FieldProductAgg when accumulator is null (apache#4842)

* [core] Introduce catalog loader in Catalog to replace MetastoreClient (apache#4874)

* [iceberg] Introduce feature to migrate table from iceberg to paimon (apache#4639)

This closes apache#4639.

* [core] Add UT for expire partitions with hms config metastore.partitioned-table  (apache#4875)

* [core] Fix remove orphan files with data file path directory (apache#4871)

* [flink] Add Tests and ITCases in flink for RESTCatalog (apache#4805)

* [core] Introduce PrivilegeManagerLoader to avoid hardcode FileBasedPrivilegeManager (apache#4877)

* [core] Support parsing row type json without field id (apache#4876)

* [log] logging specific join keys and results  for lookup join (apache#4856)

* [doc] Update metadata url for mysql cdc

* [test] remove incorrect and confusing tests in PartialUpdateMergeFunctionTest (apache#4882)

* [e2e] upgrade the flink-sql-connector-kafka version (apache#4881)

* [flink] Supports debezium-bson formats of kafka data which collected from mongodb via debezium (apache#4870)

* [rest] Supports global system tables (apache#4880)

* [core] Do not use Collections.unmodifiableMap to avoid stack overflow (apache#4883)

* [hotfix] remove the useless comment (apache#4889)

* [core] Fix invalidate tables with same tableNames in other db issue (apache#4895)

* [core] Add a http-report action to reporting partition done to remote servers. (apache#4862)

* [rest] Add name to GetTableResponse and remove path (apache#4894)

* [hotfix] fix some typos in CombinedTableCompactorSink (apache#4890)

* [core] Introduce conversion from parquet type to paimon type (apache#4888)

* [spark] Clean empty directory after removing orphan files (apache#4824)

* [docs] update link about Apache Doris (apache#4898)

* [hotfix] fix Trino version to 440 (apache#4896)

* [flink] Add a action/procedure to remove unexisting files from manifests (apache#4781)

* [core] Support partition API and update get table (apache#4879)

* [core] Extract loadTable in CatalogUtils (apache#4904)

* [flink] Introduce precommit compact for newly created files in unaware bucket tables (apache#4902)

* [doc] Add doc for precommit-compact

* [core] Fix that sequence group fields are mistakenly aggregated by default aggregator in partial update (apache#4897)

* [core] Introduce SnapshotCommit to abstract atomically commit (apache#4911)

* [core] Support view API in Rest catalog (apache#4908)

* [hotfix][doc] fix the url link in document (apache#4914)

* [core] Add min_partition_stats and max_partition_stats columns to manifests system table (apache#4922)

* [hotfix] Modify the type conversion method (apache#4928)

* [test] Fix the unstable random tests in PrimaryKeyFileStoreTableITCase (apache#4933)

* [test] Fix the unstable testCloneWithSchemaEvolution (apache#4932)

* [hotfix][doc] Add quotes to the partition (apache#4931)

* [core] Remove unnecessary toString call in `DataFilePathFactoryTest` (apache#4924)

* [core] Make FileIOLoader extends Serializable

* [spark] Introduce SparkV2FilterConverter (apache#4915)

* [rest] Add http conf and ExponentialHttpRetryInterceptor to handle retry In RESTCatalog (apache#4929)

* [common] A FileIO API to list files iteratively (apache#4834)

* [core] Make CatalogContext implements Serializable (apache#4936)

* [core] Refactor the orphan clean and expire function for external paths (apache#4903)

* [core] Introduce DataFilePathFactories to unify cache factories

* [test] Fix the unstable testNoChangelogProducerStreamingRandom (apache#4940)

* [flink] Introduce scan bounded to force bounded in streaming job (apache#4941)

* [docs] Fix typo of BIG_ENDIAN (apache#4945)

* [hotfix] Minor fix for FileIO.listFilesIterative

* [refactor] Clean unused codes in Lock (apache#4948)

* [core] Refine CommitMessage toString (apache#4950)

* [rest] Refactor AuthProviders to remove credential concept (apache#4959)

* [core] Support data token in RESTCatalog (apache#4944)

* [core] Clear cache when deleting the snapshot (apache#4966)

* [iceberg] Support skipping AWS Glue archive (apache#4962)

* [hotfix] remove_orphan_files action shouldn't check table argument (table=null means clean whole database) (apache#4961)

* [core] Fix NPE when retracting collect and merge-map (apache#4960)

=

* [hotfix] Update the maven version requirements in the readme (apache#4955)

* [rest] Refactor RESTTokenFileIO to cache FileIO in static cache (apache#4965)

* [hotfix] Fix flaky test AppendOnlyFileStoreTableTest#testBatchOrderWithCompaction (apache#4964)

* [doc][spark] Add read metadata columns (apache#4953)

* [core] Optimized iterative list implementations for FileIO (apache#4952)

* [core] Remove Catalog.fileio method (apache#4973)

* [core] Fix that sequence fields are mistakenly aggregated by default aggregator in AggregateMergeFunction (apache#4977)

* [spark] Fix update table with char type (apache#4972)

* [spark] Fix rollback not correctly identify tag or snapshot (apache#4947)

* [rest] Optimize partition methods to let rest return table not partitioned (apache#4979)

* [doc] Pypaimon api table_scan plan splits. (apache#4978)

* [filesystem] Support Tencent COSN (apache#4854)

* [core] ObjectRefresh with iterative list and batched commit (apache#4980)

* [doc] Add sys prefix to procedure (apache#4981)

* [core] Throw exception if increment query with rescale bucket (apache#4984)

* [core] Populate more metadata to object table (apache#4987)

* [hotfix] [docs] Fix cdc doc url and some typos (apache#4968)

* [spark] Fallback to spark except query if increment query with rescale bucket (apache#4989)

* [parquet] Refactory parquet reader using spark code. (apache#4982)

* [test][flink] Add tests back in PreAggregationITCase which deleted by apache#4982

* [parquet] Parquet ColumnarBatch should return ColumnarRowIterator for nested schema

* [parquet] Introduce LongIterator to Parquet RowIndexGenerator (apache#4991)

* [hotfix] Fix NPE in ColumnarRowIterator.reset

* [cdc] Add option to prevent logging of corrupted records (apache#4918)

* [flink] Replace per record in ReadOperator to work with object reuse

* [core] Refactory ColumnarRowIterator using LongIterator. (apache#4992)

* Merged Yelp customizations into Paimon  (apache#29)

* add debezium avro format

* Support debezium avro format in cdc action.

* fix ut

* fix

* fix

* fix

* Recovery cdc test

* add KafkaDebeziumAvroSyncTableActionITCase

* add test

* add avro cdc test

* fix test

* add doc

* handle complex types in TypeUtils when converting string to Object, also enable handling comlpex types in Avro

* dont use id in datafield for equals. Triggers DataField to not match and thinks is a schema change

* fix DataField comparison mismatch for nested RowTypes

* PRODENAB-66: add CODEOWNERS and PR template

Signed-off-by: Max Falk <gfalk@yelp.com>

* add owners

Signed-off-by: Max Falk <gfalk@yelp.com>

* [common][flink] Add support for complex types in kafka debezium avro cdc action

* Fix itest and separate the code flow for complex types

* Use toString instead of asText

* Add Yelp files

* PRODENAB-149 fix infinite loop when a field is a unparseable value using counter and log skipped messages in main

* PRODENAB-149: skip unreadable records for all cdc writers

Signed-off-by: Max Falk <gfalk@yelp.com>

* Run mvn spotless:apply

* Squash Yelpy setup and skipping unreadable rows

This squashes all our previous patches to paimon into a single commit.

Contains:
* Yelpy repo setup: https://github.yelpcorp.com/misc/flink-lib-paimon/commit/d1512a7ad3b7f691e6875c15795efd303c0bfcf3
* Skipping unreadable rows: https://github.yelpcorp.com/misc/flink-lib-paimon/commit/11350ab57215935138c7d122e20523c84da198bb

Signed-off-by: Max Falk <gfalk@yelp.com>

* PRODENAB-221: Fix JSON deserialization of strings

* athena cannot query the existing metadata for the iceberg compatibility mode.
trying with this commit to see if it helps get athena working

* squash Yelpy commits

Unreadable rows: https://github.yelpcorp.com/misc/flink-lib-paimon/commit/b59f96ede555e26d326b1d156529decf37ba18f2
Deserialization of strings: https://github.yelpcorp.com/misc/flink-lib-paimon/commit/788410dd3dbd460fd89e5170a1f942193f0a63ad

Signed-off-by: Max Falk <gfalk@yelp.com>

* revert metadata athena format

Signed-off-by: Max Falk <gfalk@yelp.com>

* add missing newline

Signed-off-by: Max Falk <gfalk@yelp.com>

* add enum support for flink debezium format

* DA-4260: skip glue archive with -Dglue.skip-archive=true

* enable glue.skip-archive by default

* DA-4260: move glue.skip-archive to a table option

* Set the default glue.skip-archive to false

* set CASCADE true, as before

* Use constants instead of hardcoding values

* Enhance IcebergManifestFile to correctly handle ByteBuffer conversion (apache#28)

* Enhance IcebergManifestFile to correctly handle ByteBuffer conversion and added a test for partition summary validation

* Updated test for different string lengths

* Updated tests for input length exceeding VARCHAR limit

* Refactor IcebergConversions and IcebergManifestFile for improved ByteBuffer handling; update tests for string partition null padding

* Add unit tests for IcebergConversions handling of VARCHAR types, including edge cases and concurrent access

* Added comments explaining change

* Fixed bad merge

---------

Signed-off-by: Max Falk <gfalk@yelp.com>
Co-authored-by: zhuangchong <zhuang.kerwin@gmail.com>
Co-authored-by: umesh <umesh@yelp.com>
Co-authored-by: Max Falk <gfalk@yelp.com>
Co-authored-by: Ashish Khatkar <ashish@Ashishs-Laptop.local>
Co-authored-by: Ashish Khatkar <akhatkar64@gmail.com>
Co-authored-by: Adel Atallah <adel@yelp.com>
Co-authored-by: Sofya Irwin <sofya@yelp.com>
Co-authored-by: Halit Olali <halitolali@yelp.com>
Co-authored-by: jkukreja <jkukreja@yelp.com>
Co-authored-by: Sina Siadat <sina@yelp.com>

---------

Signed-off-by: Max Falk <gfalk@yelp.com>
Co-authored-by: aiden.dong <782112163@qq.com>
Co-authored-by: yuzelin <33053040+yuzelin@users.noreply.github.com>
Co-authored-by: Zouxxyy <zouxinyu.zxy@alibaba-inc.com>
Co-authored-by: askwang <135721692+askwang@users.noreply.github.com>
Co-authored-by: HunterXHunter <cnlingmingqiang@gmail.com>
Co-authored-by: herefree <841043203@qq.com>
Co-authored-by: Jingsong <jingsonglee0@gmail.com>
Co-authored-by: lining <lining.jln@alibaba-inc.com>
Co-authored-by: JackeyLee007 <JackeyLee007@126.com>
Co-authored-by: xuzifu666 <1206332514@qq.com>
Co-authored-by: YeJunHao <41894543+leaves12138@users.noreply.github.com>
Co-authored-by: tsreaper <tsreaper96@gmail.com>
Co-authored-by: Giannis Polyzos <ipolyzos.se@gmail.com>
Co-authored-by: Yujiang Zhong <42907416+zhongyujiang@users.noreply.github.com>
Co-authored-by: yangjf2019 <54518670+yangjf2019@users.noreply.github.com>
Co-authored-by: jiangmengmeng <1804226997@qq.com>
Co-authored-by: liming.1018 <liming.1018@bytedance.com>
Co-authored-by: yunfengzhou-hub <yuri.zhouyunfeng@outlook.com>
Co-authored-by: WenjunMin <aitozi@apache.org>
Co-authored-by: LsomeYeah <94825748+LsomeYeah@users.noreply.github.com>
Co-authored-by: zyz33 <35164637+zhangyazhe@users.noreply.github.com>
Co-authored-by: Doroszlai, Attila <6454655+adoroszlai@users.noreply.github.com>
Co-authored-by: Jiao Mingye <35512473+mxdzs0612@users.noreply.github.com>
Co-authored-by: HouliangQi <neuyilan@163.com>
Co-authored-by: yuzelin <zelin.yzl@alibaba-inc.com>
Co-authored-by: Wenchao Wu <60921147+Stephen0421@users.noreply.github.com>
Co-authored-by: Ashish Khatkar <akhatkar64@gmail.com>
Co-authored-by: yinyao <144221862+RunningDB@users.noreply.github.com>
Co-authored-by: Weijie Guo <reswqa@163.com>
Co-authored-by: wangwj <hongli.wwj@gmail.com>
Co-authored-by: xiangyu0xf <xiangyu0xf@gmail.com>
Co-authored-by: Tan-JiaLiang <tanjialiang1997@gmail.com>
Co-authored-by: XiaoHongbo <1346652787@qq.com>
Co-authored-by: HunterXHunter <1356469429@qq.com>
Co-authored-by: Lucian <131578653+Moonlight-CL@users.noreply.github.com>
Co-authored-by: cxzl25 <3898450+cxzl25@users.noreply.github.com>
Co-authored-by: Xuannan <suxuannan95@gmail.com>
Co-authored-by: chuangchuang <lizc9@xiaopeng.com>
Co-authored-by: Mingyu Chen (Rayner) <yunyou@selectdb.com>
Co-authored-by: chenjian2664 <chenjian2664@gmail.com>
Co-authored-by: Xiaoguang Zhu <smdsbz@qq.com>
Co-authored-by: Gang Wu <ustcwg@gmail.com>
Co-authored-by: Sina Siadat <siadat@gmail.com>
Co-authored-by: HunterXHunter <mingqianglin1010@gmail.com>
Co-authored-by: liujinhui <965147871@qq.com>
Co-authored-by: Xiaoguang Zhu <zhuxiaoguang.zxg@alibaba-inc.com>
Co-authored-by: Adel Atallah <2213999+atallahade@users.noreply.github.com>
Co-authored-by: zhuangchong <zhuang.kerwin@gmail.com>
Co-authored-by: umesh <umesh@yelp.com>
Co-authored-by: Max Falk <gfalk@yelp.com>
Co-authored-by: Ashish Khatkar <ashish@Ashishs-Laptop.local>
Co-authored-by: Adel Atallah <adel@yelp.com>
Co-authored-by: Sofya Irwin <sofya@yelp.com>
Co-authored-by: Halit Olali <halitolali@yelp.com>
Co-authored-by: jkukreja <jkukreja@yelp.com>
Co-authored-by: Sina Siadat <sina@yelp.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants