diff --git a/chunjun-connectors/chunjun-connector-hdfs/src/main/java/com/dtstack/chunjun/connector/hdfs/converter/HdfsOrcColumnConverter.java b/chunjun-connectors/chunjun-connector-hdfs/src/main/java/com/dtstack/chunjun/connector/hdfs/converter/HdfsOrcColumnConverter.java index dc00424976..e15140cbe9 100644 --- a/chunjun-connectors/chunjun-connector-hdfs/src/main/java/com/dtstack/chunjun/connector/hdfs/converter/HdfsOrcColumnConverter.java +++ b/chunjun-connectors/chunjun-connector-hdfs/src/main/java/com/dtstack/chunjun/connector/hdfs/converter/HdfsOrcColumnConverter.java @@ -26,6 +26,7 @@ import com.dtstack.chunjun.element.ColumnRowData; import com.dtstack.chunjun.element.column.BigDecimalColumn; import com.dtstack.chunjun.element.column.BooleanColumn; +import com.dtstack.chunjun.element.column.ByteColumn; import com.dtstack.chunjun.element.column.BytesColumn; import com.dtstack.chunjun.element.column.StringColumn; import com.dtstack.chunjun.element.column.TimestampColumn; @@ -132,8 +133,7 @@ protected IDeserializationConverter createInternalConverter(String type) { case "BOOLEAN": return (IDeserializationConverter) BooleanColumn::new; case "TINYINT": - return (IDeserializationConverter) - val -> new BigDecimalColumn(val.toString()); + return (IDeserializationConverter) ByteColumn::new; case "SMALLINT": return (IDeserializationConverter) val -> new BigDecimalColumn(val.toString()); diff --git a/chunjun-connectors/chunjun-connector-hdfs/src/main/java/com/dtstack/chunjun/connector/hdfs/converter/HdfsParquetColumnConverter.java b/chunjun-connectors/chunjun-connector-hdfs/src/main/java/com/dtstack/chunjun/connector/hdfs/converter/HdfsParquetColumnConverter.java index fe39c60985..f5d5bdac4f 100644 --- a/chunjun-connectors/chunjun-connector-hdfs/src/main/java/com/dtstack/chunjun/connector/hdfs/converter/HdfsParquetColumnConverter.java +++ b/chunjun-connectors/chunjun-connector-hdfs/src/main/java/com/dtstack/chunjun/connector/hdfs/converter/HdfsParquetColumnConverter.java @@ -27,6 +27,7 @@ import com.dtstack.chunjun.element.ColumnRowData; import com.dtstack.chunjun.element.column.BigDecimalColumn; import com.dtstack.chunjun.element.column.BooleanColumn; +import com.dtstack.chunjun.element.column.ByteColumn; import com.dtstack.chunjun.element.column.BytesColumn; import com.dtstack.chunjun.element.column.StringColumn; import com.dtstack.chunjun.element.column.TimestampColumn; @@ -135,6 +136,7 @@ protected IDeserializationConverter createInternalConverter(String type) { case "BOOLEAN": return (IDeserializationConverter) BooleanColumn::new; case "TINYINT": + return (IDeserializationConverter) ByteColumn::new; case "SMALLINT": case "INT": return (IDeserializationConverter) diff --git a/chunjun-connectors/chunjun-connector-hdfs/src/main/java/com/dtstack/chunjun/connector/hdfs/converter/HdfsTextColumnConverter.java b/chunjun-connectors/chunjun-connector-hdfs/src/main/java/com/dtstack/chunjun/connector/hdfs/converter/HdfsTextColumnConverter.java index 5be9d9c380..ae00fc3e41 100644 --- a/chunjun-connectors/chunjun-connector-hdfs/src/main/java/com/dtstack/chunjun/connector/hdfs/converter/HdfsTextColumnConverter.java +++ b/chunjun-connectors/chunjun-connector-hdfs/src/main/java/com/dtstack/chunjun/connector/hdfs/converter/HdfsTextColumnConverter.java @@ -26,6 +26,7 @@ import com.dtstack.chunjun.element.ColumnRowData; import com.dtstack.chunjun.element.column.BigDecimalColumn; import com.dtstack.chunjun.element.column.BooleanColumn; +import com.dtstack.chunjun.element.column.ByteColumn; import com.dtstack.chunjun.element.column.SqlDateColumn; import com.dtstack.chunjun.element.column.StringColumn; import com.dtstack.chunjun.element.column.TimestampColumn; @@ -122,6 +123,7 @@ protected IDeserializationConverter createInternalConverter(String type) { return (IDeserializationConverter) val -> new BooleanColumn(Boolean.parseBoolean(val)); case "TINYINT": + return (IDeserializationConverter) ByteColumn::new; case "SMALLINT": case "INT": case "BIGINT": diff --git a/chunjun-connectors/chunjun-connector-http/src/main/java/com/dtstack/chunjun/connector/http/source/HttpSourceFactory.java b/chunjun-connectors/chunjun-connector-http/src/main/java/com/dtstack/chunjun/connector/http/source/HttpSourceFactory.java index 87e6f4c405..f8194ff5a8 100644 --- a/chunjun-connectors/chunjun-connector-http/src/main/java/com/dtstack/chunjun/connector/http/source/HttpSourceFactory.java +++ b/chunjun-connectors/chunjun-connector-http/src/main/java/com/dtstack/chunjun/connector/http/source/HttpSourceFactory.java @@ -91,11 +91,12 @@ public HttpSourceFactory(SyncConf config, StreamExecutionEnvironment env) { || org.apache.commons.lang3.StringUtils.isBlank( syncConf.getTransformer().getTransformSql())) { typeInformation = - TableUtil.getTypeInformation(Collections.emptyList(), getRawTypeConverter()); + TableUtil.getTypeInformation( + Collections.emptyList(), getRawTypeConverter(), true); } else { typeInformation = TableUtil.getTypeInformation( - subColumns(httpRestConfig.getColumn()), getRawTypeConverter()); + subColumns(httpRestConfig.getColumn()), getRawTypeConverter(), false); useAbstractBaseColumn = false; } super.initCommonConf(httpRestConfig); diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileSinkFactory.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileSinkFactory.java index 62c9ce9210..acfacfe1ae 100644 --- a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileSinkFactory.java +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorFileSinkFactory.java @@ -79,7 +79,7 @@ public DataStreamSink createSink(DataStream dataSet) { inceptorFileConf.getColumn(), getRawTypeConverter()); - builder.setRowConverter(rowConverter); + builder.setRowConverter(rowConverter, useAbstractBaseColumn); return createOutput(dataSet, builder.finish()); } diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorJdbcSinkFactory.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorJdbcSinkFactory.java index 503d0dad74..501f89e80f 100644 --- a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorJdbcSinkFactory.java +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/sink/InceptorJdbcSinkFactory.java @@ -57,7 +57,7 @@ public DataStreamSink createSink(DataStream dataSet) { TableUtil.createRowType(jdbcConf.getColumn(), getRawTypeConverter()); rowConverter = jdbcDialect.getRowConverter(rowType); } - builder.setRowConverter(rowConverter); + builder.setRowConverter(rowConverter, useAbstractBaseColumn); if (builder instanceof InceptorHdfsOutputFormatBuilder) { // InceptorHdfsOutputFormatBuilder 只有实时任务或者离线任务的事务表才会调用 所以此处设置为true,其余的orc text // parquet通过文件方式写入 diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorFileSourceFactory.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorFileSourceFactory.java index 5f4d00b333..0db81a3cb7 100644 --- a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorFileSourceFactory.java +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/source/InceptorFileSourceFactory.java @@ -83,7 +83,7 @@ public DataStream createSource() { inceptorFileConf.getColumn(), getRawTypeConverter()); - builder.setRowConverter(rowConverter); + builder.setRowConverter(rowConverter, useAbstractBaseColumn); return createInput(builder.finish()); } } diff --git a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/table/InceptorDynamicTableFactory.java b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/table/InceptorDynamicTableFactory.java index b099299fc9..87864b2101 100644 --- a/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/table/InceptorDynamicTableFactory.java +++ b/chunjun-connectors/chunjun-connector-inceptor/src/main/java/com/dtstack/chunjun/connector/inceptor/table/InceptorDynamicTableFactory.java @@ -265,7 +265,6 @@ protected JdbcInputFormatBuilder getInputFormatBuilder() { return null; } - @Override protected void validateConfigOptions(ReadableConfig config) { String jdbcUrl = config.get(URL); checkState(true, "Cannot handle such jdbc url: " + jdbcUrl); diff --git a/chunjun-connectors/chunjun-connector-jdbc-base/src/main/java/com/dtstack/chunjun/connector/jdbc/source/JdbcInputFormat.java b/chunjun-connectors/chunjun-connector-jdbc-base/src/main/java/com/dtstack/chunjun/connector/jdbc/source/JdbcInputFormat.java index feeb08ecdb..cb57390f4e 100644 --- a/chunjun-connectors/chunjun-connector-jdbc-base/src/main/java/com/dtstack/chunjun/connector/jdbc/source/JdbcInputFormat.java +++ b/chunjun-connectors/chunjun-connector-jdbc-base/src/main/java/com/dtstack/chunjun/connector/jdbc/source/JdbcInputFormat.java @@ -839,7 +839,6 @@ protected List createRangeSplits( protected void executeQuery(String startLocation) throws SQLException { if (currentJdbcInputSplit.isPolling()) { if (StringUtils.isBlank(startLocation)) { - // Get the startLocation from the database queryPollingWithOutStartLocation(); // Concatenated sql statement for next polling query StringBuilder builder = new StringBuilder(128); @@ -889,7 +888,7 @@ public void initPrepareStatement(String querySql) throws SQLException { ps.setQueryTimeout(jdbcConf.getQueryTimeOut()); } /** - * 间隔轮询查询起始位置 + * polling mode first query when startLocation is not set * * @throws SQLException */ diff --git a/chunjun-connectors/chunjun-connector-jdbc-base/src/main/java/com/dtstack/chunjun/connector/jdbc/util/SqlUtil.java b/chunjun-connectors/chunjun-connector-jdbc-base/src/main/java/com/dtstack/chunjun/connector/jdbc/util/SqlUtil.java index ba1986b12a..64dc4f2b27 100644 --- a/chunjun-connectors/chunjun-connector-jdbc-base/src/main/java/com/dtstack/chunjun/connector/jdbc/util/SqlUtil.java +++ b/chunjun-connectors/chunjun-connector-jdbc-base/src/main/java/com/dtstack/chunjun/connector/jdbc/util/SqlUtil.java @@ -263,7 +263,7 @@ public static String buildSplitFilterSql( sql = jdbcDialect.getSplitModFilter(jdbcInputSplit, splitColumn); } if (jdbcInputSplit.getSplitNumber() == 0) { - sql += " OR " + jdbcDialect.quoteIdentifier(splitColumn) + " IS NULL"; + sql = "(" + sql + " OR " + jdbcDialect.quoteIdentifier(splitColumn) + " IS NULL)"; } return sql; } diff --git a/chunjun-connectors/chunjun-connector-postgresql/src/main/java/com/dtstack/chunjun/connector/postgresql/converter/PostgresqlColumnConverter.java b/chunjun-connectors/chunjun-connector-postgresql/src/main/java/com/dtstack/chunjun/connector/postgresql/converter/PostgresqlColumnConverter.java index 18a6be5293..c19b6c9297 100644 --- a/chunjun-connectors/chunjun-connector-postgresql/src/main/java/com/dtstack/chunjun/connector/postgresql/converter/PostgresqlColumnConverter.java +++ b/chunjun-connectors/chunjun-connector-postgresql/src/main/java/com/dtstack/chunjun/connector/postgresql/converter/PostgresqlColumnConverter.java @@ -29,10 +29,10 @@ import com.dtstack.chunjun.converter.ISerializationConverter; import com.dtstack.chunjun.element.AbstractBaseColumn; import com.dtstack.chunjun.element.ColumnRowData; -import com.dtstack.chunjun.element.column.ArrayColumn; import com.dtstack.chunjun.element.column.BigDecimalColumn; import com.dtstack.chunjun.element.column.BooleanColumn; import com.dtstack.chunjun.element.column.BytesColumn; +import com.dtstack.chunjun.element.column.SqlArrayColumn; import com.dtstack.chunjun.element.column.SqlDateColumn; import com.dtstack.chunjun.element.column.StringColumn; import com.dtstack.chunjun.element.column.TimeColumn; @@ -89,7 +89,7 @@ public FieldNamedPreparedStatement toExternal( String field = ((ColumnRowData) rowData).getField(index).asString(); Array array = new PgArray(connection, arrayType.get(fieldTypeList.get(index)), field); - AbstractBaseColumn arrayColumn = new ArrayColumn(array); + AbstractBaseColumn arrayColumn = new SqlArrayColumn(array); ((ColumnRowData) rowData).setField(index, arrayColumn); } toExternalConverters.get(index).serialize(rowData, index, statement); diff --git a/chunjun-connectors/chunjun-connector-postgresql/src/main/java/com/dtstack/chunjun/connector/postgresql/source/PostgresqlSourceFactory.java b/chunjun-connectors/chunjun-connector-postgresql/src/main/java/com/dtstack/chunjun/connector/postgresql/source/PostgresqlSourceFactory.java index 115b085fa9..96d545c6af 100644 --- a/chunjun-connectors/chunjun-connector-postgresql/src/main/java/com/dtstack/chunjun/connector/postgresql/source/PostgresqlSourceFactory.java +++ b/chunjun-connectors/chunjun-connector-postgresql/src/main/java/com/dtstack/chunjun/connector/postgresql/source/PostgresqlSourceFactory.java @@ -19,7 +19,6 @@ package com.dtstack.chunjun.connector.postgresql.source; import com.dtstack.chunjun.conf.SyncConf; -import com.dtstack.chunjun.connector.jdbc.source.JdbcInputFormatBuilder; import com.dtstack.chunjun.connector.jdbc.source.JdbcSourceFactory; import com.dtstack.chunjun.connector.postgresql.dialect.PostgresqlDialect; @@ -37,9 +36,4 @@ public class PostgresqlSourceFactory extends JdbcSourceFactory { public PostgresqlSourceFactory(SyncConf syncConf, StreamExecutionEnvironment env) { super(syncConf, env, new PostgresqlDialect()); } - - @Override - protected JdbcInputFormatBuilder getBuilder() { - return new JdbcInputFormatBuilder(new PostgresqlInputFormat()); - } } diff --git a/chunjun-connectors/chunjun-connector-postgresql/src/main/java/com/dtstack/chunjun/connector/postgresql/table/PostgresqlDynamicTableFactory.java b/chunjun-connectors/chunjun-connector-postgresql/src/main/java/com/dtstack/chunjun/connector/postgresql/table/PostgresqlDynamicTableFactory.java index 3fb205090c..5454bfaa93 100644 --- a/chunjun-connectors/chunjun-connector-postgresql/src/main/java/com/dtstack/chunjun/connector/postgresql/table/PostgresqlDynamicTableFactory.java +++ b/chunjun-connectors/chunjun-connector-postgresql/src/main/java/com/dtstack/chunjun/connector/postgresql/table/PostgresqlDynamicTableFactory.java @@ -19,10 +19,8 @@ package com.dtstack.chunjun.connector.postgresql.table; import com.dtstack.chunjun.connector.jdbc.dialect.JdbcDialect; -import com.dtstack.chunjun.connector.jdbc.source.JdbcInputFormatBuilder; import com.dtstack.chunjun.connector.jdbc.table.JdbcDynamicTableFactory; import com.dtstack.chunjun.connector.postgresql.dialect.PostgresqlDialect; -import com.dtstack.chunjun.connector.postgresql.source.PostgresqlInputFormat; /** * @program chunjun @@ -42,9 +40,4 @@ public String factoryIdentifier() { protected JdbcDialect getDialect() { return new PostgresqlDialect(); } - - @Override - protected JdbcInputFormatBuilder getInputFormatBuilder() { - return new JdbcInputFormatBuilder(new PostgresqlInputFormat()); - } } diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarrocksSinkFactory.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarrocksSinkFactory.java index 19b49520a8..92e9603191 100644 --- a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarrocksSinkFactory.java +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarrocksSinkFactory.java @@ -157,10 +157,11 @@ public DataStreamSink createSink(DataStream dataSet) { TableUtil.createRowType(jdbcConf.getColumn(), getRawTypeConverter()); rowConverter = jdbcDialect.getRowConverter(rowType); typeInformation = - TableUtil.getTypeInformation(Collections.emptyList(), getRawTypeConverter()); + TableUtil.getTypeInformation( + Collections.emptyList(), getRawTypeConverter(), false); } else { List fieldList = syncConf.getWriter().getFieldList(); - typeInformation = TableUtil.getTypeInformation(fieldList, getRawTypeConverter()); + typeInformation = TableUtil.getTypeInformation(fieldList, getRawTypeConverter(), true); } builder.setRowConverter(rowConverter, useAbstractBaseColumn); diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/Main.java b/chunjun-core/src/main/java/com/dtstack/chunjun/Main.java index 8f0956b1ba..59a889b7d0 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/Main.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/Main.java @@ -257,7 +257,7 @@ private static DataStream syncStreamToTable( DataType[] tableDataTypes = adaptTable.getSchema().getFieldDataTypes(); String[] tableFieldNames = adaptTable.getSchema().getFieldNames(); - TypeInformation typeInformation = + TypeInformation typeInformation = TableUtil.getTypeInformation(tableDataTypes, tableFieldNames); DataStream dataStream = tableEnv.toRetractStream(adaptTable, typeInformation).map(f -> f.f1); diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/element/ColumnRowData.java b/chunjun-core/src/main/java/com/dtstack/chunjun/element/ColumnRowData.java index fbf243ae40..d27c736f09 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/element/ColumnRowData.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/element/ColumnRowData.java @@ -66,6 +66,12 @@ public ColumnRowData(RowKind kind, int arity) { byteSize = 1; } + public ColumnRowData(RowKind kind, int arity, int byteSize) { + this.columnList = new ArrayList<>(arity); + this.kind = kind; + this.byteSize = byteSize; + } + public ColumnRowData(int arity) { this(RowKind.INSERT, arity); } @@ -78,6 +84,10 @@ public void addHeader(String name) { byteSize += getStringSize(name); } + public void setHeader(Map header) { + this.header = header; + } + public void replaceHeader(String original, String another) { if (this.header == null || !this.header.containsKey(original)) { addHeader(another); @@ -143,6 +153,10 @@ public void addField(AbstractBaseColumn value) { } } + public void addFieldWithOutByteSize(AbstractBaseColumn value) { + this.columnList.add(value); + } + public void addAllField(List list) { for (AbstractBaseColumn column : list) { addField(column); @@ -274,7 +288,7 @@ public RowData getRow(int pos, int numFields) { return null; } - public long getByteSize() { + public int getByteSize() { return byteSize; } diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/BigDecimalColumn.java b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/BigDecimalColumn.java index 754f62f2ff..268ad0df5a 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/BigDecimalColumn.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/BigDecimalColumn.java @@ -75,6 +75,14 @@ public BigDecimalColumn(BigDecimal bigDecimal, String data) { super(bigDecimal, getStringSize(data)); } + public BigDecimalColumn(BigDecimal data, int byteSize) { + super(data, byteSize); + } + + public static BigDecimalColumn from(BigDecimal data) { + return new BigDecimalColumn(data, 0); + } + @Override public String asString() { if (null == data) { diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/BooleanColumn.java b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/BooleanColumn.java index de9c8e65e3..5e15f37e17 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/BooleanColumn.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/BooleanColumn.java @@ -36,6 +36,14 @@ public BooleanColumn(boolean data) { super(data, 1); } + public BooleanColumn(boolean data, int byteSize) { + super(data, byteSize); + } + + public static BooleanColumn from(boolean data) { + return new BooleanColumn(data, 0); + } + @Override public Boolean asBoolean() { if (null == data) { diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/ByteColumn.java b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/ByteColumn.java index dad40136e4..491a536f2e 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/ByteColumn.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/ByteColumn.java @@ -35,6 +35,14 @@ public ByteColumn(byte data) { super(data, 1); } + public ByteColumn(byte data, int byteSize) { + super(data, byteSize); + } + + public static ByteColumn from(byte data) { + return new ByteColumn(data, 0); + } + public ByteColumn(char data) { super(data, 1); } @@ -56,7 +64,7 @@ public String asString() { @Override public BigDecimal asBigDecimal() { - throw new CastException("byte", "BigDecimal", String.valueOf(data)); + return new BigDecimal((byte) data); } @Override diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/BytesColumn.java b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/BytesColumn.java index 0f5f6fee9e..7ef6f8f953 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/BytesColumn.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/BytesColumn.java @@ -44,6 +44,14 @@ public BytesColumn(byte[] data, String encoding) { this.encoding = encoding; } + public BytesColumn(byte[] data, int byteSize) { + super(data, byteSize); + } + + public static BytesColumn from(byte[] data) { + return new BytesColumn(data, 0); + } + @Override public Boolean asBoolean() { if (null == data) { diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/MapColumn.java b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/MapColumn.java index 2795b23b8a..fc669ffc1d 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/MapColumn.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/MapColumn.java @@ -42,6 +42,14 @@ public MapColumn(Map data) { } } + public MapColumn(Map data, int byteSize) { + super(data, byteSize); + } + + public static MapColumn from(Map data) { + return new MapColumn(data, 0); + } + @Override public String asString() { if (null == data) { diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/ArrayColumn.java b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/SqlArrayColumn.java similarity index 95% rename from chunjun-core/src/main/java/com/dtstack/chunjun/element/column/ArrayColumn.java rename to chunjun-core/src/main/java/com/dtstack/chunjun/element/column/SqlArrayColumn.java index c9e5ffb215..c02825286c 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/ArrayColumn.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/SqlArrayColumn.java @@ -36,12 +36,12 @@ * @author shitou * @date 2022/4/15 */ -public class ArrayColumn extends AbstractBaseColumn { +public class SqlArrayColumn extends AbstractBaseColumn { private static final long serialVersionUID = 1L; protected Array data; - public ArrayColumn(final Array data) { + public SqlArrayColumn(final Array data) { super(data, data.toString().length()); } diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/SqlDateColumn.java b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/SqlDateColumn.java index 5d8fc82223..df05c3220b 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/SqlDateColumn.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/SqlDateColumn.java @@ -33,10 +33,18 @@ public SqlDateColumn(Date data) { super(data, 8); } + public SqlDateColumn(Date data, int byteSize) { + super(data, 0); + } + public SqlDateColumn(long data) { super(Date.valueOf(LocalDate.ofEpochDay(data)), 8); } + public static SqlDateColumn from(Date date) { + return new SqlDateColumn(date, 0); + } + @Override public Boolean asBoolean() { if (null == data) { diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/StringColumn.java b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/StringColumn.java index 4cabfa2586..50ee177e30 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/StringColumn.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/StringColumn.java @@ -58,6 +58,16 @@ public StringColumn(final String data, String format) { byteSize += getStringSize(data); } + public StringColumn(String data, String format, boolean isCustomFormat, int byteSize) { + super(data, byteSize); + this.format = format; + this.isCustomFormat = isCustomFormat; + } + + public static StringColumn from(final String data, String format, boolean isCustomFormat) { + return new StringColumn(data, format, isCustomFormat, 0); + } + public StringColumn(Byte aByte) { super(aByte, 0); byteSize += 1; @@ -256,4 +266,12 @@ public String asTimestampStr() { public boolean isCustomFormat() { return isCustomFormat; } + + public String getFormat() { + return format; + } + + public void setFormat(String format) { + this.format = format; + } } diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/TimeColumn.java b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/TimeColumn.java index 2a18cc2ee9..f27256f23b 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/TimeColumn.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/TimeColumn.java @@ -34,6 +34,14 @@ public TimeColumn(Time data) { super(data, 8); } + public TimeColumn(Time data, int byteSize) { + super(data, byteSize); + } + + public static TimeColumn from(Time data) { + return new TimeColumn(data, 0); + } + public TimeColumn(int data) { super(Time.valueOf(LocalTime.ofNanoOfDay(data * 1_000_000L)), 8); } diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/TimestampColumn.java b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/TimestampColumn.java index da6c618099..6da01d7df8 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/TimestampColumn.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/element/column/TimestampColumn.java @@ -58,16 +58,34 @@ public TimestampColumn(Timestamp data, int precision) { this.precision = precision; } + public TimestampColumn(Timestamp data, int precision, int byteSize) { + super(data, byteSize); + this.precision = precision; + } + public TimestampColumn(long data, int precision) { super(new Timestamp(data), 8); this.precision = precision; } + public TimestampColumn(long data, int precision, int byteSize) { + super(new Timestamp(data), byteSize); + this.precision = precision; + } + public TimestampColumn(Date data, int precision) { super(new Timestamp(data.getTime()), 8); this.precision = precision; } + public static TimestampColumn from(long data, int precision) { + return new TimestampColumn(data, precision, 0); + } + + public static TimestampColumn from(Timestamp data, int precision) { + return new TimestampColumn(data, precision, 0); + } + @Override public Boolean asBoolean() { if (null == data) { diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/enums/ColumnType.java b/chunjun-core/src/main/java/com/dtstack/chunjun/enums/ColumnType.java index d6118f413d..db8ce81988 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/enums/ColumnType.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/enums/ColumnType.java @@ -141,6 +141,7 @@ public enum ColumnType { DECFLOAT, DECIMAL, NUMBER, + NUMERIC, PRECISION, DOUBLEPRECISION); diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/metrics/RowSizeCalculator.java b/chunjun-core/src/main/java/com/dtstack/chunjun/metrics/RowSizeCalculator.java index 747a574b12..0496a67ccf 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/metrics/RowSizeCalculator.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/metrics/RowSizeCalculator.java @@ -53,18 +53,18 @@ public static RowSizeCalculator getRowSizeCalculator( * if jdk support,use {@link jdk.nashorn.internal.ir.debug.ObjectSizeCalculator} else use * toString().getBytes().length * + *

ObjectSizeCalculator is in jre/lib/ext/nashorn.jar,In order to be able to determine + * correctly in different JDK, we call it directly once + * * @return RowSizeCalculator */ public static RowSizeCalculator getRowSizeCalculator() { - String vmName = System.getProperty("java.vm.name"); - String dataModel = System.getProperty("sun.arch.data.model"); - if (vmName != null - && (vmName.startsWith("OpenJDK ") || vmName.startsWith("Java HotSpot(TM) "))) { - if ("32".equals(dataModel) || "64".equals(dataModel)) { - return new RowObjectSizeCalculator(); - } + try { + ObjectSizeCalculator.getObjectSize(""); + return new RowObjectSizeCalculator(); + } catch (Throwable e) { + return new RowToStringCalculator(); } - return new RowToStringCalculator(); } static class RowObjectSizeCalculator extends RowSizeCalculator { @@ -123,7 +123,7 @@ public static CalculatorType getCalculatorTypeByName(String name) { } throw new ChunJunRuntimeException( String.format( - "ChunJun CalculatorType only one of %s", + "ChunJun CalculatorType only support one of %s", Arrays.stream(CalculatorType.values()) .map(CalculatorType::getTypeName) .collect(Collectors.toList()))); diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/source/SourceFactory.java b/chunjun-core/src/main/java/com/dtstack/chunjun/source/SourceFactory.java index a442ca6196..1f487a9bf6 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/source/SourceFactory.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/source/SourceFactory.java @@ -38,7 +38,7 @@ import org.apache.commons.lang3.StringUtils; -import java.util.Collections; +import java.util.ArrayList; import java.util.List; /** @@ -59,13 +59,24 @@ public abstract class SourceFactory implements RawTypeConvertible { protected SourceFactory(SyncConf syncConf, StreamExecutionEnvironment env) { this.env = env; this.syncConf = syncConf; - - if (syncConf.getTransformer() == null - || StringUtils.isBlank(syncConf.getTransformer().getTransformSql())) { - fieldList = Collections.emptyList(); - } else { - fieldList = syncConf.getReader().getFieldList(); + List readerFiledConfList = syncConf.getReader().getFieldList(); + this.fieldList = new ArrayList<>(readerFiledConfList.size()); + fieldList.addAll(readerFiledConfList); + if (syncConf.getTransformer() != null + && !StringUtils.isBlank(syncConf.getTransformer().getTransformSql())) { useAbstractBaseColumn = false; + } else { + fieldList.forEach( + fieldConf -> { + /* + In a synchronous job, we don't care about the name field when + typeInformation is generated.To avoid name duplication/empty, use + indexValue + */ + if (fieldConf.getName() == null) { + fieldConf.setName(String.valueOf(fieldConf.getIndex())); + } + }); } } @@ -134,7 +145,9 @@ public void initCommonConf(ChunJunCommonConf commonConf) { protected TypeInformation getTypeInformation() { if (typeInformation == null) { - typeInformation = TableUtil.getTypeInformation(fieldList, getRawTypeConverter()); + typeInformation = + TableUtil.getTypeInformation( + fieldList, getRawTypeConverter(), useAbstractBaseColumn); } return typeInformation; } diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/ColumnRowDataTypeInfo.java b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/ColumnRowDataTypeInfo.java new file mode 100644 index 0000000000..bfa5a47421 --- /dev/null +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/ColumnRowDataTypeInfo.java @@ -0,0 +1,118 @@ +/* + * 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 com.dtstack.chunjun.typeutil; + +import com.dtstack.chunjun.typeutil.serializer.ColumnRowDataSerializer; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.Preconditions; + +import java.util.Objects; + +/** @author liuliu 2022/5/12 */ +public class ColumnRowDataTypeInfo extends TypeInformation { + + private static final String FORMAT = "%s(%s, %s)"; + + private final LogicalType type; + private final Class typeClass; + private final ColumnRowDataSerializer typeSerializer; + + public ColumnRowDataTypeInfo( + LogicalType type, Class typeClass, ColumnRowDataSerializer typeSerializer) { + this.type = Preconditions.checkNotNull(type); + this.typeClass = Preconditions.checkNotNull(typeClass); + this.typeSerializer = Preconditions.checkNotNull(typeSerializer); + } + + public static ColumnRowDataTypeInfo of(RowType type) { + return new ColumnRowDataTypeInfo<>(type, RowData.class, new ColumnRowDataSerializer(type)); + } + + @Override + public boolean isBasicType() { + return false; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 1; + } + + @Override + public int getTotalFields() { + return 1; + } + + @Override + public Class getTypeClass() { + return typeClass; + } + + @Override + public boolean isKeyType() { + return false; + } + + @Override + public TypeSerializer createSerializer(ExecutionConfig config) { + return (TypeSerializer) typeSerializer; + } + + @Override + public String toString() { + return String.format( + FORMAT, + type.asSummaryString(), + typeClass.getName(), + typeSerializer.getClass().getName()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final ColumnRowDataTypeInfo that = (ColumnRowDataTypeInfo) o; + return typeSerializer.equals(that.typeSerializer); + } + + @Override + public int hashCode() { + return Objects.hash(typeSerializer); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof ColumnRowDataTypeInfo; + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/AbstractColumnSerializerUtil.java b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/AbstractColumnSerializerUtil.java new file mode 100644 index 0000000000..48de0123ff --- /dev/null +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/AbstractColumnSerializerUtil.java @@ -0,0 +1,100 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.typeutil.serializer.base.BooleanColumnSerializer; +import com.dtstack.chunjun.typeutil.serializer.base.ByteColumnSerializer; +import com.dtstack.chunjun.typeutil.serializer.base.BytesColumnSerializer; +import com.dtstack.chunjun.typeutil.serializer.base.DecimalColumnSerializer; +import com.dtstack.chunjun.typeutil.serializer.base.DoubleColumnSerializer; +import com.dtstack.chunjun.typeutil.serializer.base.FloatColumnSerializer; +import com.dtstack.chunjun.typeutil.serializer.base.IntColumnSerializer; +import com.dtstack.chunjun.typeutil.serializer.base.LongColumnSerializer; +import com.dtstack.chunjun.typeutil.serializer.base.MapColumnSerializer; +import com.dtstack.chunjun.typeutil.serializer.base.NullColumnSerializer; +import com.dtstack.chunjun.typeutil.serializer.base.ShortColumnSerializer; +import com.dtstack.chunjun.typeutil.serializer.base.SqlDateColumnSerializer; +import com.dtstack.chunjun.typeutil.serializer.base.StringColumnSerializer; +import com.dtstack.chunjun.typeutil.serializer.base.TimeColumnSerializer; +import com.dtstack.chunjun.typeutil.serializer.base.TimestampColumnSerializer; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.types.logical.LogicalType; + +/** @author liuliu 2022/5/12 */ +public class AbstractColumnSerializerUtil { + + /** + * Creates a TypeSerializer for internal data structures of the given LogicalType and + * descriptor. + */ + public static TypeSerializer getTypeSerializer( + LogicalType logicalType, String descriptor) { + switch (logicalType.getTypeRoot()) { + case CHAR: + case VARCHAR: + case STRUCTURED_TYPE: + return new StringColumnSerializer(descriptor); + case BOOLEAN: + return BooleanColumnSerializer.INSTANCE; + case BINARY: + case VARBINARY: + return BytesColumnSerializer.INSTANCE; + case DECIMAL: + return DecimalColumnSerializer.INSTANCE; + case TINYINT: + return ByteColumnSerializer.INSTANCE; + case SMALLINT: + return ShortColumnSerializer.INSTANCE; + case INTERVAL_YEAR_MONTH: + case INTEGER: + return IntColumnSerializer.INSTANCE; + case DATE: + return SqlDateColumnSerializer.INSTANCE; + case TIME_WITHOUT_TIME_ZONE: + return TimeColumnSerializer.INSTANCE; + case BIGINT: + case INTERVAL_DAY_TIME: + return LongColumnSerializer.INSTANCE; + case FLOAT: + return FloatColumnSerializer.INSTANCE; + case DOUBLE: + return DoubleColumnSerializer.INSTANCE; + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return TimestampColumnSerializer.INSTANCE; + case MAP: + return MapColumnSerializer.INSTANCE; + case NULL: + return NullColumnSerializer.INSTANCE; + case TIMESTAMP_WITH_TIME_ZONE: + case ARRAY: + case MULTISET: + case ROW: + case DISTINCT_TYPE: + case RAW: + case SYMBOL: + case UNRESOLVED: + default: + throw new UnsupportedOperationException( + "Unsupported type '" + logicalType + "' to get internal serializer"); + } + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/ColumnRowDataSerializer.java b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/ColumnRowDataSerializer.java new file mode 100644 index 0000000000..084894c156 --- /dev/null +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/ColumnRowDataSerializer.java @@ -0,0 +1,315 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.ColumnRowData; + +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerUtil; +import org.apache.flink.api.common.typeutils.NestedSerializersSnapshotDelegate; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream; +import org.apache.flink.api.java.typeutils.runtime.DataOutputViewStream; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.InstantiationUtil; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +/** @author liuliu 2022/5/12 */ +public class ColumnRowDataSerializer extends TypeSerializer { + + StringSerializer stringSerializer = StringSerializer.INSTANCE; + int size; + + private final LogicalType[] types; + private final TypeSerializer[] fieldSerializers; + + @SuppressWarnings("unchecked") + public ColumnRowDataSerializer(RowType rowType) { + this( + rowType.getChildren().toArray(new LogicalType[0]), + rowType.getFields().stream() + .map( + field -> + AbstractColumnSerializerUtil.getTypeSerializer( + field.getType(), + field.getDescription().isPresent() + ? field.getDescription().get() + : "")) + .toArray(TypeSerializer[]::new)); + } + + public ColumnRowDataSerializer( + LogicalType[] types, TypeSerializer[] fieldSerializers) { + this.types = types; + this.fieldSerializers = fieldSerializers; + this.size = fieldSerializers.length; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + TypeSerializer[] duplicateFieldSerializers = + new TypeSerializer[fieldSerializers.length]; + for (int i = 0; i < fieldSerializers.length; i++) { + duplicateFieldSerializers[i] = fieldSerializers[i].duplicate(); + } + return new ColumnRowDataSerializer(types, duplicateFieldSerializers); + } + + @Override + public ColumnRowData createInstance() { + return new ColumnRowData(fieldSerializers.length); + } + + @Override + public RowData copy(RowData from) { + return from; + } + + @Override + public RowData copy(RowData from, RowData reuse) { + return from; + } + + @Override + public int getLength() { + return -1; + } + + /** rowKind + headerInfoSize + headerInfo + extHeaderSize + extHeader + fields */ + @Override + public void serialize(RowData record, DataOutputView target) throws IOException { + ColumnRowData columnRowData = (ColumnRowData) record; + target.writeByte(record.getRowKind().toByteValue()); + target.writeInt(columnRowData.getByteSize()); + + if (columnRowData.getHeaderInfo() == null) { + target.writeInt(-1); + } else { + Map headerInfo = columnRowData.getHeaderInfo(); + target.writeInt(headerInfo.size()); + for (Map.Entry entry : headerInfo.entrySet()) { + stringSerializer.serialize(entry.getKey(), target); + if (entry.getValue() == null) { + target.writeBoolean(false); + } else { + target.writeBoolean(true); + target.writeInt(entry.getValue()); + } + } + } + + Set extHeader = columnRowData.getExtHeader(); + target.writeInt(extHeader.size()); + for (String ext : columnRowData.getExtHeader()) { + stringSerializer.serialize(ext, target); + } + + for (int i = 0; i < size; i++) { + fieldSerializers[i].serialize(columnRowData.getField(i), target); + } + } + + /** rowKind + headerInfoSize + headerInfo + extHeaderSize + extHeader + fields */ + @Override + public ColumnRowData deserialize(DataInputView source) throws IOException { + RowKind rowKind = RowKind.fromByteValue(source.readByte()); + int byteSize = source.readInt(); + ColumnRowData columnRowData = new ColumnRowData(rowKind, fieldSerializers.length, byteSize); + + int infoSize = source.readInt(); + if (infoSize >= 0) { + final Map headerInfo = new HashMap<>(infoSize); + for (int i = 0; i < infoSize; i++) { + String key = stringSerializer.deserialize(source); + boolean isNotNull = source.readBoolean(); + Integer value = isNotNull ? source.readInt() : null; + headerInfo.put(key, value); + } + columnRowData.setHeader(headerInfo); + } + + int extHeaderSize = source.readInt(); + Set extHeader = columnRowData.getExtHeader(); + for (int i = 0; i < extHeaderSize; i++) { + extHeader.add(stringSerializer.deserialize(source)); + } + + for (TypeSerializer typeSerializer : fieldSerializers) { + columnRowData.addFieldWithOutByteSize(typeSerializer.deserialize(source)); + } + + return columnRowData; + } + + @Override + public ColumnRowData deserialize(RowData reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + target.writeByte(source.readByte()); + target.writeInt(source.readInt()); + int infoSize = source.readInt(); + target.writeInt(infoSize); + for (int i = 0; i < infoSize; i++) { + stringSerializer.serialize(stringSerializer.deserialize(source), target); + boolean isNotNull = source.readBoolean(); + target.writeBoolean(isNotNull); + if (isNotNull) { + target.writeInt(source.readInt()); + } + } + int extHeaderSize = source.readInt(); + target.writeInt(extHeaderSize); + for (int i = 0; i < extHeaderSize; i++) { + stringSerializer.serialize(stringSerializer.deserialize(source), target); + } + for (TypeSerializer typeSerializer : fieldSerializers) { + typeSerializer.serialize(typeSerializer.deserialize(source), target); + } + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof ColumnRowDataSerializer) { + ColumnRowDataSerializer other = (ColumnRowDataSerializer) obj; + return Arrays.equals(fieldSerializers, other.fieldSerializers); + } + return false; + } + + @Override + public int hashCode() { + return Arrays.hashCode(fieldSerializers); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new ColumnRowDataSerializerSnapshot(types, fieldSerializers); + } + + public static final class ColumnRowDataSerializerSnapshot + implements TypeSerializerSnapshot { + private static final int CURRENT_VERSION = 3; + + private LogicalType[] previousTypes; + private NestedSerializersSnapshotDelegate nestedSerializersSnapshotDelegate; + + @SuppressWarnings("unused") + public ColumnRowDataSerializerSnapshot() { + // this constructor is used when restoring from a checkpoint/savepoint. + } + + ColumnRowDataSerializerSnapshot(LogicalType[] types, TypeSerializer[] serializers) { + this.previousTypes = types; + this.nestedSerializersSnapshotDelegate = + new NestedSerializersSnapshotDelegate(serializers); + } + + @Override + public int getCurrentVersion() { + return CURRENT_VERSION; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + out.writeInt(previousTypes.length); + DataOutputViewStream stream = new DataOutputViewStream(out); + for (LogicalType previousType : previousTypes) { + InstantiationUtil.serializeObject(stream, previousType); + } + nestedSerializersSnapshotDelegate.writeNestedSerializerSnapshots(out); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) + throws IOException { + int length = in.readInt(); + DataInputViewStream stream = new DataInputViewStream(in); + previousTypes = new LogicalType[length]; + for (int i = 0; i < length; i++) { + try { + previousTypes[i] = + InstantiationUtil.deserializeObject(stream, userCodeClassLoader); + } catch (ClassNotFoundException e) { + throw new IOException(e); + } + } + this.nestedSerializersSnapshotDelegate = + NestedSerializersSnapshotDelegate.readNestedSerializerSnapshots( + in, userCodeClassLoader); + } + + @Override + public ColumnRowDataSerializer restoreSerializer() { + return new ColumnRowDataSerializer( + previousTypes, + (TypeSerializer[]) + nestedSerializersSnapshotDelegate.getRestoredNestedSerializers()); + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializer newSerializer) { + if (!(newSerializer instanceof ColumnRowDataSerializer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + ColumnRowDataSerializer newColumnRowSerializer = + (ColumnRowDataSerializer) newSerializer; + if (!Arrays.equals(previousTypes, newColumnRowSerializer.types)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + CompositeTypeSerializerUtil.IntermediateCompatibilityResult + intermediateResult = + CompositeTypeSerializerUtil.constructIntermediateCompatibilityResult( + newColumnRowSerializer.fieldSerializers, + nestedSerializersSnapshotDelegate + .getNestedSerializerSnapshots()); + + if (intermediateResult.isCompatibleWithReconfiguredSerializer()) { + ColumnRowDataSerializer reconfiguredCompositeSerializer = restoreSerializer(); + return TypeSerializerSchemaCompatibility.compatibleWithReconfiguredSerializer( + reconfiguredCompositeSerializer); + } + + return intermediateResult.getFinalResult(); + } + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/BooleanColumnSerializer.java b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/BooleanColumnSerializer.java new file mode 100644 index 0000000000..e5185aec2b --- /dev/null +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/BooleanColumnSerializer.java @@ -0,0 +1,137 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.BooleanColumn; +import com.dtstack.chunjun.element.column.BytesColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; + +/** @author liuliu 2022/5/12 */ +public class BooleanColumnSerializer extends TypeSerializerSingleton { + + /** Sharable instance of the BooleanColumnSerializer. */ + public static final BooleanColumnSerializer INSTANCE = new BooleanColumnSerializer(); + + private static final BooleanColumn FALSE = new BooleanColumn(false); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public AbstractBaseColumn createInstance() { + return FALSE; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from) { + return from; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from, AbstractBaseColumn reuse) { + return from; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(AbstractBaseColumn record, DataOutputView target) throws IOException { + if (record == null || record instanceof NullColumn) { + target.writeInt(0); + } else if (record instanceof BytesColumn) { + target.writeInt(2); + byte[] bytes = record.asBytes(); + target.writeInt(bytes.length); + target.write(bytes); + } else { + target.writeInt(1); + target.writeBoolean(record.asBoolean()); + } + } + + @Override + public AbstractBaseColumn deserialize(DataInputView source) throws IOException { + int value = source.readInt(); + switch (value) { + case 0: + return new NullColumn(); + case 1: + return BooleanColumn.from(source.readBoolean()); + case 2: + int len = source.readInt(); + byte[] result = new byte[len]; + source.readFully(result); + return BytesColumn.from(result); + default: + // you should not be here + throw new ChunJunRuntimeException(""); + } + } + + @Override + public AbstractBaseColumn deserialize(AbstractBaseColumn reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + int type = source.readInt(); + target.writeInt(type); + if (type == 1) { + target.writeBoolean(source.readBoolean()); + } else if (type == 2) { + int len = source.readInt(); + byte[] result = new byte[len]; + source.readFully(result); + target.writeInt(len); + target.write(result); + } + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new BooleanColumnSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class BooleanColumnSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public BooleanColumnSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/ByteColumnSerializer.java b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/ByteColumnSerializer.java new file mode 100644 index 0000000000..cd1461c15b --- /dev/null +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/ByteColumnSerializer.java @@ -0,0 +1,143 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.ByteColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; + +/** @author liuliu 2022/5/12 */ +public class ByteColumnSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + /** Sharable instance of the ByteColumnSerializer. */ + public static final ByteColumnSerializer INSTANCE = new ByteColumnSerializer(); + + private static final ByteColumn EMPTY = new ByteColumn((byte) 0); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public AbstractBaseColumn createInstance() { + return EMPTY; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from) { + return from; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from, AbstractBaseColumn reuse) { + return from; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(AbstractBaseColumn record, DataOutputView target) throws IOException { + if (record == null || record instanceof NullColumn) { + target.writeInt(0); + } else if (record instanceof BigDecimalColumn) { + target.writeInt(2); + BigDecimal bigDecimal = record.asBigDecimal(); + byte[] bytes = bigDecimal.unscaledValue().toByteArray(); + target.writeInt(bytes.length); + target.write(bytes); + target.writeInt(bigDecimal.scale()); + } else { + target.writeInt(1); + target.writeByte((byte) record.getData()); + } + } + + @Override + public AbstractBaseColumn deserialize(DataInputView source) throws IOException { + int type = source.readInt(); + switch (type) { + case 0: + return new NullColumn(); + case 1: + return ByteColumn.from(source.readByte()); + case 2: + int length = source.readInt(); + byte[] bytes = new byte[length]; + source.readFully(bytes); + int scale = source.readInt(); + return BigDecimalColumn.from(new BigDecimal(new BigInteger(bytes), scale)); + default: + // you should not be here + throw new ChunJunRuntimeException(""); + } + } + + @Override + public AbstractBaseColumn deserialize(AbstractBaseColumn reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + int type = source.readInt(); + target.writeInt(type); + if (type == 1) { + target.writeByte(source.readByte()); + } else if (type == 2) { + int len = source.readInt(); + target.writeInt(len); + target.write(source, len); + target.writeInt(source.readInt()); + } + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new ByteColumnSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class ByteColumnSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public ByteColumnSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/BytesColumnSerializer.java b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/BytesColumnSerializer.java new file mode 100644 index 0000000000..050c5f45bb --- /dev/null +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/BytesColumnSerializer.java @@ -0,0 +1,132 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.BytesColumn; +import com.dtstack.chunjun.element.column.NullColumn; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; + +/** @author liuliu 2022/5/12 */ +public class BytesColumnSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + /** Sharable instance of the BytesColumnSerializer. */ + public static final BytesColumnSerializer INSTANCE = new BytesColumnSerializer(); + + private static final BytesColumn EMPTY = new BytesColumn(new byte[0]); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public AbstractBaseColumn createInstance() { + return EMPTY; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from) { + if (from == null || from instanceof NullColumn) { + return new NullColumn(); + } + byte[] bytes = from.asBytes(); + byte[] copy = new byte[bytes.length]; + System.arraycopy(bytes, 0, copy, 0, bytes.length); + return new BytesColumn(copy); + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from, AbstractBaseColumn reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(AbstractBaseColumn record, DataOutputView target) throws IOException { + if (record == null || record instanceof NullColumn) { + target.writeBoolean(false); + } else { + target.writeBoolean(true); + byte[] bytes = record.asBytes(); + target.writeInt(bytes.length); + target.write(bytes); + } + } + + @Override + public AbstractBaseColumn deserialize(DataInputView source) throws IOException { + boolean isNotNull = source.readBoolean(); + if (isNotNull) { + final int len = source.readInt(); + byte[] result = new byte[len]; + source.readFully(result); + return BytesColumn.from(result); + } else { + return new NullColumn(); + } + } + + @Override + public AbstractBaseColumn deserialize(AbstractBaseColumn reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + boolean isNotNull = source.readBoolean(); + target.writeBoolean(isNotNull); + if (isNotNull) { + int len = source.readInt(); + byte[] result = new byte[len]; + source.readFully(result); + target.writeInt(len); + target.write(result); + } + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new BytesColumnSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class BytesColumnSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public BytesColumnSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/DecimalColumnSerializer.java b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/DecimalColumnSerializer.java new file mode 100644 index 0000000000..d2f74caa9f --- /dev/null +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/DecimalColumnSerializer.java @@ -0,0 +1,129 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.NullColumn; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; + +/** @author liuliu 2022/5/12 */ +public class DecimalColumnSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + public static final DecimalColumnSerializer INSTANCE = new DecimalColumnSerializer(); + + private static final BigDecimalColumn EMPTY = new BigDecimalColumn(0); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public AbstractBaseColumn createInstance() { + return EMPTY; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from) { + return from; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from, AbstractBaseColumn reuse) { + return from; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(AbstractBaseColumn record, DataOutputView target) throws IOException { + if (record == null || record instanceof NullColumn) { + target.writeBoolean(false); + } else { + target.writeBoolean(true); + BigDecimal bigDecimal = record.asBigDecimal(); + byte[] bytes = bigDecimal.unscaledValue().toByteArray(); + target.writeInt(bytes.length); + target.write(bytes); + target.writeInt(bigDecimal.scale()); + } + } + + @Override + public AbstractBaseColumn deserialize(DataInputView source) throws IOException { + boolean isNotNull = source.readBoolean(); + if (isNotNull) { + int length = source.readInt(); + byte[] bytes = new byte[length]; + source.readFully(bytes); + int scale = source.readInt(); + return BigDecimalColumn.from(new BigDecimal(new BigInteger(bytes), scale)); + } else { + return new NullColumn(); + } + } + + @Override + public AbstractBaseColumn deserialize(AbstractBaseColumn reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + boolean isNotNull = source.readBoolean(); + target.writeBoolean(isNotNull); + if (isNotNull) { + int len = source.readInt(); + target.writeInt(len); + target.write(source, len); + target.writeInt(source.readInt()); + } + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new DecimalColumnSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class DecimalColumnSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public DecimalColumnSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/DoubleColumnSerializer.java b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/DoubleColumnSerializer.java new file mode 100644 index 0000000000..d471cd67ea --- /dev/null +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/DoubleColumnSerializer.java @@ -0,0 +1,118 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.NullColumn; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; +import java.math.BigDecimal; + +/** @author liuliu 2022/5/12 */ +public class DoubleColumnSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + /** Sharable instance of the DoubleColumnSerializer. */ + public static final DoubleColumnSerializer INSTANCE = new DoubleColumnSerializer(); + + private static final BigDecimalColumn EMPTY = new BigDecimalColumn(0); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public AbstractBaseColumn createInstance() { + return EMPTY; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from) { + return from; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from, AbstractBaseColumn reuse) { + return from; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(AbstractBaseColumn record, DataOutputView target) throws IOException { + if (record == null || record instanceof NullColumn) { + target.writeBoolean(false); + } else { + target.writeBoolean(true); + target.writeDouble(record.asDouble()); + } + } + + @Override + public AbstractBaseColumn deserialize(DataInputView source) throws IOException { + boolean isNotNull = source.readBoolean(); + if (isNotNull) { + return BigDecimalColumn.from(BigDecimal.valueOf(source.readDouble())); + } else { + return new NullColumn(); + } + } + + @Override + public AbstractBaseColumn deserialize(AbstractBaseColumn reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + boolean isNotNull = source.readBoolean(); + target.writeBoolean(isNotNull); + if (isNotNull) { + target.writeDouble(source.readDouble()); + } + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new DoubleColumnSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class DoubleColumnSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public DoubleColumnSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/FloatColumnSerializer.java b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/FloatColumnSerializer.java new file mode 100644 index 0000000000..a0842fc4c6 --- /dev/null +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/FloatColumnSerializer.java @@ -0,0 +1,119 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.NullColumn; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; +import java.math.BigDecimal; + +/** @author liuliu 2022/5/12 */ +public class FloatColumnSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + /** Sharable instance of the FloatColumnSerializer. */ + public static final FloatColumnSerializer INSTANCE = new FloatColumnSerializer(); + + private static final BigDecimalColumn EMPTY = new BigDecimalColumn(0); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public AbstractBaseColumn createInstance() { + return EMPTY; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from) { + return from; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from, AbstractBaseColumn reuse) { + return from; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(AbstractBaseColumn record, DataOutputView target) throws IOException { + if (record == null || record instanceof NullColumn) { + target.writeBoolean(false); + } else { + target.writeBoolean(true); + Float aFloat = record.asFloat(); + target.writeFloat(aFloat); + } + } + + @Override + public AbstractBaseColumn deserialize(DataInputView source) throws IOException { + boolean isNotNull = source.readBoolean(); + if (isNotNull) { + return BigDecimalColumn.from(new BigDecimal(String.valueOf(source.readFloat()))); + } else { + return new NullColumn(); + } + } + + @Override + public AbstractBaseColumn deserialize(AbstractBaseColumn reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + boolean isNotNull = source.readBoolean(); + target.writeBoolean(isNotNull); + if (isNotNull) { + target.writeFloat(source.readFloat()); + } + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new FloatColumnSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class FloatColumnSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public FloatColumnSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/IntColumnSerializer.java b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/IntColumnSerializer.java new file mode 100644 index 0000000000..2f18bcb2e6 --- /dev/null +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/IntColumnSerializer.java @@ -0,0 +1,118 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.NullColumn; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; +import java.math.BigDecimal; + +/** @author liuliu 2022/5/12 */ +public class IntColumnSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + /** Sharable instance of the IntColumnSerializer. */ + public static final IntColumnSerializer INSTANCE = new IntColumnSerializer(); + + private static final BigDecimalColumn EMPTY = new BigDecimalColumn(0); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public AbstractBaseColumn createInstance() { + return EMPTY; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from) { + return from; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from, AbstractBaseColumn reuse) { + return from; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(AbstractBaseColumn record, DataOutputView target) throws IOException { + if (record == null || record instanceof NullColumn) { + target.writeBoolean(false); + } else { + target.writeBoolean(true); + target.writeInt(record.asInt()); + } + } + + @Override + public AbstractBaseColumn deserialize(DataInputView source) throws IOException { + boolean isNotNull = source.readBoolean(); + if (isNotNull) { + return BigDecimalColumn.from(BigDecimal.valueOf(source.readInt())); + } else { + return new NullColumn(); + } + } + + @Override + public AbstractBaseColumn deserialize(AbstractBaseColumn reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + boolean isNotNull = source.readBoolean(); + target.writeBoolean(isNotNull); + if (isNotNull) { + target.writeInt(source.readInt()); + } + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new IntColumnSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class IntColumnSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public IntColumnSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/LongColumnSerializer.java b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/LongColumnSerializer.java new file mode 100644 index 0000000000..368f4c3b44 --- /dev/null +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/LongColumnSerializer.java @@ -0,0 +1,118 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.NullColumn; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; +import java.math.BigDecimal; + +/** @author liuliu 2022/5/12 */ +public class LongColumnSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + /** Sharable instance of the LongColumnSerializer. */ + public static final LongColumnSerializer INSTANCE = new LongColumnSerializer(); + + private static final BigDecimalColumn EMPTY = new BigDecimalColumn(0); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public AbstractBaseColumn createInstance() { + return EMPTY; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from) { + return from; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from, AbstractBaseColumn reuse) { + return from; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(AbstractBaseColumn record, DataOutputView target) throws IOException { + if (record == null || record instanceof NullColumn) { + target.writeBoolean(false); + } else { + target.writeBoolean(true); + target.writeLong(record.asLong()); + } + } + + @Override + public AbstractBaseColumn deserialize(DataInputView source) throws IOException { + boolean isNotNull = source.readBoolean(); + if (isNotNull) { + return BigDecimalColumn.from(BigDecimal.valueOf(source.readLong())); + } else { + return new NullColumn(); + } + } + + @Override + public AbstractBaseColumn deserialize(AbstractBaseColumn reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + boolean isNotNull = source.readBoolean(); + target.writeBoolean(isNotNull); + if (isNotNull) { + target.writeLong(source.readLong()); + } + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new LongColumnSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class LongColumnSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public LongColumnSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/MapColumnSerializer.java b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/MapColumnSerializer.java new file mode 100644 index 0000000000..d698f1432a --- /dev/null +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/MapColumnSerializer.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.MapColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.util.JsonUtil; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; +import java.util.Map; + +/** @author liuliu 2022/5/25 */ +public class MapColumnSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + /** Sharable instance of the MapColumnSerializer. */ + public static final MapColumnSerializer INSTANCE = new MapColumnSerializer(); + + private final StringSerializer stringSerializer = StringSerializer.INSTANCE; + + private static final MapColumn EMPTY = new MapColumn(null); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public AbstractBaseColumn createInstance() { + return EMPTY; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from) { + return from; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from, AbstractBaseColumn reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(AbstractBaseColumn record, DataOutputView target) throws IOException { + if (record == null || record instanceof NullColumn) { + target.writeBoolean(false); + } else { + target.writeBoolean(true); + stringSerializer.serialize(JsonUtil.toJson(record.getData()), target); + } + } + + @Override + public AbstractBaseColumn deserialize(DataInputView source) throws IOException { + boolean isNotNull = source.readBoolean(); + if (isNotNull) { + return MapColumn.from( + JsonUtil.toObject(stringSerializer.deserialize(source), Map.class)); + } else { + return new NullColumn(); + } + } + + @Override + public AbstractBaseColumn deserialize(AbstractBaseColumn reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + boolean isNotNull = source.readBoolean(); + target.writeBoolean(isNotNull); + if (isNotNull) { + stringSerializer.copy(source, target); + } + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new MapColumnSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class MapColumnSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public MapColumnSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/NullColumnSerializer.java b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/NullColumnSerializer.java new file mode 100644 index 0000000000..74d2ae97ba --- /dev/null +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/NullColumnSerializer.java @@ -0,0 +1,103 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.NullColumn; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; + +/** @author liuliu 2022/5/12 */ +public class NullColumnSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + /** Sharable instance of the NullColumnSerializer. */ + public static final NullColumnSerializer INSTANCE = new NullColumnSerializer(); + + private static final NullColumn EMPTY = new NullColumn(); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public AbstractBaseColumn createInstance() { + return EMPTY; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from) { + return from; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from, AbstractBaseColumn reuse) { + return from; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(AbstractBaseColumn record, DataOutputView target) throws IOException { + target.writeByte(0); + } + + @Override + public AbstractBaseColumn deserialize(DataInputView source) throws IOException { + source.readByte(); + return EMPTY; + } + + @Override + public AbstractBaseColumn deserialize(AbstractBaseColumn reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + target.writeByte(0); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new NullColumnSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class NullColumnSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public NullColumnSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/ShortColumnSerializer.java b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/ShortColumnSerializer.java new file mode 100644 index 0000000000..d05766caeb --- /dev/null +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/ShortColumnSerializer.java @@ -0,0 +1,118 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.NullColumn; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; +import java.math.BigDecimal; + +/** @author liuliu 2022/5/12 */ +public class ShortColumnSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + /** Sharable instance of the ShortColumnSerializer. */ + public static final ShortColumnSerializer INSTANCE = new ShortColumnSerializer(); + + private static final BigDecimalColumn EMPTY = new BigDecimalColumn(0); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public AbstractBaseColumn createInstance() { + return EMPTY; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from) { + return from; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from, AbstractBaseColumn reuse) { + return from; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(AbstractBaseColumn record, DataOutputView target) throws IOException { + if (record == null || record instanceof NullColumn) { + target.writeBoolean(false); + } else { + target.writeBoolean(true); + target.writeShort(record.asShort()); + } + } + + @Override + public AbstractBaseColumn deserialize(DataInputView source) throws IOException { + boolean isNotNull = source.readBoolean(); + if (isNotNull) { + return BigDecimalColumn.from(BigDecimal.valueOf(source.readShort())); + } else { + return new NullColumn(); + } + } + + @Override + public AbstractBaseColumn deserialize(AbstractBaseColumn reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + boolean isNotNull = source.readBoolean(); + target.writeBoolean(isNotNull); + if (isNotNull) { + target.writeShort(source.readShort()); + } + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new ShortColumnSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class ShortColumnSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public ShortColumnSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/SqlDateColumnSerializer.java b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/SqlDateColumnSerializer.java new file mode 100644 index 0000000000..2dd912f413 --- /dev/null +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/SqlDateColumnSerializer.java @@ -0,0 +1,138 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.element.column.SqlDateColumn; +import com.dtstack.chunjun.element.column.TimestampColumn; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; +import java.sql.Date; +import java.sql.Timestamp; + +/** @author liuliu 2022/5/12 */ +public class SqlDateColumnSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + /** Sharable instance of the SqlDateColumnSerializer. */ + public static final SqlDateColumnSerializer INSTANCE = new SqlDateColumnSerializer(); + + private static final SqlDateColumn EMPTY = new SqlDateColumn(0); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public AbstractBaseColumn createInstance() { + return EMPTY; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from) { + if (from instanceof NullColumn) { + return new NullColumn(); + } + if (from instanceof TimestampColumn) { + return new TimestampColumn( + ((Timestamp) from.getData()).getTime(), + ((TimestampColumn) from).getPrecision()); + } + return SqlDateColumn.from(from.asSqlDate()); + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from, AbstractBaseColumn reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(AbstractBaseColumn record, DataOutputView target) throws IOException { + if (record == null || record instanceof NullColumn) { + target.writeInt(0); + } else if (record instanceof TimestampColumn) { + target.writeInt(2); + target.writeLong(record.asTimestamp().getTime()); + } else { + target.writeInt(1); + target.writeLong(record.asSqlDate().getTime()); + } + } + + @Override + public AbstractBaseColumn deserialize(DataInputView source) throws IOException { + int type = source.readInt(); + switch (type) { + case 0: + return new NullColumn(); + case 1: + return SqlDateColumn.from(new Date(source.readLong())); + case 2: + return TimestampColumn.from(source.readLong(), 0); + default: + // you should not be here + throw new ChunJunRuntimeException(""); + } + } + + @Override + public AbstractBaseColumn deserialize(AbstractBaseColumn reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + int type = source.readInt(); + target.writeInt(type); + if (type != 0) { + target.writeLong(source.readLong()); + } + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new SqlDateColumnSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class SqlDateColumnSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public SqlDateColumnSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/StringColumnSerializer.java b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/StringColumnSerializer.java new file mode 100644 index 0000000000..7ff3764126 --- /dev/null +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/StringColumnSerializer.java @@ -0,0 +1,209 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.element.column.StringColumn; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.BooleanSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import org.apache.commons.lang.StringUtils; + +import java.io.IOException; + +/** @author liuliu 2022/5/12 */ +public class StringColumnSerializer extends TypeSerializer { + + private static final long serialVersionUID = 1L; + + private final StringSerializer stringSerializer = StringSerializer.INSTANCE; + private final BooleanSerializer booleanSerializer = BooleanSerializer.INSTANCE; + + private final String format; + private final boolean isCustomFormat; + + private static final StringColumn EMPTY = new StringColumn("", "", false, 0); + + public StringColumnSerializer(String format) { + if (StringUtils.isNotBlank(format)) { + this.format = format; + isCustomFormat = true; + } else { + this.format = "yyyy-MM-dd HH:mm:ss"; + isCustomFormat = false; + } + } + + public StringColumnSerializer(String format, boolean isCustomFormat) { + this.format = format; + this.isCustomFormat = isCustomFormat; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new StringColumnSerializer(format, isCustomFormat); + } + + @Override + public AbstractBaseColumn createInstance() { + return EMPTY; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from) { + return from; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from, AbstractBaseColumn reuse) { + return from; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(AbstractBaseColumn record, DataOutputView target) throws IOException { + if (record == null || record instanceof NullColumn) { + target.writeBoolean(false); + } else { + StringColumn column = (StringColumn) record; + target.writeBoolean(true); + stringSerializer.serialize((String) column.getData(), target); + } + } + + @Override + public AbstractBaseColumn deserialize(DataInputView source) throws IOException { + boolean isNotNull = source.readBoolean(); + if (isNotNull) { + return StringColumn.from(stringSerializer.deserialize(source), format, isCustomFormat); + } else { + return new NullColumn(); + } + } + + @Override + public AbstractBaseColumn deserialize(AbstractBaseColumn reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + boolean isNotNull = source.readBoolean(); + target.writeBoolean(isNotNull); + if (isNotNull) { + stringSerializer.copy(source, target); + } + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof StringColumnSerializer) { + StringColumnSerializer that = (StringColumnSerializer) obj; + return that.isCustomFormat == this.isCustomFormat && that.format.equals(this.format); + } + + return false; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new StringColumnSerializerSnapshot(format, isCustomFormat); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class StringColumnSerializerSnapshot + implements TypeSerializerSnapshot { + + private static final int CURRENT_VERSION = 3; + private static final StringSerializer stringSerializer = StringSerializer.INSTANCE; + private String format; + private boolean isCustomFormat; + + @SuppressWarnings("unused") + public StringColumnSerializerSnapshot() { + // this constructor is used when restoring from a checkpoint/savepoint. + } + + public StringColumnSerializerSnapshot(String format, boolean isCustomFormat) { + this.format = format; + this.isCustomFormat = isCustomFormat; + } + + @Override + public int getCurrentVersion() { + return CURRENT_VERSION; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + stringSerializer.serialize(format, out); + out.writeBoolean(isCustomFormat); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) + throws IOException { + this.format = stringSerializer.deserialize(in); + this.isCustomFormat = in.readBoolean(); + } + + @Override + public TypeSerializer restoreSerializer() { + return new StringColumnSerializer(format, isCustomFormat); + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializer newSerializer) { + if (!(newSerializer instanceof StringColumnSerializer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + StringColumnSerializer that = (StringColumnSerializer) newSerializer; + if (!format.equals(that.format) || isCustomFormat != that.isCustomFormat) { + return TypeSerializerSchemaCompatibility.incompatible(); + } else { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + } + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/TimeColumnSerializer.java b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/TimeColumnSerializer.java new file mode 100644 index 0000000000..dd7f0438f4 --- /dev/null +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/TimeColumnSerializer.java @@ -0,0 +1,124 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.element.column.TimeColumn; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; +import java.sql.Time; + +/** @author liuliu 2022/5/12 */ +public class TimeColumnSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + /** Sharable instance of the TimeColumnSerializer. */ + public static final TimeColumnSerializer INSTANCE = new TimeColumnSerializer(); + + private static final TimeColumn EMPTY = new TimeColumn(0); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public AbstractBaseColumn createInstance() { + return EMPTY; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from) { + if (from instanceof NullColumn) { + return new NullColumn(); + } + return TimeColumn.from(from.asTime()); + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from, AbstractBaseColumn reuse) { + if (from == null || from instanceof NullColumn) { + return new NullColumn(); + } + return TimeColumn.from(from.asTime()); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(AbstractBaseColumn record, DataOutputView target) throws IOException { + if (record == null || record instanceof NullColumn) { + target.writeBoolean(false); + } else { + target.writeBoolean(true); + target.writeLong(record.asTime().getTime()); + } + } + + @Override + public AbstractBaseColumn deserialize(DataInputView source) throws IOException { + boolean isNotNull = source.readBoolean(); + if (isNotNull) { + return TimeColumn.from(new Time(source.readLong())); + } else { + return new NullColumn(); + } + } + + @Override + public AbstractBaseColumn deserialize(AbstractBaseColumn reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + boolean isNotNull = source.readBoolean(); + target.writeBoolean(isNotNull); + if (isNotNull) { + target.writeLong(source.readLong()); + } + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new TimeColumnSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class TimeColumnSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public TimeColumnSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/TimestampColumnSerializer.java b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/TimestampColumnSerializer.java new file mode 100644 index 0000000000..bc249c85bb --- /dev/null +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/typeutil/serializer/base/TimestampColumnSerializer.java @@ -0,0 +1,141 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.element.column.TimestampColumn; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; +import java.sql.Timestamp; + +/** @author liuliu 2022/5/12 */ +public class TimestampColumnSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + /** Sharable instance of the TimestampColumnSerializer. */ + public static final TimestampColumnSerializer INSTANCE = new TimestampColumnSerializer(); + + private static final TimestampColumn EMPTY = new TimestampColumn(0); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public AbstractBaseColumn createInstance() { + return EMPTY; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from) { + return from; + } + + @Override + public AbstractBaseColumn copy(AbstractBaseColumn from, AbstractBaseColumn reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(AbstractBaseColumn record, DataOutputView target) throws IOException { + if (record == null || record instanceof NullColumn) { + target.writeBoolean(false); + } else { + target.writeBoolean(true); + TimestampColumn timestampColumn = (TimestampColumn) record; + Timestamp timestamp = timestampColumn.asTimestamp(); + target.writeLong(timestamp.getTime()); + int precision = timestampColumn.getPrecision(); + target.writeInt(precision); + if (isCompact(precision)) { + assert timestamp.getNanos() == 0; + } else { + target.writeInt(timestamp.getNanos()); + } + } + } + + @Override + public AbstractBaseColumn deserialize(DataInputView source) throws IOException { + boolean isNotNull = source.readBoolean(); + if (isNotNull) { + Timestamp timestamp = new Timestamp(source.readLong()); + int precision = source.readInt(); + if (!isCompact(precision)) { + timestamp.setNanos(source.readInt()); + } + return TimestampColumn.from(timestamp, precision); + } else { + return new NullColumn(); + } + } + + @Override + public AbstractBaseColumn deserialize(AbstractBaseColumn reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + boolean isNotNull = source.readBoolean(); + target.writeBoolean(isNotNull); + if (isNotNull) { + target.writeLong(source.readLong()); + int precision = source.readInt(); + target.writeInt(precision); + if (!isCompact(precision)) { + target.writeInt(source.readInt()); + } + } + } + + public boolean isCompact(int precision) { + return precision <= 3; + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new TimestampColumnSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class TimestampColumnSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public TimestampColumnSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/util/TableUtil.java b/chunjun-core/src/main/java/com/dtstack/chunjun/util/TableUtil.java index f0be0172f5..b30e5dd9db 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/util/TableUtil.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/util/TableUtil.java @@ -18,7 +18,9 @@ package com.dtstack.chunjun.util; import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.constants.ConstantValue; import com.dtstack.chunjun.converter.RawTypeConverter; +import com.dtstack.chunjun.typeutil.ColumnRowDataTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.GenericTypeInfo; @@ -28,10 +30,13 @@ import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; import org.apache.flink.table.types.logical.RowType; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Objects; import java.util.stream.Collectors; /** @@ -48,15 +53,17 @@ public class TableUtil { * @return TypeInformation */ public static TypeInformation getTypeInformation( - List fieldList, RawTypeConverter converter) { + List fieldList, RawTypeConverter converter, boolean useAbstractBaseColumn) { List fieldName = fieldList.stream().map(FieldConf::getName).collect(Collectors.toList()); - if (fieldName.size() == 0) { + String[] fieldTypes = fieldList.stream().map(FieldConf::getType).toArray(String[]::new); + String[] fieldFormat = fieldList.stream().map(FieldConf::getFormat).toArray(String[]::new); + String[] fieldNames = fieldList.stream().map(FieldConf::getName).toArray(String[]::new); + if (fieldName.size() == 0 + || fieldName.get(0).equalsIgnoreCase(ConstantValue.STAR_SYMBOL) + || Arrays.stream(fieldTypes).anyMatch(Objects::isNull)) { return new GenericTypeInfo<>(RowData.class); } - - String[] fieldNames = fieldList.stream().map(FieldConf::getName).toArray(String[]::new); - String[] fieldTypes = fieldList.stream().map(FieldConf::getType).toArray(String[]::new); TableSchema.Builder builder = TableSchema.builder(); for (int i = 0; i < fieldTypes.length; i++) { DataType dataType = converter.apply(fieldTypes[i]); @@ -65,7 +72,7 @@ public static TypeInformation getTypeInformation( DataType[] dataTypes = builder.build().toRowDataType().getChildren().toArray(new DataType[] {}); - return getTypeInformation(dataTypes, fieldNames); + return getTypeInformation(dataTypes, fieldNames, fieldFormat, useAbstractBaseColumn); } /** @@ -77,7 +84,37 @@ public static TypeInformation getTypeInformation( */ public static TypeInformation getTypeInformation( DataType[] dataTypes, String[] fieldNames) { - return InternalTypeInfo.of(getRowType(dataTypes, fieldNames)); + return getTypeInformation(dataTypes, fieldNames, new String[fieldNames.length], false); + } + + /** + * 获取TypeInformation + * + * @param dataTypes + * @param fieldNames + * @return + */ + public static TypeInformation getTypeInformation( + DataType[] dataTypes, + String[] fieldNames, + String[] fieldFormat, + boolean useAbstractBaseColumn) { + RowType rowType = getRowType(dataTypes, fieldNames, fieldFormat); + + if (useAbstractBaseColumn) { + if (useGenericTypeInfo(rowType)) { + return new GenericTypeInfo<>(RowData.class); + } + return ColumnRowDataTypeInfo.of(rowType); + } else { + return InternalTypeInfo.of(getRowType(dataTypes, fieldNames, fieldFormat)); + } + } + + public static boolean useGenericTypeInfo(RowType rowType) { + return rowType.getChildren().stream() + .map(LogicalType::getTypeRoot) + .anyMatch(logicalTypeRoot -> logicalTypeRoot == LogicalTypeRoot.ARRAY); } /** @@ -87,10 +124,16 @@ public static TypeInformation getTypeInformation( * @param fieldNames * @return */ - public static RowType getRowType(DataType[] dataTypes, String[] fieldNames) { - return RowType.of( - Arrays.stream(dataTypes).map(DataType::getLogicalType).toArray(LogicalType[]::new), - fieldNames); + public static RowType getRowType( + DataType[] dataTypes, String[] fieldNames, String[] formatField) { + List rowFieldList = new ArrayList<>(dataTypes.length); + for (int i = 0; i < dataTypes.length; i++) { + rowFieldList.add( + new RowType.RowField( + fieldNames[i], dataTypes[i].getLogicalType(), formatField[i])); + } + + return new RowType(rowFieldList); } /** diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/ColumnRowDataSerializerTest.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/ColumnRowDataSerializerTest.java new file mode 100644 index 0000000000..71e36770d5 --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/ColumnRowDataSerializerTest.java @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.typeutil.serializer; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.ColumnRowData; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.BooleanColumn; +import com.dtstack.chunjun.element.column.ByteColumn; +import com.dtstack.chunjun.element.column.BytesColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.element.column.SqlDateColumn; +import com.dtstack.chunjun.element.column.StringColumn; +import com.dtstack.chunjun.element.column.TimeColumn; +import com.dtstack.chunjun.element.column.TimestampColumn; +import com.dtstack.chunjun.typeutil.serializer.base.BooleanColumnSerializerTest; +import com.dtstack.chunjun.typeutil.serializer.base.ByteColumnSerializerTest; +import com.dtstack.chunjun.typeutil.serializer.base.BytesColumnSerializerTest; +import com.dtstack.chunjun.typeutil.serializer.base.DecimalColumnSerializerTest; +import com.dtstack.chunjun.typeutil.serializer.base.DoubleColumnSerializerTest; +import com.dtstack.chunjun.typeutil.serializer.base.FloatColumnSerializerTest; +import com.dtstack.chunjun.typeutil.serializer.base.IntColumnSerializerTest; +import com.dtstack.chunjun.typeutil.serializer.base.LongColumnSerializerTest; +import com.dtstack.chunjun.typeutil.serializer.base.NullColumnSerializerTest; +import com.dtstack.chunjun.typeutil.serializer.base.ShortColumnSerializerTest; +import com.dtstack.chunjun.typeutil.serializer.base.SqlDateColumnSerializerTest; +import com.dtstack.chunjun.typeutil.serializer.base.StringColumnSerializerTest; +import com.dtstack.chunjun.typeutil.serializer.base.TimeColumnSerializerTest; +import com.dtstack.chunjun.typeutil.serializer.base.TimestampColumnSerializerTest; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; + +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.function.BiFunction; + +/** @author liuliu 2022/5/13 */ +public class ColumnRowDataSerializerTest extends SerializerTestBase { + + private final DeeplyEqualsChecker.CustomEqualityChecker[] customEqualityCheckers = { + new BooleanColumnSerializerTest.BooleanColumnChecker(), + new BooleanColumnSerializerTest.BooleanColumnChecker(), + new ByteColumnSerializerTest.ByteColumnChecker(), + new ByteColumnSerializerTest.ByteColumnChecker(), + new BytesColumnSerializerTest.BytesColumnChecker(), + new DecimalColumnSerializerTest.DecimalColumnChecker(), + new DoubleColumnSerializerTest.DoubleColumnChecker(), + new FloatColumnSerializerTest.FloatColumnChecker(), + new IntColumnSerializerTest.IntColumnChecker(), + new LongColumnSerializerTest.LongColumnChecker(), + new ShortColumnSerializerTest.ShortColumnChecker(), + new SqlDateColumnSerializerTest.SqlDateColumnChecker(), + new SqlDateColumnSerializerTest.SqlDateColumnChecker(), + new StringColumnSerializerTest.StringColumnChecker(), + new TimeColumnSerializerTest.TimeColumnChecker(), + new TimestampColumnSerializerTest.TimestampColumnChecker(), + new NullColumnSerializerTest.NullColumnChecker() + }; + + @Override + protected Tuple2, DeeplyEqualsChecker.CustomEqualityChecker> + getCustomChecker() { + return Tuple2.of( + new BiFunction() { + @Override + public Boolean apply(Object o, Object o2) { + return o instanceof ColumnRowData && o2 instanceof ColumnRowData; + } + }, + new ColumnRowDataColumnChecker()); + } + + @Override + protected TypeSerializer createSerializer() { + return new ColumnRowDataSerializer(getRowType()); + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return RowData.class; + } + + @Override + protected RowData[] getTestData() { + return new RowData[] {getColumnRowData(), getColumnRowData()}; + } + + public class ColumnRowDataColumnChecker implements DeeplyEqualsChecker.CustomEqualityChecker { + @Override + public boolean check(Object o1, Object o2, DeeplyEqualsChecker checker) { + if (o1 instanceof ColumnRowData && o2 instanceof ColumnRowData) { + ColumnRowData want = (ColumnRowData) o1; + ColumnRowData that = (ColumnRowData) o2; + if (want.getRowKind() != that.getRowKind() + || want.getHeaderInfo().size() != that.getHeaderInfo().size() + || want.getExtHeader().size() != that.getExtHeader().size() + || want.getArity() != that.getArity() + || want.getByteSize() != that.getByteSize()) { + return false; + } + for (Map.Entry entry : want.getHeaderInfo().entrySet()) { + if (!Objects.equals( + entry.getValue(), that.getHeaderInfo().get(entry.getKey()))) { + return false; + } + } + for (String extStr : want.getExtHeader()) { + if (!that.getExtHeader().contains(extStr)) { + return false; + } + } + for (int i = 0; i < want.getArity(); i++) { + AbstractBaseColumn wantFiled = want.getField(i); + AbstractBaseColumn thatField = that.getField(i); + if (!customEqualityCheckers[i].check(wantFiled, thatField, null)) { + return false; + } + } + } + return true; + } + } + + public RowType getRowType() { + List rowFieldList = new ArrayList<>(13); + rowFieldList.add(new RowType.RowField("BooleanData", DataTypes.BOOLEAN().getLogicalType())); + rowFieldList.add( + new RowType.RowField("BooleanData2", DataTypes.BOOLEAN().getLogicalType())); + rowFieldList.add(new RowType.RowField("ByteData", DataTypes.TINYINT().getLogicalType())); + rowFieldList.add(new RowType.RowField("ByteData2", DataTypes.TINYINT().getLogicalType())); + rowFieldList.add(new RowType.RowField("BytesData", DataTypes.BYTES().getLogicalType())); + rowFieldList.add( + new RowType.RowField("DecimalData", DataTypes.DECIMAL(1, 1).getLogicalType())); + rowFieldList.add(new RowType.RowField("DoubleData", DataTypes.DOUBLE().getLogicalType())); + rowFieldList.add(new RowType.RowField("FloatData", DataTypes.FLOAT().getLogicalType())); + rowFieldList.add(new RowType.RowField("IntData", DataTypes.INT().getLogicalType())); + rowFieldList.add(new RowType.RowField("LongData", DataTypes.BIGINT().getLogicalType())); + rowFieldList.add(new RowType.RowField("ShortData", DataTypes.SMALLINT().getLogicalType())); + rowFieldList.add(new RowType.RowField("SqlDateData", DataTypes.DATE().getLogicalType())); + rowFieldList.add(new RowType.RowField("SqlDateData2", DataTypes.DATE().getLogicalType())); + rowFieldList.add( + new RowType.RowField( + "StringData", + DataTypes.STRING().getLogicalType(), + "yyyy-MM-dd HH:mm:ss.SSS")); + rowFieldList.add(new RowType.RowField("TimeData", DataTypes.TIME().getLogicalType())); + rowFieldList.add( + new RowType.RowField("TimestampData", DataTypes.TIMESTAMP().getLogicalType())); + rowFieldList.add(new RowType.RowField("NullColumn", DataTypes.NULL().getLogicalType())); + return new RowType(rowFieldList); + } + + public ColumnRowData getColumnRowData() { + ColumnRowData columnRowData = new ColumnRowData(13); + columnRowData.addHeader("123"); + columnRowData.addExtHeader("1234"); + columnRowData.addField(new BooleanColumn(false)); + columnRowData.addField(new BytesColumn(new byte[] {1, 2})); + columnRowData.addField(new ByteColumn((byte) 1)); + columnRowData.addField(new BigDecimalColumn((byte) 1)); + columnRowData.addField(new BytesColumn((new byte[] {1, 2}))); + columnRowData.addField(new BigDecimalColumn(new BigDecimal("1234123123"))); + columnRowData.addField(new BigDecimalColumn(123.123)); + columnRowData.addField(new BigDecimalColumn((float) 1.12)); + columnRowData.addField(new BigDecimalColumn(123)); + columnRowData.addField(new BigDecimalColumn(123123123L)); + columnRowData.addField(new BigDecimalColumn((short) 12)); + columnRowData.addField(new SqlDateColumn(100)); + columnRowData.addField(new TimestampColumn(System.currentTimeMillis(), 0)); + columnRowData.addField(new StringColumn("123", "yyyy-MM-dd HH:mm:ss.SSS")); + columnRowData.addField(new TimeColumn(19)); + columnRowData.addField(new TimestampColumn(System.currentTimeMillis())); + columnRowData.addField(new NullColumn()); + return columnRowData; + } +} diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/CustomEqualityMatcher.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/CustomEqualityMatcher.java new file mode 100644 index 0000000000..09db8de0fe --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/CustomEqualityMatcher.java @@ -0,0 +1,71 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer; + +import org.apache.flink.api.java.tuple.Tuple; + +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; + +import java.util.Arrays; + +public class CustomEqualityMatcher extends BaseMatcher { + + private final Object wanted; + + private final DeeplyEqualsChecker checker; + + private CustomEqualityMatcher(Object wanted, DeeplyEqualsChecker checker) { + this.wanted = wanted; + this.checker = checker; + } + + /** + * This matcher performs similar comparison to {@link org.hamcrest.core.IsEqual}, which + * resembles {@link java.util.Objects#deepEquals(Object, Object)} logic. The only difference + * here is that {@link Tuple}s are treated similarly to arrays. + * + *

This means that if we compare two Tuples that contain arrays, those arrays will be + * compared with {@link Arrays#deepEquals(Object[], Object[])} rather than with reference + * comparison. + * + * @param item expected value + */ + public static CustomEqualityMatcher deeplyEquals(Object item) { + return new CustomEqualityMatcher(item, new DeeplyEqualsChecker()); + } + + /** + * Performs assertions with this customly configured {@link DeeplyEqualsChecker}. It might have + * some additional rules applied. + */ + public CustomEqualityMatcher withChecker(DeeplyEqualsChecker checker) { + return new CustomEqualityMatcher(wanted, checker); + } + + @Override + public boolean matches(Object item) { + return checker.deepEquals(item, wanted); + } + + @Override + public void describeTo(Description description) { + description.appendValue(wanted); + } +} diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/DeeplyEqualsChecker.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/DeeplyEqualsChecker.java new file mode 100644 index 0000000000..a5af685484 --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/DeeplyEqualsChecker.java @@ -0,0 +1,137 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer; + +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple2; + +import java.lang.reflect.Array; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.function.BiFunction; + +/** + * Deep equality checker for tests. It performs deep checks for objects which have no proper + * deepEquals methods like: + * + *

    + *
  • {@link Tuple}s + *
  • Java arrays + *
  • {@link Throwable} + *
+ * + *

One can also provide custom check for additional categories of objects with {@link + * DeeplyEqualsChecker#withCustomCheck(BiFunction, CustomEqualityChecker)}. This is used e.g. in + * scala's tests. + */ +public class DeeplyEqualsChecker { + + /** + * Checker that compares o1 and o2 objects if they are deeply equal. + * + *

NOTE: All nested comparisons should be done through checker. + */ + public interface CustomEqualityChecker { + boolean check(Object o1, Object o2, DeeplyEqualsChecker checker); + } + + private final List, CustomEqualityChecker>> + customCheckers = new ArrayList<>(); + + /** + * Adds custom check. Those check are always performed first, only after that it fallbacks to + * default checks. + * + * @param shouldCheck function to evaluate if the objects should be compared with comparator + * @param comparator to perform equality comparison if the shouldCheck passed + * @return checker with added custom checks + */ + public DeeplyEqualsChecker withCustomCheck( + BiFunction shouldCheck, CustomEqualityChecker comparator) { + customCheckers.add(Tuple2.of(shouldCheck, comparator)); + return this; + } + + public boolean deepEquals(Object o1, Object o2) { + if (o1 == o2) { + return true; + } else if (o1 == null || o2 == null) { + return false; + } else { + return customCheck(o1, o2).orElseGet(() -> deepEquals0(o1, o2)); + } + } + + private Optional customCheck(Object o1, Object o2) { + return customCheckers.stream() + .filter(checker -> checker.f0.apply(o1, o2)) + .findAny() + .map(checker -> checker.f1.check(o1, o2, this)); + } + + private boolean deepEquals0(Object e1, Object e2) { + if (e1.getClass().isArray() && e2.getClass().isArray()) { + return deepEqualsArray(e1, e2); + } else if (e1 instanceof Tuple && e2 instanceof Tuple) { + return deepEqualsTuple((Tuple) e1, (Tuple) e2); + } else if (e1 instanceof Throwable && e2 instanceof Throwable) { + return ((Throwable) e1).getMessage().equals(((Throwable) e2).getMessage()); + } else { + return e1.equals(e2); + } + } + + private boolean deepEqualsTuple(Tuple tuple1, Tuple tuple2) { + if (tuple1.getArity() != tuple2.getArity()) { + return false; + } + + for (int i = 0; i < tuple1.getArity(); i++) { + Object o1 = tuple1.getField(i); + Object o2 = tuple2.getField(i); + + if (!deepEquals(o1, o2)) { + return false; + } + } + + return true; + } + + private boolean deepEqualsArray(Object array1, Object array2) { + int length1 = Array.getLength(array1); + int length2 = Array.getLength(array2); + + if (length1 != length2) { + return false; + } + + for (int i = 0; i < length1; i++) { + Object o1 = Array.get(array1, i); + Object o2 = Array.get(array2, i); + + if (!deepEquals(o1, o2)) { + return false; + } + } + + return true; + } +} diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/SerializerTestBase.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/SerializerTestBase.java new file mode 100644 index 0000000000..e5947169cf --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/SerializerTestBase.java @@ -0,0 +1,635 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshotSerializationUtil; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.InstantiationUtil; + +import org.apache.commons.lang3.SerializationException; +import org.apache.commons.lang3.SerializationUtils; +import org.junit.Assert; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CyclicBarrier; +import java.util.function.BiFunction; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Abstract test base for serializers. + * + *

We have a toString() call on all deserialized values because this is further evidence that the + * deserialized value is actually correct. (JodaTime DataTime) with the default KryoSerializer used + * to pass this test but the internal state would be corrupt, which becomes evident when toString is + * called. + */ +public abstract class SerializerTestBase { + + private final DeeplyEqualsChecker checker; + + protected SerializerTestBase() { + this.checker = new DeeplyEqualsChecker(); + Tuple2, DeeplyEqualsChecker.CustomEqualityChecker> + customChecker = getCustomChecker(); + if (customChecker != null) { + checker.withCustomCheck(customChecker.f0, customChecker.f1); + } + } + + protected SerializerTestBase(DeeplyEqualsChecker checker) { + this.checker = checker; + } + + protected abstract Tuple2< + BiFunction, DeeplyEqualsChecker.CustomEqualityChecker> + getCustomChecker(); + + protected abstract TypeSerializer createSerializer(); + + /** + * Gets the expected length for the serializer's {@link TypeSerializer#getLength()} method. + * + *

The expected length should be positive, for fix-length data types, or {@code -1} for + * variable-length types. + */ + protected abstract int getLength(); + + protected abstract Class getTypeClass(); + + protected abstract T[] getTestData(); + + /** + * Allows {@link TypeSerializer#createInstance()} to return null. + * + *

The {@link KryoSerializer} is one example. + */ + protected boolean allowNullInstances(TypeSerializer serializer) { + return serializer.getClass().getName().endsWith("KryoSerializer"); + } + + // -------------------------------------------------------------------------------------------- + + @Test + public void testInstantiate() { + try { + TypeSerializer serializer = getSerializer(); + T instance = serializer.createInstance(); + if (instance == null && allowNullInstances(serializer)) { + return; + } + assertNotNull("The created instance must not be null.", instance); + + Class type = getTypeClass(); + assertNotNull("The test is corrupt: type class is null.", type); + + if (!type.isAssignableFrom(instance.getClass())) { + fail( + "Type of the instantiated object is wrong. " + + "Expected Type: " + + type + + " present type " + + instance.getClass()); + } + } catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + fail("Exception in test: " + e.getMessage()); + } + } + + @Test + public void testConfigSnapshotInstantiation() { + TypeSerializerSnapshot configSnapshot = getSerializer().snapshotConfiguration(); + + InstantiationUtil.instantiate(configSnapshot.getClass()); + } + + @Test + public void testSnapshotConfigurationAndReconfigure() throws Exception { + final TypeSerializer serializer = getSerializer(); + final TypeSerializerSnapshot configSnapshot = serializer.snapshotConfiguration(); + + byte[] serializedConfig; + try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { + TypeSerializerSnapshotSerializationUtil.writeSerializerSnapshot( + new DataOutputViewStreamWrapper(out), configSnapshot, serializer); + serializedConfig = out.toByteArray(); + } + + TypeSerializerSnapshot restoredConfig; + try (ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) { + restoredConfig = + TypeSerializerSnapshotSerializationUtil.readSerializerSnapshot( + new DataInputViewStreamWrapper(in), + Thread.currentThread().getContextClassLoader(), + getSerializer()); + } + + TypeSerializerSchemaCompatibility strategy = + restoredConfig.resolveSchemaCompatibility(getSerializer()); + final TypeSerializer restoreSerializer; + if (strategy.isCompatibleAsIs()) { + restoreSerializer = restoredConfig.restoreSerializer(); + } else if (strategy.isCompatibleWithReconfiguredSerializer()) { + restoreSerializer = strategy.getReconfiguredSerializer(); + } else { + throw new AssertionError("Unable to restore serializer with " + strategy); + } + assertEquals(serializer.getClass(), restoreSerializer.getClass()); + } + + @Test + public void testGetLength() { + final int len = getLength(); + + if (len == 0) { + fail("Broken serializer test base - zero length cannot be the expected length"); + } + + try { + TypeSerializer serializer = getSerializer(); + assertEquals(len, serializer.getLength()); + } catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + fail("Exception in test: " + e.getMessage()); + } + } + + @Test + public void testCopy() { + try { + TypeSerializer serializer = getSerializer(); + T[] testData = getData(); + + for (T datum : testData) { + T copy = serializer.copy(datum); + checkToString(copy); + deepEquals("Copied element is not equal to the original element.", datum, copy); + } + } catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + fail("Exception in test: " + e.getMessage()); + } + } + + @Test + public void testCopyIntoNewElements() { + try { + TypeSerializer serializer = getSerializer(); + T[] testData = getData(); + + for (T datum : testData) { + T copy = serializer.copy(datum, serializer.createInstance()); + checkToString(copy); + deepEquals("Copied element is not equal to the original element.", datum, copy); + } + } catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + fail("Exception in test: " + e.getMessage()); + } + } + + @Test + public void testCopyIntoReusedElements() { + try { + TypeSerializer serializer = getSerializer(); + T[] testData = getData(); + + T target = serializer.createInstance(); + + for (T datum : testData) { + T copy = serializer.copy(datum, target); + checkToString(copy); + deepEquals("Copied element is not equal to the original element.", datum, copy); + target = copy; + } + } catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + fail("Exception in test: " + e.getMessage()); + } + } + + @Test + public void testSerializeIndividually() { + try { + TypeSerializer serializer = getSerializer(); + T[] testData = getData(); + + for (T value : testData) { + TestOutputView out = new TestOutputView(); + serializer.serialize(value, out); + TestInputView in = out.getInputView(); + + assertTrue("No data available during deserialization.", in.available() > 0); + + T deserialized = serializer.deserialize(serializer.createInstance(), in); + checkToString(deserialized); + + deepEquals("Deserialized value if wrong.", value, deserialized); + + assertTrue("Trailing data available after deserialization.", in.available() == 0); + } + } catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + fail("Exception in test: " + e.getMessage()); + } + } + + @Test + public void testSerializeIndividuallyReusingValues() { + try { + TypeSerializer serializer = getSerializer(); + T[] testData = getData(); + + T reuseValue = serializer.createInstance(); + + for (T value : testData) { + TestOutputView out = new TestOutputView(); + serializer.serialize(value, out); + TestInputView in = out.getInputView(); + + assertTrue("No data available during deserialization.", in.available() > 0); + + T deserialized = serializer.deserialize(reuseValue, in); + checkToString(deserialized); + + deepEquals("Deserialized value if wrong.", value, deserialized); + + assertTrue("Trailing data available after deserialization.", in.available() == 0); + + reuseValue = deserialized; + } + } catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + fail("Exception in test: " + e.getMessage()); + } + } + + @Test + public void testSerializeAsSequenceNoReuse() { + try { + TypeSerializer serializer = getSerializer(); + T[] testData = getData(); + + TestOutputView out = new TestOutputView(); + for (T value : testData) { + serializer.serialize(value, out); + } + + TestInputView in = out.getInputView(); + + int num = 0; + while (in.available() > 0) { + T deserialized = serializer.deserialize(in); + checkToString(deserialized); + + deepEquals("Deserialized value if wrong.", testData[num], deserialized); + num++; + } + + assertEquals("Wrong number of elements deserialized.", testData.length, num); + } catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + fail("Exception in test: " + e.getMessage()); + } + } + + @Test + public void testSerializeAsSequenceReusingValues() { + try { + TypeSerializer serializer = getSerializer(); + T[] testData = getData(); + + TestOutputView out = new TestOutputView(); + for (T value : testData) { + serializer.serialize(value, out); + } + + TestInputView in = out.getInputView(); + T reuseValue = serializer.createInstance(); + + int num = 0; + while (in.available() > 0) { + T deserialized = serializer.deserialize(reuseValue, in); + checkToString(deserialized); + + deepEquals("Deserialized value if wrong.", testData[num], deserialized); + reuseValue = deserialized; + num++; + } + + assertEquals("Wrong number of elements deserialized.", testData.length, num); + } catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + fail("Exception in test: " + e.getMessage()); + } + } + + @Test + public void testSerializedCopyIndividually() { + try { + TypeSerializer serializer = getSerializer(); + T[] testData = getData(); + + for (T value : testData) { + TestOutputView out = new TestOutputView(); + serializer.serialize(value, out); + + TestInputView source = out.getInputView(); + TestOutputView target = new TestOutputView(); + serializer.copy(source, target); + + TestInputView toVerify = target.getInputView(); + + assertTrue("No data available copying.", toVerify.available() > 0); + + T deserialized = serializer.deserialize(serializer.createInstance(), toVerify); + checkToString(deserialized); + + deepEquals("Deserialized value if wrong.", value, deserialized); + + assertTrue( + "Trailing data available after deserialization.", + toVerify.available() == 0); + } + } catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + fail("Exception in test: " + e.getMessage()); + } + } + + @Test + public void testSerializedCopyAsSequence() { + try { + TypeSerializer serializer = getSerializer(); + T[] testData = getData(); + + TestOutputView out = new TestOutputView(); + for (T value : testData) { + serializer.serialize(value, out); + } + + TestInputView source = out.getInputView(); + TestOutputView target = new TestOutputView(); + for (int i = 0; i < testData.length; i++) { + serializer.copy(source, target); + } + + TestInputView toVerify = target.getInputView(); + int num = 0; + + while (toVerify.available() > 0) { + T deserialized = serializer.deserialize(serializer.createInstance(), toVerify); + checkToString(deserialized); + + deepEquals("Deserialized value if wrong.", testData[num], deserialized); + num++; + } + + assertEquals("Wrong number of elements copied.", testData.length, num); + } catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + fail("Exception in test: " + e.getMessage()); + } + } + + @Test + public void testSerializabilityAndEquals() { + try { + TypeSerializer ser1 = getSerializer(); + TypeSerializer ser2; + try { + ser2 = SerializationUtils.clone(ser1); + } catch (SerializationException e) { + fail("The serializer is not serializable: " + e); + return; + } + + assertEquals( + "The copy of the serializer is not equal to the original one.", ser1, ser2); + } catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + fail("Exception in test: " + e.getMessage()); + } + } + + @Test + public void testDuplicate() throws Exception { + final int numThreads = 10; + final TypeSerializer serializer = getSerializer(); + final CyclicBarrier startLatch = new CyclicBarrier(numThreads); + final List> concurrentRunners = new ArrayList<>(numThreads); + Assert.assertEquals(serializer, serializer.duplicate()); + + T[] testData = getData(); + + for (int i = 0; i < numThreads; ++i) { + SerializerRunner runner = + new SerializerRunner<>( + startLatch, serializer.duplicate(), testData, 120L, checker); + + runner.start(); + concurrentRunners.add(runner); + } + + for (SerializerRunner concurrentRunner : concurrentRunners) { + concurrentRunner.join(); + concurrentRunner.checkResult(); + } + } + + // -------------------------------------------------------------------------------------------- + + private void deepEquals(String message, T should, T is) { + assertThat(message, is, CustomEqualityMatcher.deeplyEquals(should).withChecker(checker)); + } + + // -------------------------------------------------------------------------------------------- + + protected TypeSerializer getSerializer() { + TypeSerializer serializer = createSerializer(); + if (serializer == null) { + throw new RuntimeException("Test case corrupt. Returns null as serializer."); + } + return serializer; + } + + private T[] getData() { + T[] data = getTestData(); + if (data == null) { + throw new RuntimeException("Test case corrupt. Returns null as test data."); + } + return data; + } + + // -------------------------------------------------------------------------------------------- + + private static final class TestOutputView extends DataOutputStream implements DataOutputView { + + public TestOutputView() { + super(new ByteArrayOutputStream(4096)); + } + + public TestInputView getInputView() { + ByteArrayOutputStream baos = (ByteArrayOutputStream) out; + return new TestInputView(baos.toByteArray()); + } + + @Override + public void skipBytesToWrite(int numBytes) throws IOException { + for (int i = 0; i < numBytes; i++) { + write(0); + } + } + + @Override + public void write(DataInputView source, int numBytes) throws IOException { + byte[] buffer = new byte[numBytes]; + source.readFully(buffer); + write(buffer); + } + } + + /** + * Runner to test serializer duplication via concurrency. + * + * @param type of the test elements. + */ + static class SerializerRunner extends Thread { + final CyclicBarrier allReadyBarrier; + final TypeSerializer serializer; + final T[] testData; + final long durationLimitMillis; + Throwable failure; + final DeeplyEqualsChecker checker; + + SerializerRunner( + CyclicBarrier allReadyBarrier, + TypeSerializer serializer, + T[] testData, + long testTargetDurationMillis, + DeeplyEqualsChecker checker) { + + this.allReadyBarrier = allReadyBarrier; + this.serializer = serializer; + this.testData = testData; + this.durationLimitMillis = testTargetDurationMillis; + this.checker = checker; + this.failure = null; + } + + @Override + public void run() { + DataInputDeserializer dataInputDeserializer = new DataInputDeserializer(); + DataOutputSerializer dataOutputSerializer = new DataOutputSerializer(128); + try { + allReadyBarrier.await(); + final long endTimeNanos = System.nanoTime() + durationLimitMillis * 1_000_000L; + while (true) { + for (T testItem : testData) { + serializer.serialize(testItem, dataOutputSerializer); + dataInputDeserializer.setBuffer( + dataOutputSerializer.getSharedBuffer(), + 0, + dataOutputSerializer.length()); + T serdeTestItem = serializer.deserialize(dataInputDeserializer); + T copySerdeTestItem = serializer.copy(serdeTestItem); + dataOutputSerializer.clear(); + + assertThat( + "Serialization/Deserialization cycle resulted in an object that are not equal to the original.", + copySerdeTestItem, + CustomEqualityMatcher.deeplyEquals(testItem).withChecker(checker)); + + // try to enforce some upper bound to the test time + if (System.nanoTime() >= endTimeNanos) { + return; + } + } + } + } catch (Throwable ex) { + failure = ex; + } + } + + void checkResult() throws Exception { + if (failure != null) { + if (failure instanceof AssertionError) { + throw (AssertionError) failure; + } else { + throw (Exception) failure; + } + } + } + } + + private static final class TestInputView extends DataInputStream implements DataInputView { + + public TestInputView(byte[] data) { + super(new ByteArrayInputStream(data)); + } + + @Override + public void skipBytesToRead(int numBytes) throws IOException { + while (numBytes > 0) { + int skipped = skipBytes(numBytes); + numBytes -= skipped; + } + } + } + + private static void checkToString(T value) { + if (value != null) { + value.toString(); + } + } +} diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/BooleanColumnSerializerTest.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/BooleanColumnSerializerTest.java new file mode 100644 index 0000000000..be6d2df801 --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/BooleanColumnSerializerTest.java @@ -0,0 +1,96 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.BooleanColumn; +import com.dtstack.chunjun.element.column.BytesColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.typeutil.serializer.DeeplyEqualsChecker; +import com.dtstack.chunjun.typeutil.serializer.SerializerTestBase; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; + +import java.util.function.BiFunction; + +/** @author liuliu 2022/5/13 */ +public class BooleanColumnSerializerTest extends SerializerTestBase { + + @Override + protected Tuple2, DeeplyEqualsChecker.CustomEqualityChecker> + getCustomChecker() { + return Tuple2.of( + new BiFunction() { + @Override + public Boolean apply(Object o, Object o2) { + return (o instanceof BooleanColumn && o2 instanceof BooleanColumn) + || (o instanceof BytesColumn && o2 instanceof BytesColumn) + || (o instanceof NullColumn && o2 instanceof NullColumn); + } + }, + new BooleanColumnChecker()); + } + + @Override + protected TypeSerializer createSerializer() { + return BooleanColumnSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return AbstractBaseColumn.class; + } + + @Override + protected AbstractBaseColumn[] getTestData() { + return new AbstractBaseColumn[] { + new NullColumn(), new BooleanColumn(true), new BytesColumn(new byte[] {1, 2}) + }; + } + + public static class BooleanColumnChecker implements DeeplyEqualsChecker.CustomEqualityChecker { + + @Override + public boolean check(Object o1, Object o2, DeeplyEqualsChecker checker) { + if (o1 instanceof BooleanColumn && o2 instanceof BooleanColumn) { + return ((BooleanColumn) o1).getData() == ((BooleanColumn) o2).getData(); + } else if (o1 instanceof BytesColumn && o2 instanceof BytesColumn) { + byte[] wantBytes = ((BytesColumn) o1).asBytes(); + byte[] wasBytes = ((BytesColumn) o2).asBytes(); + if (wantBytes.length == wasBytes.length) { + for (int i = 0; i < wantBytes.length; i++) { + if (wantBytes[i] != wasBytes[i]) { + return false; + } + } + return true; + } + return false; + } else { + return o1 instanceof NullColumn && o2 instanceof NullColumn; + } + } + } +} diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/ByteColumnSerializerTest.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/ByteColumnSerializerTest.java new file mode 100644 index 0000000000..0cfbc8490c --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/ByteColumnSerializerTest.java @@ -0,0 +1,89 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.ByteColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.typeutil.serializer.DeeplyEqualsChecker; +import com.dtstack.chunjun.typeutil.serializer.SerializerTestBase; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; + +import java.util.function.BiFunction; + +/** @author liuliu 2022/5/13 */ +public class ByteColumnSerializerTest extends SerializerTestBase { + + @Override + protected Tuple2, DeeplyEqualsChecker.CustomEqualityChecker> + getCustomChecker() { + return Tuple2.of( + new BiFunction() { + @Override + public Boolean apply(Object o, Object o2) { + return (o instanceof ByteColumn && o2 instanceof ByteColumn) + || (o instanceof BigDecimalColumn && o2 instanceof BigDecimalColumn) + || (o instanceof NullColumn && o2 instanceof NullColumn); + } + }, + new ByteColumnChecker()); + } + + @Override + protected TypeSerializer createSerializer() { + return ByteColumnSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return AbstractBaseColumn.class; + } + + @Override + protected AbstractBaseColumn[] getTestData() { + return new AbstractBaseColumn[] { + new NullColumn(), new ByteColumn((byte) 0), new ByteColumn((byte) 12) + }; + } + + public static class ByteColumnChecker implements DeeplyEqualsChecker.CustomEqualityChecker { + + @Override + public boolean check(Object o1, Object o2, DeeplyEqualsChecker checker) { + if (o1 instanceof ByteColumn && o2 instanceof ByteColumn) { + return ((ByteColumn) o1).getData() == ((ByteColumn) o2).getData(); + } else if (o1 instanceof BigDecimalColumn && o2 instanceof BigDecimalColumn) { + return ((BigDecimalColumn) o1) + .asBigDecimal() + .compareTo(((BigDecimalColumn) o2).asBigDecimal()) + == 0; + } else { + return o1 instanceof NullColumn && o2 instanceof NullColumn; + } + } + } +} diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/BytesColumnSerializerTest.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/BytesColumnSerializerTest.java new file mode 100644 index 0000000000..ef68b8090f --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/BytesColumnSerializerTest.java @@ -0,0 +1,94 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.BytesColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.typeutil.serializer.DeeplyEqualsChecker; +import com.dtstack.chunjun.typeutil.serializer.SerializerTestBase; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; + +import java.util.function.BiFunction; + +/** @author liuliu 2022/5/13 */ +public class BytesColumnSerializerTest extends SerializerTestBase { + + @Override + protected Tuple2, DeeplyEqualsChecker.CustomEqualityChecker> + getCustomChecker() { + return Tuple2.of( + new BiFunction() { + @Override + public Boolean apply(Object o, Object o2) { + return o instanceof BytesColumn && o2 instanceof BytesColumn + || (o instanceof NullColumn && o2 instanceof NullColumn); + } + }, + new BytesColumnChecker()); + } + + @Override + protected TypeSerializer createSerializer() { + return BytesColumnSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return AbstractBaseColumn.class; + } + + @Override + protected AbstractBaseColumn[] getTestData() { + return new AbstractBaseColumn[] { + new NullColumn(), + new BytesColumn(new byte[] {(byte) 0}), + new BytesColumn(new byte[] {(byte) 12, (byte) 123}) + }; + } + + public static class BytesColumnChecker implements DeeplyEqualsChecker.CustomEqualityChecker { + + @Override + public boolean check(Object o1, Object o2, DeeplyEqualsChecker checker) { + if (o1 instanceof BytesColumn && o2 instanceof BytesColumn) { + byte[] wantBytes = ((BytesColumn) o1).asBytes(); + byte[] wasBytes = ((BytesColumn) o2).asBytes(); + if (wantBytes.length == wasBytes.length) { + for (int i = 0; i < wantBytes.length; i++) { + if (wantBytes[i] != wasBytes[i]) { + return false; + } + } + return true; + } + return false; + } else { + return o1 instanceof NullColumn && o2 instanceof NullColumn; + } + } + } +} diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/DecimalColumnSerializerTest.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/DecimalColumnSerializerTest.java new file mode 100644 index 0000000000..f8a0f52c80 --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/DecimalColumnSerializerTest.java @@ -0,0 +1,90 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.typeutil.serializer.DeeplyEqualsChecker; +import com.dtstack.chunjun.typeutil.serializer.SerializerTestBase; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; + +import java.math.BigDecimal; +import java.util.function.BiFunction; + +/** @author liuliu 2022/5/13 */ +public class DecimalColumnSerializerTest extends SerializerTestBase { + + @Override + protected Tuple2, DeeplyEqualsChecker.CustomEqualityChecker> + getCustomChecker() { + return Tuple2.of( + new BiFunction() { + @Override + public Boolean apply(Object o, Object o2) { + return (o instanceof BigDecimalColumn && o2 instanceof BigDecimalColumn) + || (o instanceof NullColumn && o2 instanceof NullColumn); + } + }, + new DecimalColumnChecker()); + } + + @Override + protected TypeSerializer createSerializer() { + return DecimalColumnSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return AbstractBaseColumn.class; + } + + @Override + protected AbstractBaseColumn[] getTestData() { + return new AbstractBaseColumn[] { + new NullColumn(), + new BigDecimalColumn(new BigDecimal("1.12")), + new BigDecimalColumn(new BigDecimal("2.123")), + new BigDecimalColumn(new BigDecimal("3.1234")), + new BigDecimalColumn(new BigDecimal("4321.1234")) + }; + } + + public static class DecimalColumnChecker implements DeeplyEqualsChecker.CustomEqualityChecker { + + @Override + public boolean check(Object o1, Object o2, DeeplyEqualsChecker checker) { + if (o1 instanceof BigDecimalColumn && o2 instanceof BigDecimalColumn) { + return ((BigDecimalColumn) o1) + .asBigDecimal() + .compareTo(((BigDecimalColumn) o2).asBigDecimal()) + == 0; + } else { + return o1 instanceof NullColumn && o2 instanceof NullColumn; + } + } + } +} diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/DoubleColumnSerializerTest.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/DoubleColumnSerializerTest.java new file mode 100644 index 0000000000..0a38116e86 --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/DoubleColumnSerializerTest.java @@ -0,0 +1,90 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.typeutil.serializer.DeeplyEqualsChecker; +import com.dtstack.chunjun.typeutil.serializer.SerializerTestBase; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; + +import java.math.BigDecimal; +import java.util.function.BiFunction; + +/** @author liuliu 2022/5/13 */ +public class DoubleColumnSerializerTest extends SerializerTestBase { + + @Override + protected Tuple2, DeeplyEqualsChecker.CustomEqualityChecker> + getCustomChecker() { + return Tuple2.of( + new BiFunction() { + @Override + public Boolean apply(Object o, Object o2) { + return (o instanceof BigDecimalColumn && o2 instanceof BigDecimalColumn) + || (o instanceof NullColumn && o2 instanceof NullColumn); + } + }, + new DoubleColumnChecker()); + } + + @Override + protected TypeSerializer createSerializer() { + return DoubleColumnSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return AbstractBaseColumn.class; + } + + @Override + protected AbstractBaseColumn[] getTestData() { + return new AbstractBaseColumn[] { + new NullColumn(), + new BigDecimalColumn(new BigDecimal("1.12")), + new BigDecimalColumn(new BigDecimal("2.123")), + new BigDecimalColumn(new BigDecimal("3.1234")), + new BigDecimalColumn(new BigDecimal("12312.1234567891")) + }; + } + + public static class DoubleColumnChecker implements DeeplyEqualsChecker.CustomEqualityChecker { + + @Override + public boolean check(Object o1, Object o2, DeeplyEqualsChecker checker) { + if (o1 instanceof BigDecimalColumn && o2 instanceof BigDecimalColumn) { + return ((BigDecimalColumn) o1) + .asBigDecimal() + .compareTo(((BigDecimalColumn) o2).asBigDecimal()) + == 0; + } else { + return o1 instanceof NullColumn && o2 instanceof NullColumn; + } + } + } +} diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/FloatColumnSerializerTest.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/FloatColumnSerializerTest.java new file mode 100644 index 0000000000..b0f6a2ca89 --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/FloatColumnSerializerTest.java @@ -0,0 +1,89 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.typeutil.serializer.DeeplyEqualsChecker; +import com.dtstack.chunjun.typeutil.serializer.SerializerTestBase; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; + +import java.math.BigDecimal; +import java.util.function.BiFunction; + +/** @author liuliu 2022/5/13 */ +public class FloatColumnSerializerTest extends SerializerTestBase { + + @Override + protected Tuple2, DeeplyEqualsChecker.CustomEqualityChecker> + getCustomChecker() { + return Tuple2.of( + new BiFunction() { + @Override + public Boolean apply(Object o, Object o2) { + return (o instanceof BigDecimalColumn && o2 instanceof BigDecimalColumn) + || (o instanceof NullColumn && o2 instanceof NullColumn); + } + }, + new FloatColumnChecker()); + } + + @Override + protected TypeSerializer createSerializer() { + return FloatColumnSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return AbstractBaseColumn.class; + } + + @Override + protected AbstractBaseColumn[] getTestData() { + return new AbstractBaseColumn[] { + new NullColumn(), + new BigDecimalColumn(new BigDecimal("1.12")), + new BigDecimalColumn(new BigDecimal("2.123")), + new BigDecimalColumn(new BigDecimal("3.1234")), + }; + } + + public static class FloatColumnChecker implements DeeplyEqualsChecker.CustomEqualityChecker { + + @Override + public boolean check(Object o1, Object o2, DeeplyEqualsChecker checker) { + if (o1 instanceof BigDecimalColumn && o2 instanceof BigDecimalColumn) { + return ((BigDecimalColumn) o1) + .asBigDecimal() + .compareTo(((BigDecimalColumn) o2).asBigDecimal()) + == 0; + } else { + return o1 instanceof NullColumn && o2 instanceof NullColumn; + } + } + } +} diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/IntColumnSerializerTest.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/IntColumnSerializerTest.java new file mode 100644 index 0000000000..1c2bcc066c --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/IntColumnSerializerTest.java @@ -0,0 +1,89 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.typeutil.serializer.DeeplyEqualsChecker; +import com.dtstack.chunjun.typeutil.serializer.SerializerTestBase; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; + +import java.math.BigDecimal; +import java.util.function.BiFunction; + +/** @author liuliu 2022/5/13 */ +public class IntColumnSerializerTest extends SerializerTestBase { + + @Override + protected Tuple2, DeeplyEqualsChecker.CustomEqualityChecker> + getCustomChecker() { + return Tuple2.of( + new BiFunction() { + @Override + public Boolean apply(Object o, Object o2) { + return (o instanceof BigDecimalColumn && o2 instanceof BigDecimalColumn) + || (o instanceof NullColumn && o2 instanceof NullColumn); + } + }, + new IntColumnChecker()); + } + + @Override + protected TypeSerializer createSerializer() { + return IntColumnSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return AbstractBaseColumn.class; + } + + @Override + protected AbstractBaseColumn[] getTestData() { + return new AbstractBaseColumn[] { + new NullColumn(), + new BigDecimalColumn(new BigDecimal("123")), + new BigDecimalColumn(new BigDecimal("212")), + new BigDecimalColumn(new BigDecimal("1234312")), + }; + } + + public static class IntColumnChecker implements DeeplyEqualsChecker.CustomEqualityChecker { + + @Override + public boolean check(Object o1, Object o2, DeeplyEqualsChecker checker) { + if (o1 instanceof BigDecimalColumn && o2 instanceof BigDecimalColumn) { + return ((BigDecimalColumn) o1) + .asBigDecimal() + .compareTo(((BigDecimalColumn) o2).asBigDecimal()) + == 0; + } else { + return o1 instanceof NullColumn && o2 instanceof NullColumn; + } + } + } +} diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/LongColumnSerializerTest.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/LongColumnSerializerTest.java new file mode 100644 index 0000000000..e1a8db227f --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/LongColumnSerializerTest.java @@ -0,0 +1,89 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.typeutil.serializer.DeeplyEqualsChecker; +import com.dtstack.chunjun.typeutil.serializer.SerializerTestBase; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; + +import java.math.BigDecimal; +import java.util.function.BiFunction; + +/** @author liuliu 2022/5/13 */ +public class LongColumnSerializerTest extends SerializerTestBase { + + @Override + protected Tuple2, DeeplyEqualsChecker.CustomEqualityChecker> + getCustomChecker() { + return Tuple2.of( + new BiFunction() { + @Override + public Boolean apply(Object o, Object o2) { + return (o instanceof BigDecimalColumn && o2 instanceof BigDecimalColumn) + || (o instanceof NullColumn && o2 instanceof NullColumn); + } + }, + new LongColumnChecker()); + } + + @Override + protected TypeSerializer createSerializer() { + return LongColumnSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return AbstractBaseColumn.class; + } + + @Override + protected AbstractBaseColumn[] getTestData() { + return new AbstractBaseColumn[] { + new NullColumn(), + new BigDecimalColumn(new BigDecimal("123123123123")), + new BigDecimalColumn(new BigDecimal("212123123123")), + new BigDecimalColumn(new BigDecimal("1234312123123")), + }; + } + + public static class LongColumnChecker implements DeeplyEqualsChecker.CustomEqualityChecker { + + @Override + public boolean check(Object o1, Object o2, DeeplyEqualsChecker checker) { + if (o1 instanceof BigDecimalColumn && o2 instanceof BigDecimalColumn) { + return ((BigDecimalColumn) o1) + .asBigDecimal() + .compareTo(((BigDecimalColumn) o2).asBigDecimal()) + == 0; + } else { + return o1 instanceof NullColumn && o2 instanceof NullColumn; + } + } + } +} diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/MapColumnSerializerTest.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/MapColumnSerializerTest.java new file mode 100644 index 0000000000..5b3f44e5c2 --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/MapColumnSerializerTest.java @@ -0,0 +1,98 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.MapColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.typeutil.serializer.DeeplyEqualsChecker; +import com.dtstack.chunjun.typeutil.serializer.SerializerTestBase; +import com.dtstack.chunjun.util.JsonUtil; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; + +/** @author liuliu 2022/5/25 */ +public class MapColumnSerializerTest extends SerializerTestBase { + @Override + protected Tuple2, DeeplyEqualsChecker.CustomEqualityChecker> + getCustomChecker() { + return Tuple2.of( + new BiFunction() { + @Override + public Boolean apply(Object o, Object o2) { + return (o instanceof MapColumn && o2 instanceof MapColumn) + || (o instanceof NullColumn && o2 instanceof NullColumn); + } + }, + new MapColumnChecker()); + } + + @Override + protected TypeSerializer createSerializer() { + return MapColumnSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return AbstractBaseColumn.class; + } + + @Override + protected AbstractBaseColumn[] getTestData() { + Map map = new HashMap<>(); + List> arrayListData = new ArrayList<>(); + Map mapData = new HashMap<>(); + mapData.put("1", 1); + mapData.put("2", 2); + arrayListData.add(mapData); + map.put("arrayListData", arrayListData); + map.put("3", 3); + map.put("4", "4"); + return new AbstractBaseColumn[] {new MapColumn(map)}; + } + + public static class MapColumnChecker implements DeeplyEqualsChecker.CustomEqualityChecker { + + @Override + public boolean check(Object o1, Object o2, DeeplyEqualsChecker checker) { + if (o1 instanceof MapColumn && o2 instanceof MapColumn) { + Map want = (Map) ((MapColumn) o1).getData(); + Map that = (Map) ((MapColumn) o2).getData(); + if (want.size() != that.size()) { + return false; + } + return JsonUtil.toJson(want).equals(JsonUtil.toJson(that)); + } else { + return o1 instanceof NullColumn && o2 instanceof NullColumn; + } + } + } +} diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/NullColumnSerializerTest.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/NullColumnSerializerTest.java new file mode 100644 index 0000000000..4d98154c6b --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/NullColumnSerializerTest.java @@ -0,0 +1,72 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.typeutil.serializer.DeeplyEqualsChecker; +import com.dtstack.chunjun.typeutil.serializer.SerializerTestBase; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; + +import java.util.function.BiFunction; + +/** @author liuliu 2022/5/25 */ +public class NullColumnSerializerTest extends SerializerTestBase { + @Override + protected Tuple2, DeeplyEqualsChecker.CustomEqualityChecker> + getCustomChecker() { + return Tuple2.of( + new BiFunction() { + @Override + public Boolean apply(Object o, Object o2) { + return o instanceof NullColumn && o2 instanceof NullColumn; + } + }, + new NullColumnChecker()); + } + + @Override + protected TypeSerializer createSerializer() { + return NullColumnSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return AbstractBaseColumn.class; + } + + @Override + protected AbstractBaseColumn[] getTestData() { + return new AbstractBaseColumn[] {new NullColumn()}; + } + + public static class NullColumnChecker implements DeeplyEqualsChecker.CustomEqualityChecker { + @Override + public boolean check(Object o1, Object o2, DeeplyEqualsChecker checker) { + return o1 instanceof NullColumn && o2 instanceof NullColumn; + } + } +} diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/ShortColumnSerializerTest.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/ShortColumnSerializerTest.java new file mode 100644 index 0000000000..b6c4ff8e48 --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/ShortColumnSerializerTest.java @@ -0,0 +1,89 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.typeutil.serializer.DeeplyEqualsChecker; +import com.dtstack.chunjun.typeutil.serializer.SerializerTestBase; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; + +import java.math.BigDecimal; +import java.util.function.BiFunction; + +/** @author liuliu 2022/5/13 */ +public class ShortColumnSerializerTest extends SerializerTestBase { + + @Override + protected Tuple2, DeeplyEqualsChecker.CustomEqualityChecker> + getCustomChecker() { + return Tuple2.of( + new BiFunction() { + @Override + public Boolean apply(Object o, Object o2) { + return (o instanceof BigDecimalColumn && o2 instanceof BigDecimalColumn) + || (o instanceof NullColumn && o2 instanceof NullColumn); + } + }, + new ShortColumnChecker()); + } + + @Override + protected TypeSerializer createSerializer() { + return ShortColumnSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return AbstractBaseColumn.class; + } + + @Override + protected AbstractBaseColumn[] getTestData() { + return new AbstractBaseColumn[] { + new NullColumn(), + new BigDecimalColumn(new BigDecimal("123")), + new BigDecimalColumn(new BigDecimal("12")), + new BigDecimalColumn(new BigDecimal("1")), + }; + } + + public static class ShortColumnChecker implements DeeplyEqualsChecker.CustomEqualityChecker { + + @Override + public boolean check(Object o1, Object o2, DeeplyEqualsChecker checker) { + if (o1 instanceof BigDecimalColumn && o2 instanceof BigDecimalColumn) { + return ((BigDecimalColumn) o1) + .asBigDecimal() + .compareTo(((BigDecimalColumn) o2).asBigDecimal()) + == 0; + } else { + return o1 instanceof NullColumn && o2 instanceof NullColumn; + } + } + } +} diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/SqlDateColumnSerializerTest.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/SqlDateColumnSerializerTest.java new file mode 100644 index 0000000000..409fc572a8 --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/SqlDateColumnSerializerTest.java @@ -0,0 +1,94 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.element.column.SqlDateColumn; +import com.dtstack.chunjun.element.column.TimestampColumn; +import com.dtstack.chunjun.typeutil.serializer.DeeplyEqualsChecker; +import com.dtstack.chunjun.typeutil.serializer.SerializerTestBase; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; + +import java.sql.Date; +import java.util.function.BiFunction; + +/** @author liuliu 2022/5/13 */ +public class SqlDateColumnSerializerTest extends SerializerTestBase { + + @Override + protected Tuple2, DeeplyEqualsChecker.CustomEqualityChecker> + getCustomChecker() { + return Tuple2.of( + new BiFunction() { + @Override + public Boolean apply(Object o, Object o2) { + return (o instanceof SqlDateColumn && o2 instanceof SqlDateColumn) + || (o instanceof TimestampColumn && o2 instanceof TimestampColumn) + || (o instanceof NullColumn && o2 instanceof NullColumn); + } + }, + new SqlDateColumnChecker()); + } + + @Override + protected TypeSerializer createSerializer() { + return SqlDateColumnSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return AbstractBaseColumn.class; + } + + @Override + protected AbstractBaseColumn[] getTestData() { + return new AbstractBaseColumn[] { + new NullColumn(), + new SqlDateColumn(100), + new SqlDateColumn(new Date(System.currentTimeMillis())), + new TimestampColumn(System.currentTimeMillis(), 0) + }; + } + + public static class SqlDateColumnChecker implements DeeplyEqualsChecker.CustomEqualityChecker { + + @Override + public boolean check(Object o1, Object o2, DeeplyEqualsChecker checker) { + if (o1 instanceof SqlDateColumn && o2 instanceof SqlDateColumn) { + return ((SqlDateColumn) o1).asSqlDate().getTime() + == ((SqlDateColumn) o2).asSqlDate().getTime(); + } else if (o1 instanceof TimestampColumn && o2 instanceof TimestampColumn) { + return ((TimestampColumn) o1).asTimestamp().getTime() + == ((TimestampColumn) o2).asTimestamp().getTime() + && ((TimestampColumn) o1).getPrecision() + == ((TimestampColumn) o2).getPrecision(); + } else { + return o1 instanceof NullColumn && o2 instanceof NullColumn; + } + } + } +} diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/StringColumnSerializerTest.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/StringColumnSerializerTest.java new file mode 100644 index 0000000000..b5b6a00838 --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/StringColumnSerializerTest.java @@ -0,0 +1,81 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.element.column.StringColumn; +import com.dtstack.chunjun.typeutil.serializer.DeeplyEqualsChecker; +import com.dtstack.chunjun.typeutil.serializer.SerializerTestBase; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; + +import java.util.function.BiFunction; + +/** @author liuliu 2022/5/13 */ +public class StringColumnSerializerTest extends SerializerTestBase { + + @Override + protected Tuple2, DeeplyEqualsChecker.CustomEqualityChecker> + getCustomChecker() { + return Tuple2.of( + new BiFunction() { + @Override + public Boolean apply(Object o, Object o2) { + return (o instanceof StringColumn && o2 instanceof StringColumn) + || (o instanceof NullColumn && o2 instanceof NullColumn); + } + }, + new StringColumnChecker()); + } + + @Override + protected TypeSerializer createSerializer() { + return new StringColumnSerializer(""); + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return AbstractBaseColumn.class; + } + + @Override + protected AbstractBaseColumn[] getTestData() { + return new AbstractBaseColumn[] { + new NullColumn(), new StringColumn("100"), new StringColumn("123123") + }; + } + + public static class StringColumnChecker implements DeeplyEqualsChecker.CustomEqualityChecker { + @Override + public boolean check(Object o1, Object o2, DeeplyEqualsChecker checker) { + if (o1 instanceof StringColumn && o2 instanceof StringColumn) { + return ((StringColumn) o1).getData().equals(((StringColumn) o2).getData()); + } else { + return o1 instanceof NullColumn && o2 instanceof NullColumn; + } + } + } +} diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/TimeColumnSerializerTest.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/TimeColumnSerializerTest.java new file mode 100644 index 0000000000..88c8ddbe39 --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/TimeColumnSerializerTest.java @@ -0,0 +1,85 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.element.column.TimeColumn; +import com.dtstack.chunjun.typeutil.serializer.DeeplyEqualsChecker; +import com.dtstack.chunjun.typeutil.serializer.SerializerTestBase; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; + +import java.sql.Time; +import java.util.function.BiFunction; + +/** @author liuliu 2022/5/13 */ +public class TimeColumnSerializerTest extends SerializerTestBase { + + @Override + protected Tuple2, DeeplyEqualsChecker.CustomEqualityChecker> + getCustomChecker() { + return Tuple2.of( + new BiFunction() { + @Override + public Boolean apply(Object o, Object o2) { + return (o instanceof TimeColumn && o2 instanceof TimeColumn) + || (o instanceof NullColumn && o2 instanceof NullColumn); + } + }, + new TimeColumnChecker()); + } + + @Override + protected TypeSerializer createSerializer() { + return TimeColumnSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return AbstractBaseColumn.class; + } + + @Override + protected AbstractBaseColumn[] getTestData() { + return new AbstractBaseColumn[] { + new NullColumn(), + new TimeColumn(100), + new TimeColumn(new Time(System.currentTimeMillis())) + }; + } + + public static class TimeColumnChecker implements DeeplyEqualsChecker.CustomEqualityChecker { + + @Override + public boolean check(Object o1, Object o2, DeeplyEqualsChecker checker) { + if (o1 instanceof TimeColumn && o2 instanceof TimeColumn) { + return ((TimeColumn) o1).asTime().getTime() == ((TimeColumn) o2).asTime().getTime(); + } else { + return o1 instanceof NullColumn && o2 instanceof NullColumn; + } + } + } +} diff --git a/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/TimestampColumnSerializerTest.java b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/TimestampColumnSerializerTest.java new file mode 100644 index 0000000000..fd6b06e072 --- /dev/null +++ b/chunjun-core/src/test/java/com/dtstack/chunjun/typeutil/serializer/base/TimestampColumnSerializerTest.java @@ -0,0 +1,87 @@ +/* + * 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 com.dtstack.chunjun.typeutil.serializer.base; + +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.column.NullColumn; +import com.dtstack.chunjun.element.column.TimestampColumn; +import com.dtstack.chunjun.typeutil.serializer.DeeplyEqualsChecker; +import com.dtstack.chunjun.typeutil.serializer.SerializerTestBase; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; + +import java.sql.Timestamp; +import java.util.function.BiFunction; + +/** @author liuliu 2022/5/13 */ +public class TimestampColumnSerializerTest extends SerializerTestBase { + + @Override + protected Tuple2, DeeplyEqualsChecker.CustomEqualityChecker> + getCustomChecker() { + return Tuple2.of( + new BiFunction() { + @Override + public Boolean apply(Object o, Object o2) { + return (o instanceof TimestampColumn && o2 instanceof TimestampColumn) + || (o instanceof NullColumn && o2 instanceof NullColumn); + } + }, + new TimestampColumnChecker()); + } + + @Override + protected TypeSerializer createSerializer() { + return TimestampColumnSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return AbstractBaseColumn.class; + } + + @Override + protected AbstractBaseColumn[] getTestData() { + return new AbstractBaseColumn[] { + new NullColumn(), + new TimestampColumn(100), + new TimestampColumn(new Timestamp(System.currentTimeMillis())) + }; + } + + public static class TimestampColumnChecker + implements DeeplyEqualsChecker.CustomEqualityChecker { + + @Override + public boolean check(Object o1, Object o2, DeeplyEqualsChecker checker) { + if (o1 instanceof TimestampColumn && o2 instanceof TimestampColumn) { + return ((TimestampColumn) o1).asTimestamp().getTime() + == ((TimestampColumn) o2).asTimestamp().getTime(); + } else { + return o1 instanceof NullColumn && o2 instanceof NullColumn; + } + } + } +} diff --git a/chunjun-local-test/src/main/java/com/dtstack/chunjun/local/test/LocalTest.java b/chunjun-local-test/src/main/java/com/dtstack/chunjun/local/test/LocalTest.java index 14279f26cd..92cfdb493e 100644 --- a/chunjun-local-test/src/main/java/com/dtstack/chunjun/local/test/LocalTest.java +++ b/chunjun-local-test/src/main/java/com/dtstack/chunjun/local/test/LocalTest.java @@ -52,7 +52,7 @@ public static void main(String[] args) throws Exception { // confProperties.setProperty("state.checkpoints.dir", "file:///ck"); String userDir = System.getProperty("user.dir"); - String jobPath = userDir + "/chunjun-examples/json/stream/stream.json"; + String jobPath = userDir + "/chunjun-examples/json/oracle/oracle_stream.json"; String chunjunDistDir = userDir + "/start-chunjun-dist"; String s = "";