假设有一张很大的表,字段很多,单个job写的很慢,这个时候要并发写。paimon 基于 ‘merge-engine’ = 'partial-update’和Sequence Group可以支持并发写,多个job写不同的字段,每个job维护自己的Sequence Group
由于本地环境的问题,暂时的模拟是启动三个程序消费三个socket流,三个socket流是同一份数据,然后写入同一张表
CREATE TABLE if not exists MyTable(
id INT ,
col1_1 STRING,
col1_2 STRING,
col1_3 STRING,
col1_4 STRING,
version1 INT ,
col2_1 STRING,
col2_2 STRING,
col2_3 STRING,
col2_4 STRING,
version2 INT,
col3_1 STRING,
col3_2 STRING,
col3_3 STRING,
col3_4 STRING,
version3 INT ,
PRIMARY KEY (id) NOT ENFORCED
)
WITH (
'merge-engine'='partial-update',
'fields.version1.sequence-group'='col1_1,col1_2,col1_3,col1_4',
'fields.version2.sequence-group'='col2_1,col2_2,col2_3,col2_4',
'fields.version3.sequence-group'='col3_1,col3_2,col3_3,col3_4',
'bucket' = '2'
);
{"id":1000,"col1_1":"col1_1-0","col1_2":"col1_2-0","col1_3":"col1_3-0","col1_4":"col1_4-0","version1":0,"col2_1":"col2_1-0","col2_2":"col2_2-0","col2_3":"col2_3-0","col2_4":"col2_4-0","version2":0,"col3_1":"col3_1-0","col3_2":"col3_2-0","col3_3":"col3_3-0","col3_4":"col3_4-0","version3":0}
{"id":1000,"col1_1":"col1_1-0","col1_2":"dddd","col1_3":"bbbb","col1_4":"col1_4-0","version1":1,"col2_1":"qqqq-0","col2_2":"cccc","col2_3":"col2_3-0","col2_4":"col2_4-0","version2":2,"col3_1":"col3_1-0","col3_2":"col3_2-0","col3_3":"tyty-0","col3_4":"aaaa","version3":3
//三个程序分别执行下面三个语句
insert into MyTable
(id,
col1_1,
col1_2,
col1_3,
col1_4,
version1,
col2_1,
col2_2,
col2_3,
col2_4,
version2,
col3_1,
col3_2,
col3_3,
col3_4,
version3)
select
id,
col1_1,
col1_2,
col1_3,
col1_4,
version1,
CAST(NULL AS String)as col2_1,
CAST(NULL AS String)as col2_2,
CAST(NULL AS String)as col2_3,
CAST(NULL AS String)as col2_4,
CAST(NULL AS INT)as version2,
CAST(NULL AS String)as col3_1,
CAST(NULL AS String)as col3_2,
CAST(NULL AS String)as col3_3,
CAST(NULL AS String)as col3_4,
CAST(NULL AS INT)as version3
from
InputTable;
//
insert into MyTable (id ,
col1_1 ,
col1_2 ,
col1_3 ,
col1_4 ,
version1 ,
col2_1 ,
col2_2 ,
col2_3 ,
col2_4 ,
version2 ,
col3_1 ,
col3_2 ,
col3_3 ,
col3_4 ,
version3) select
id ,
CAST(NULL AS String) as col1_1 ,
CAST(NULL AS String) as col1_2 ,
CAST(NULL AS String) as col1_3 ,
CAST(NULL AS String) as col1_4 ,
CAST(NULL AS INT) version1 ,
col2_1 ,
col2_2 ,
col2_3 ,
col2_4 ,
version2 ,
CAST(NULL AS String) as col3_1 ,
CAST(NULL AS String) as col3_2 ,
CAST(NULL AS String) as col3_3 ,
CAST(NULL AS String) as col3_4 ,
CAST(NULL AS INT) as version3
from
InputTable;
//
insert into MyTable (id ,
col1_1 ,
col1_2 ,
col1_3 ,
col1_4 ,
version1 ,
col2_1 ,
col2_2 ,
col2_3 ,
col2_4 ,
version2 ,
col3_1 ,
col3_2 ,
col3_3 ,
col3_4 ,
version3) select
id ,
CAST(NULL AS String) as col1_1 ,
CAST(NULL AS String) as col1_2 ,
CAST(NULL AS String) as col1_3 ,
CAST(NULL AS String) as col1_4 ,
CAST(NULL AS INT) version1 ,
CAST(NULL AS String) as col2_1 ,
CAST(NULL AS String) as col2_2 ,
CAST(NULL AS String) as col2_3 ,
CAST(NULL AS String) as col2_4 ,
CAST(NULL AS INT) as version2 ,
col3_1 ,
col3_2 ,
col3_3 ,
col3_4 ,
version3
from
InputTable;
import com.alibaba.fastjson.JSON;
import com.alibaba.fastjson.JSONObject;
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.RestOptions;
import org.apache.flink.streaming.api.CheckpointingMode;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.Schema;
import org.apache.flink.table.api.Table;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
import org.apache.flink.table.types.DataType;
import org.apache.flink.types.Row;
import java.util.List;
public class Sink2Paimon {
public static void writeTo() {
Configuration config = new Configuration();
// 指定WebUI界面的访问端口,默认是8081
config.setString(RestOptions.BIND_PORT, "9999");
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config);
env.enableCheckpointing(5000, CheckpointingMode.EXACTLY_ONCE);
DataStream<String> socketStream = env.socketTextStream("localhost", 9000);
StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env);
Schema schema1 = Schema.newBuilder()
.column("id", DataTypes.INT())
.column("col1_1", DataTypes.STRING())
.column("col1_2", DataTypes.STRING())
.column("col1_3", DataTypes.STRING())
.column("col1_4", DataTypes.STRING())
.column("version1", DataTypes.INT())
.column("col2_1", DataTypes.STRING())
.column("col2_2", DataTypes.STRING())
.column("col2_3", DataTypes.STRING())
.column("col2_4", DataTypes.STRING())
.column("version2", DataTypes.INT())
.column("col3_1", DataTypes.STRING())
.column("col3_2", DataTypes.STRING())
.column("col3_3", DataTypes.STRING())
.column("col3_4", DataTypes.STRING())
.column("version3", DataTypes.INT())
.build();
SingleOutputStreamOperator<Row> returns = socketStream.map(new MapFunction<String, Row>() {
@Override
public Row map(String value) throws Exception {
Schema aaa = Schema.newBuilder()
.column("id", DataTypes.INT())
.column("col1_1", DataTypes.STRING())
.column("col1_2", DataTypes.STRING())
.column("col1_3", DataTypes.STRING())
.column("col1_4", DataTypes.STRING())
.column("version1", DataTypes.INT())
.column("col2_1", DataTypes.STRING())
.column("col2_2", DataTypes.STRING())
.column("col2_3", DataTypes.STRING())
.column("col2_4", DataTypes.STRING())
.column("version2", DataTypes.INT())
.column("col3_1", DataTypes.STRING())
.column("col3_2", DataTypes.STRING())
.column("col3_3", DataTypes.STRING())
.column("col3_4", DataTypes.STRING())
.column("version3", DataTypes.INT())
.build();
JSONObject jsonObject = JSON.parseObject(value);
List<Schema.UnresolvedColumn> columns = aaa.getColumns();
Row row = new Row(columns.size());
for (int i = 0; i < columns.size(); i++) {
String name = columns.get(i).getName();
row.setField(i, jsonObject.get(name));
}
return row;
}
}).returns(Types.ROW_NAMED(
new String[]{"id",
"col1_1",
"col1_2",
"col1_3",
"col1_4",
"version1",
"col2_1",
"col2_2",
"col2_3",
"col2_4",
"version2",
"col3_1",
"col3_2",
"col3_3",
"col3_4",
"version3"},
Types.INT, Types.STRING, Types.STRING, Types.STRING,
Types.STRING, Types.INT, Types.STRING, Types.STRING,
Types.STRING, Types.STRING, Types.INT, Types.STRING, Types.STRING, Types.STRING, Types.STRING, Types.INT
));
Table table = tableEnv.fromDataStream(returns, schema1);
tableEnv.executeSql("CREATE CATALOG paimon WITH ('type' = 'paimon', 'warehouse'='file:///D://data')");
tableEnv.executeSql("USE CATALOG paimon");
tableEnv.createTemporaryView("InputTable", table);
tableEnv.executeSql("CREATE TABLE if not exists MyTable(\n" +
"id INT ,\n" +
"col1_1 STRING,\n" +
"col1_2 STRING,\n" +
"col1_3 STRING,\n" +
"col1_4 STRING,\n" +
"version1 INT ,\n" +
"col2_1 STRING,\n" +
"col2_2 STRING,\n" +
"col2_3 STRING,\n" +
"col2_4 STRING,\n" +
"version2 INT,\n" +
"col3_1 STRING,\n" +
"col3_2 STRING,\n" +
"col3_3 STRING,\n" +
"col3_4 STRING,\n" +
"version3 INT ,\n" +
"PRIMARY KEY (id) NOT ENFORCED\n" +
")\n" +
"WITH (\n" +
"'merge-engine'='partial-update',\n" +
"'fields.version1.sequence-group'='col1_1,col1_2,col1_3,col1_4',\n" +
"'fields.version2.sequence-group'='col2_1,col2_2,col2_3,col2_4',\n" +
"'fields.version3.sequence-group'='col3_1,col3_2,col3_3,col3_4',\n" +
"'bucket' = '2'\n" +
")");
//1
tableEnv.executeSql(
"insert into MyTable\n" +
"(id,\n" +
"col1_1,\n" +
"col1_2,\n" +
"col1_3,\n" +
"col1_4,\n" +
"version1,\n" +
"col2_1,\n" +
"col2_2,\n" +
"col2_3,\n" +
"col2_4,\n" +
"version2,\n" +
"col3_1,\n" +
"col3_2,\n" +
"col3_3,\n" +
"col3_4,\n" +
"version3)\n" +
"select\n" +
"id,\n" +
"col1_1,\n" +
"col1_2,\n" +
"col1_3,\n" +
"col1_4,\n" +
"version1,\n" +
"CAST(NULL AS String)as col2_1,\n" +
"CAST(NULL AS String)as col2_2,\n" +
"CAST(NULL AS String)as col2_3,\n" +
"CAST(NULL AS String)as col2_4,\n" +
"CAST(NULL AS INT)as version2,\n" +
"CAST(NULL AS String)as col3_1,\n" +
"CAST(NULL AS String)as col3_2,\n" +
"CAST(NULL AS String)as col3_3,\n" +
"CAST(NULL AS String)as col3_4,\n" +
"CAST(NULL AS INT)as version3\n" +
"from\n" +
"InputTable");
}
public static void main(String[] args) {
writeTo();
}
}
//{"id":1000,"col1_1":"col1_1-0","col1_2":"col1_2-0","col1_3":"col1_3-0","col1_4":"col1_4-0","version1":0,"col2_1":"col2_1-0","col2_2":"col2_2-0","col2_3":"col2_3-0","col2_4":"col2_4-0","version2":0,"col3_1":"col3_1-0","col3_2":"col3_2-0","col3_3":"col3_3-0","col3_4":"col3_4-0","version3":0}
+------+-----------+-----------+-----------+------------+-------------+----------+---------+-------------+-----------+-------------+----------+----------+---------+----------+-------------+
| id | col1_1 | col1_2 | col1_3 | col1_4 | version1 | col2_1 | col2_2 | col2_3 | col2_4 | version2 | col3_1 | col3_2 | col3_3 | col3_4 | version3 |
+------+-----------+-----------+-----------+------------+-------------+----------+---------+-------------+-----------+-------------+----------+----------+---------+----------+-------------+
| 1000 | col1_1-0 | col1_2-0 | col1_3-0 | col1_4-0 | 0 | col2_1-0 |col2_2-0 | col2_3-0 | col2_4-0 | 0 | col3_1-0 | col3_2-0 |col3_3-0 | col3_4-0 | 0 |
+------+-----------+-----------+-----------+------------+-------------+----------+---------+-------------+-----------+-------------+----------+----------+---------+----------+-------------+
//{"id":1000,"col1_1":"col1_1-0","col1_2":"dddd","col1_3":"bbbb","col1_4":"col1_4-0","version1":1,"col2_1":"qqqq-0","col2_2":"cccc","col2_3":"col2_3-0","col2_4":"col2_4-0","version2":2,"col3_1":"col3_1-0","col3_2":"col3_2-0","col3_3":"tyty-0","col3_4":"aaaa","version3":3}
+------+-----------+-----------+-----------+------------+-------------+----------+---------+-------------+-----------+-------------+----------+----------+---------+----------+-------------+
| id | col1_1 | col1_2 | col1_3 | col1_4 | version1 | col2_1 | col2_2 | col2_3 | col2_4 | version2 | col3_1 | col3_2 | col3_3 | col3_4 | version3 |
+------+-----------+-----------+-----------+------------+-------------+----------+---------+-------------+-----------+-------------+----------+----------+---------+----------+-------------+
| 1000 | col1_1-0 | dddd | bbbb | col1_4-0 | 1 | qqqq-0 | cccc | col2_3-0 | col2_4-0 | 2 | col3_1-0 | col3_2-0 | tyty-0 | aaaa | 3 |
+------+-----------+-----------+-----------+------------+-------------+----------+---------+-------------+-----------+-------------+----------+----------+---------+----------+-------------+
从结果来看paimon对于并发写是没问题的,上面只是两条数据简单的测了下,后于有机会的话会再试下