[oceanbase] Add new deserialization schema with runtime converter (#1356)
This closes #980.pull/1569/head
parent
d3435381a6
commit
b1b092c97e
@ -0,0 +1,47 @@
|
|||||||
|
/*
|
||||||
|
* Copyright 2022 Ververica Inc.
|
||||||
|
*
|
||||||
|
* Licensed 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.ververica.cdc.connectors.oceanbase.source;
|
||||||
|
|
||||||
|
import com.oceanbase.oms.logmessage.ByteString;
|
||||||
|
import sun.reflect.generics.reflectiveObjects.NotImplementedException;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
import java.nio.charset.StandardCharsets;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Runtime converter that converts objects of OceanBase into objects of Flink Table & SQL internal
|
||||||
|
* data structures.
|
||||||
|
*/
|
||||||
|
public interface OceanBaseDeserializationRuntimeConverter extends Serializable {
|
||||||
|
|
||||||
|
default Object convert(Object object) throws Exception {
|
||||||
|
if (object instanceof ByteString) {
|
||||||
|
return convertChangeEvent(
|
||||||
|
((ByteString) object).toString(StandardCharsets.UTF_8.name()));
|
||||||
|
} else {
|
||||||
|
return convertSnapshotEvent(object);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
default Object convertSnapshotEvent(Object object) throws Exception {
|
||||||
|
throw new NotImplementedException();
|
||||||
|
}
|
||||||
|
|
||||||
|
default Object convertChangeEvent(String string) throws Exception {
|
||||||
|
throw new NotImplementedException();
|
||||||
|
}
|
||||||
|
}
|
@ -1,220 +0,0 @@
|
|||||||
/*
|
|
||||||
* Copyright 2022 Ververica Inc.
|
|
||||||
*
|
|
||||||
* Licensed 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.ververica.cdc.connectors.oceanbase.source;
|
|
||||||
|
|
||||||
import com.oceanbase.oms.logmessage.ByteString;
|
|
||||||
import com.oceanbase.oms.logmessage.DataMessage;
|
|
||||||
import io.debezium.config.CommonConnectorConfig;
|
|
||||||
import io.debezium.jdbc.JdbcValueConverters;
|
|
||||||
import io.debezium.jdbc.TemporalPrecisionMode;
|
|
||||||
import io.debezium.relational.ValueConverterProvider;
|
|
||||||
import io.debezium.util.NumberConversions;
|
|
||||||
import org.apache.kafka.connect.data.Schema;
|
|
||||||
|
|
||||||
import java.math.BigDecimal;
|
|
||||||
import java.math.BigInteger;
|
|
||||||
import java.nio.ByteBuffer;
|
|
||||||
import java.nio.charset.StandardCharsets;
|
|
||||||
import java.sql.Date;
|
|
||||||
import java.sql.Time;
|
|
||||||
import java.sql.Timestamp;
|
|
||||||
import java.sql.Types;
|
|
||||||
import java.time.ZoneOffset;
|
|
||||||
import java.util.Arrays;
|
|
||||||
|
|
||||||
/** Utils to convert jdbc type and value of a field. */
|
|
||||||
public class OceanBaseJdbcConverter {
|
|
||||||
|
|
||||||
public static ValueConverterProvider valueConverterProvider(ZoneOffset zoneOffset) {
|
|
||||||
return new JdbcValueConverters(
|
|
||||||
JdbcValueConverters.DecimalMode.STRING,
|
|
||||||
TemporalPrecisionMode.ADAPTIVE_TIME_MICROSECONDS,
|
|
||||||
zoneOffset,
|
|
||||||
null,
|
|
||||||
JdbcValueConverters.BigIntUnsignedMode.PRECISE,
|
|
||||||
CommonConnectorConfig.BinaryHandlingMode.BYTES);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static Object getField(int jdbcType, Object value) {
|
|
||||||
if (value == null) {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
jdbcType = getType(jdbcType, null);
|
|
||||||
switch (jdbcType) {
|
|
||||||
case Types.BIT:
|
|
||||||
if (value instanceof Boolean) {
|
|
||||||
return new byte[] {NumberConversions.getByte((Boolean) value)};
|
|
||||||
}
|
|
||||||
return value;
|
|
||||||
case Types.INTEGER:
|
|
||||||
if (value instanceof Boolean) {
|
|
||||||
return NumberConversions.getInteger((Boolean) value);
|
|
||||||
}
|
|
||||||
if (value instanceof Date) {
|
|
||||||
return ((Date) value).getYear() + 1900;
|
|
||||||
}
|
|
||||||
return value;
|
|
||||||
case Types.FLOAT:
|
|
||||||
Float f = (Float) value;
|
|
||||||
return f.doubleValue();
|
|
||||||
case Types.DECIMAL:
|
|
||||||
if (value instanceof BigInteger) {
|
|
||||||
return value.toString();
|
|
||||||
}
|
|
||||||
BigDecimal decimal = (BigDecimal) value;
|
|
||||||
return decimal.toString();
|
|
||||||
case Types.DATE:
|
|
||||||
Date date = (Date) value;
|
|
||||||
return io.debezium.time.Date.toEpochDay(date, null);
|
|
||||||
case Types.TIME:
|
|
||||||
Time time = (Time) value;
|
|
||||||
return io.debezium.time.MicroTime.toMicroOfDay(time, true);
|
|
||||||
case Types.TIMESTAMP:
|
|
||||||
Timestamp timestamp = (Timestamp) value;
|
|
||||||
return io.debezium.time.MicroTimestamp.toEpochMicros(timestamp, null);
|
|
||||||
default:
|
|
||||||
return value;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static Object getField(
|
|
||||||
Schema.Type schemaType, DataMessage.Record.Field.Type fieldType, ByteString value) {
|
|
||||||
if (value == null) {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
int jdbcType = getType(fieldType);
|
|
||||||
switch (jdbcType) {
|
|
||||||
case Types.NULL:
|
|
||||||
return null;
|
|
||||||
case Types.INTEGER:
|
|
||||||
if (schemaType.equals(Schema.Type.INT64)) {
|
|
||||||
return Long.parseLong(value.toString());
|
|
||||||
}
|
|
||||||
return Integer.parseInt(value.toString());
|
|
||||||
case Types.BIGINT:
|
|
||||||
if (schemaType.equals(Schema.Type.STRING)) {
|
|
||||||
return value.toString();
|
|
||||||
}
|
|
||||||
return Long.parseLong(value.toString());
|
|
||||||
case Types.DOUBLE:
|
|
||||||
return Double.parseDouble(value.toString());
|
|
||||||
case Types.DATE:
|
|
||||||
Date date = Date.valueOf(value.toString());
|
|
||||||
return io.debezium.time.Date.toEpochDay(date, null);
|
|
||||||
case Types.TIME:
|
|
||||||
Time time = Time.valueOf(value.toString());
|
|
||||||
return io.debezium.time.MicroTime.toMicroOfDay(time, true);
|
|
||||||
case Types.TIMESTAMP:
|
|
||||||
Timestamp timestamp = Timestamp.valueOf(value.toString());
|
|
||||||
return io.debezium.time.MicroTimestamp.toEpochMicros(timestamp, null);
|
|
||||||
case Types.BIT:
|
|
||||||
long v = Long.parseLong(value.toString());
|
|
||||||
byte[] bytes = ByteBuffer.allocate(8).putLong(v).array();
|
|
||||||
int i = 0;
|
|
||||||
while (bytes[i] == 0 && i < Long.BYTES - 1) {
|
|
||||||
i++;
|
|
||||||
}
|
|
||||||
return Arrays.copyOfRange(bytes, i, Long.BYTES);
|
|
||||||
case Types.BINARY:
|
|
||||||
return ByteBuffer.wrap(value.toString().getBytes(StandardCharsets.UTF_8));
|
|
||||||
default:
|
|
||||||
return value.toString(StandardCharsets.UTF_8.toString());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static boolean isBoolean(int jdbcType, String typeName) {
|
|
||||||
return jdbcType == Types.BOOLEAN || (jdbcType == Types.BIT && "TINYINT".equals(typeName));
|
|
||||||
}
|
|
||||||
|
|
||||||
public static int getType(int jdbcType, String typeName) {
|
|
||||||
// treat boolean as tinyint type
|
|
||||||
if (isBoolean(jdbcType, typeName)) {
|
|
||||||
jdbcType = Types.TINYINT;
|
|
||||||
}
|
|
||||||
// treat year as int type
|
|
||||||
if ("YEAR".equals(typeName)) {
|
|
||||||
jdbcType = Types.INTEGER;
|
|
||||||
}
|
|
||||||
|
|
||||||
// upcasting
|
|
||||||
if ("INT UNSIGNED".equals(typeName)) {
|
|
||||||
jdbcType = Types.BIGINT;
|
|
||||||
}
|
|
||||||
if ("BIGINT UNSIGNED".equals(typeName)) {
|
|
||||||
jdbcType = Types.DECIMAL;
|
|
||||||
}
|
|
||||||
|
|
||||||
// widening conversion according to com.mysql.jdbc.ResultSetImpl#getObject
|
|
||||||
switch (jdbcType) {
|
|
||||||
case Types.TINYINT:
|
|
||||||
case Types.SMALLINT:
|
|
||||||
return Types.INTEGER;
|
|
||||||
case Types.REAL:
|
|
||||||
return Types.FLOAT;
|
|
||||||
default:
|
|
||||||
return jdbcType;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static int getType(DataMessage.Record.Field.Type fieldType) {
|
|
||||||
switch (fieldType) {
|
|
||||||
case NULL:
|
|
||||||
return Types.NULL;
|
|
||||||
case INT8:
|
|
||||||
case INT16:
|
|
||||||
case INT24:
|
|
||||||
case INT32:
|
|
||||||
case YEAR:
|
|
||||||
return Types.INTEGER;
|
|
||||||
case INT64:
|
|
||||||
return Types.BIGINT;
|
|
||||||
case FLOAT:
|
|
||||||
case DOUBLE:
|
|
||||||
return Types.DOUBLE;
|
|
||||||
case DECIMAL:
|
|
||||||
return Types.DECIMAL;
|
|
||||||
case ENUM:
|
|
||||||
case SET:
|
|
||||||
case STRING:
|
|
||||||
case JSON:
|
|
||||||
return Types.CHAR;
|
|
||||||
case TIMESTAMP:
|
|
||||||
case DATETIME:
|
|
||||||
case TIMESTAMP_WITH_TIME_ZONE:
|
|
||||||
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
|
|
||||||
case TIMESTAMP_NANO:
|
|
||||||
return Types.TIMESTAMP;
|
|
||||||
case DATE:
|
|
||||||
return Types.DATE;
|
|
||||||
case TIME:
|
|
||||||
return Types.TIME;
|
|
||||||
case BIT:
|
|
||||||
return Types.BIT;
|
|
||||||
case BLOB:
|
|
||||||
case BINARY:
|
|
||||||
return Types.BINARY;
|
|
||||||
case INTERVAL_YEAR_TO_MONTH:
|
|
||||||
case INTERVAL_DAY_TO_SECOND:
|
|
||||||
case GEOMETRY:
|
|
||||||
case RAW:
|
|
||||||
// it's weird to get wrong type from TEXT column, temporarily treat it as a string
|
|
||||||
case UNKOWN:
|
|
||||||
default:
|
|
||||||
return Types.VARCHAR;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,51 +0,0 @@
|
|||||||
/*
|
|
||||||
* Copyright 2022 Ververica Inc.
|
|
||||||
*
|
|
||||||
* Licensed 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.ververica.cdc.connectors.oceanbase.source;
|
|
||||||
|
|
||||||
import org.apache.kafka.connect.data.Schema;
|
|
||||||
import org.apache.kafka.connect.data.SchemaBuilder;
|
|
||||||
import org.apache.kafka.connect.data.Struct;
|
|
||||||
|
|
||||||
/** Utils to deal with OceanBase SourceRecord schema. */
|
|
||||||
public class OceanBaseSchemaUtils {
|
|
||||||
|
|
||||||
public static Schema sourceSchema() {
|
|
||||||
return SchemaBuilder.struct()
|
|
||||||
.field("tenant", Schema.STRING_SCHEMA)
|
|
||||||
.field("database", Schema.STRING_SCHEMA)
|
|
||||||
.field("table", Schema.STRING_SCHEMA)
|
|
||||||
.field("timestamp", Schema.OPTIONAL_STRING_SCHEMA)
|
|
||||||
.field("unique_id", Schema.OPTIONAL_STRING_SCHEMA)
|
|
||||||
.build();
|
|
||||||
}
|
|
||||||
|
|
||||||
public static Struct sourceStruct(
|
|
||||||
String tenant, String database, String table, String timestamp, String uniqueId) {
|
|
||||||
Struct struct =
|
|
||||||
new Struct(sourceSchema())
|
|
||||||
.put("tenant", tenant)
|
|
||||||
.put("database", database)
|
|
||||||
.put("table", table);
|
|
||||||
if (timestamp != null) {
|
|
||||||
struct.put("timestamp", timestamp);
|
|
||||||
}
|
|
||||||
if (uniqueId != null) {
|
|
||||||
struct.put("unique_id", uniqueId);
|
|
||||||
}
|
|
||||||
return struct;
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,128 +0,0 @@
|
|||||||
/*
|
|
||||||
* Copyright 2022 Ververica Inc.
|
|
||||||
*
|
|
||||||
* Licensed 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.ververica.cdc.connectors.oceanbase.source;
|
|
||||||
|
|
||||||
import io.debezium.data.Envelope;
|
|
||||||
import io.debezium.relational.Column;
|
|
||||||
import io.debezium.relational.ColumnEditor;
|
|
||||||
import io.debezium.relational.CustomConverterRegistry;
|
|
||||||
import io.debezium.relational.Table;
|
|
||||||
import io.debezium.relational.TableEditor;
|
|
||||||
import io.debezium.relational.TableId;
|
|
||||||
import io.debezium.relational.TableSchema;
|
|
||||||
import io.debezium.relational.TableSchemaBuilder;
|
|
||||||
import io.debezium.util.SchemaNameAdjuster;
|
|
||||||
import org.apache.kafka.connect.data.Schema;
|
|
||||||
import org.apache.kafka.connect.data.SchemaBuilder;
|
|
||||||
|
|
||||||
import java.math.BigInteger;
|
|
||||||
import java.sql.Types;
|
|
||||||
import java.time.ZoneOffset;
|
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
/** Utils to deal with table schema of OceanBase. */
|
|
||||||
public class OceanBaseTableSchema {
|
|
||||||
|
|
||||||
public static TableSchemaBuilder tableSchemaBuilder(ZoneOffset zoneOffset) {
|
|
||||||
return new TableSchemaBuilder(
|
|
||||||
OceanBaseJdbcConverter.valueConverterProvider(zoneOffset),
|
|
||||||
SchemaNameAdjuster.create(),
|
|
||||||
new CustomConverterRegistry(null),
|
|
||||||
OceanBaseSchemaUtils.sourceSchema(),
|
|
||||||
false);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static TableId tableId(String databaseName, String tableName) {
|
|
||||||
return new TableId(databaseName, null, tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static Column getColumn(String name, int jdbcType) {
|
|
||||||
// we can't get the scale and length of decimal, timestamp and bit columns from log,
|
|
||||||
// so here we set a constant value to these fields to be compatible with the logic of
|
|
||||||
// JdbcValueConverters#schemaBuilder
|
|
||||||
ColumnEditor columnEditor =
|
|
||||||
Column.editor().name(name).jdbcType(jdbcType).optional(true).scale(0);
|
|
||||||
if (columnEditor.jdbcType() == Types.TIMESTAMP || columnEditor.jdbcType() == Types.BIT) {
|
|
||||||
columnEditor.length(6);
|
|
||||||
}
|
|
||||||
return columnEditor.create();
|
|
||||||
}
|
|
||||||
|
|
||||||
public static TableSchema getTableSchema(
|
|
||||||
String topicName,
|
|
||||||
String databaseName,
|
|
||||||
String tableName,
|
|
||||||
String[] columnNames,
|
|
||||||
int[] jdbcTypes,
|
|
||||||
ZoneOffset zoneOffset) {
|
|
||||||
TableEditor tableEditor = Table.editor().tableId(tableId(databaseName, tableName));
|
|
||||||
for (int i = 0; i < columnNames.length; i++) {
|
|
||||||
tableEditor.addColumn(getColumn(columnNames[i], jdbcTypes[i]));
|
|
||||||
}
|
|
||||||
return tableSchemaBuilder(zoneOffset)
|
|
||||||
.create(
|
|
||||||
null,
|
|
||||||
Envelope.schemaName(topicName),
|
|
||||||
tableEditor.create(),
|
|
||||||
null,
|
|
||||||
null,
|
|
||||||
null);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static Schema upcastingSchemaType(Schema schema, String value) {
|
|
||||||
if (schema.type().equals(Schema.Type.INT32) && Long.parseLong(value) > Integer.MAX_VALUE) {
|
|
||||||
return Schema.INT64_SCHEMA;
|
|
||||||
}
|
|
||||||
if (schema.type().equals(Schema.Type.INT64)) {
|
|
||||||
BigInteger bigInt = new BigInteger(value);
|
|
||||||
if (bigInt.compareTo(BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
|
|
||||||
return Schema.STRING_SCHEMA;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return schema;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static Schema upcastingValueSchema(Schema valueSchema, Map<String, String> fields) {
|
|
||||||
SchemaBuilder schemaBuilder = SchemaBuilder.struct().optional();
|
|
||||||
for (Map.Entry<String, String> entry : fields.entrySet()) {
|
|
||||||
Schema fieldSchema = valueSchema.field(entry.getKey()).schema();
|
|
||||||
fieldSchema = upcastingSchemaType(fieldSchema, entry.getValue());
|
|
||||||
schemaBuilder.field(entry.getKey(), fieldSchema);
|
|
||||||
}
|
|
||||||
return schemaBuilder.build();
|
|
||||||
}
|
|
||||||
|
|
||||||
public static Envelope getEnvelope(String name, Schema valueSchema) {
|
|
||||||
return Envelope.defineSchema()
|
|
||||||
.withName(name)
|
|
||||||
.withRecord(valueSchema)
|
|
||||||
.withSource(OceanBaseSchemaUtils.sourceSchema())
|
|
||||||
.build();
|
|
||||||
}
|
|
||||||
|
|
||||||
public static TableSchema upcastingTableSchema(
|
|
||||||
String topicName, TableSchema tableSchema, Map<String, String> fields) {
|
|
||||||
Schema valueSchema = upcastingValueSchema(tableSchema.valueSchema(), fields);
|
|
||||||
return new TableSchema(
|
|
||||||
tableSchema.id(),
|
|
||||||
null,
|
|
||||||
null,
|
|
||||||
getEnvelope(Envelope.schemaName(topicName), valueSchema),
|
|
||||||
valueSchema,
|
|
||||||
null);
|
|
||||||
}
|
|
||||||
}
|
|
@ -0,0 +1,679 @@
|
|||||||
|
/*
|
||||||
|
* Copyright 2022 Ververica Inc.
|
||||||
|
*
|
||||||
|
* Licensed 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.ververica.cdc.connectors.oceanbase.source;
|
||||||
|
|
||||||
|
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
||||||
|
import org.apache.flink.table.data.DecimalData;
|
||||||
|
import org.apache.flink.table.data.GenericArrayData;
|
||||||
|
import org.apache.flink.table.data.GenericRowData;
|
||||||
|
import org.apache.flink.table.data.RowData;
|
||||||
|
import org.apache.flink.table.data.StringData;
|
||||||
|
import org.apache.flink.table.data.TimestampData;
|
||||||
|
import org.apache.flink.table.types.logical.DecimalType;
|
||||||
|
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.Collector;
|
||||||
|
|
||||||
|
import com.oceanbase.oms.logmessage.ByteString;
|
||||||
|
import com.ververica.cdc.connectors.oceanbase.table.OceanBaseAppendMetadataCollector;
|
||||||
|
import com.ververica.cdc.connectors.oceanbase.table.OceanBaseDeserializationSchema;
|
||||||
|
import com.ververica.cdc.connectors.oceanbase.table.OceanBaseMetadataConverter;
|
||||||
|
import com.ververica.cdc.connectors.oceanbase.table.OceanBaseRecord;
|
||||||
|
import com.ververica.cdc.debezium.utils.TemporalConversions;
|
||||||
|
|
||||||
|
import java.math.BigDecimal;
|
||||||
|
import java.math.BigInteger;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.nio.charset.StandardCharsets;
|
||||||
|
import java.sql.Date;
|
||||||
|
import java.sql.Time;
|
||||||
|
import java.sql.Timestamp;
|
||||||
|
import java.time.ZoneId;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import static org.apache.flink.util.Preconditions.checkNotNull;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Deserialization schema from OceanBase object to Flink Table/SQL internal data structure {@link
|
||||||
|
* RowData}.
|
||||||
|
*/
|
||||||
|
public class RowDataOceanBaseDeserializationSchema
|
||||||
|
implements OceanBaseDeserializationSchema<RowData> {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
/** TypeInformation of the produced {@link RowData}. * */
|
||||||
|
private final TypeInformation<RowData> resultTypeInfo;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Runtime converter that OceanBase record data into {@link RowData} consisted of physical
|
||||||
|
* column values.
|
||||||
|
*/
|
||||||
|
private final OceanBaseDeserializationRuntimeConverter physicalConverter;
|
||||||
|
|
||||||
|
/** Whether the deserializer needs to handle metadata columns. */
|
||||||
|
private final boolean hasMetadata;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A wrapped output collector which is used to append metadata columns after physical columns.
|
||||||
|
*/
|
||||||
|
private final OceanBaseAppendMetadataCollector appendMetadataCollector;
|
||||||
|
|
||||||
|
/** Returns a builder to build {@link RowDataOceanBaseDeserializationSchema}. */
|
||||||
|
public static RowDataOceanBaseDeserializationSchema.Builder newBuilder() {
|
||||||
|
return new RowDataOceanBaseDeserializationSchema.Builder();
|
||||||
|
}
|
||||||
|
|
||||||
|
RowDataOceanBaseDeserializationSchema(
|
||||||
|
RowType physicalDataType,
|
||||||
|
OceanBaseMetadataConverter[] metadataConverters,
|
||||||
|
TypeInformation<RowData> resultTypeInfo,
|
||||||
|
ZoneId serverTimeZone) {
|
||||||
|
this.hasMetadata = checkNotNull(metadataConverters).length > 0;
|
||||||
|
this.appendMetadataCollector = new OceanBaseAppendMetadataCollector(metadataConverters);
|
||||||
|
this.physicalConverter = createConverter(checkNotNull(physicalDataType), serverTimeZone);
|
||||||
|
this.resultTypeInfo = checkNotNull(resultTypeInfo);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void deserialize(OceanBaseRecord record, Collector<RowData> out) throws Exception {
|
||||||
|
RowData physicalRow;
|
||||||
|
if (record.isSnapshotRecord()) {
|
||||||
|
physicalRow = (GenericRowData) physicalConverter.convert(record.getJdbcFields());
|
||||||
|
physicalRow.setRowKind(RowKind.INSERT);
|
||||||
|
emit(record, physicalRow, out);
|
||||||
|
} else {
|
||||||
|
switch (record.getOpt()) {
|
||||||
|
case INSERT:
|
||||||
|
physicalRow =
|
||||||
|
(GenericRowData)
|
||||||
|
physicalConverter.convert(record.getLogMessageFieldsAfter());
|
||||||
|
physicalRow.setRowKind(RowKind.INSERT);
|
||||||
|
emit(record, physicalRow, out);
|
||||||
|
break;
|
||||||
|
case DELETE:
|
||||||
|
physicalRow =
|
||||||
|
(GenericRowData)
|
||||||
|
physicalConverter.convert(record.getLogMessageFieldsBefore());
|
||||||
|
physicalRow.setRowKind(RowKind.DELETE);
|
||||||
|
emit(record, physicalRow, out);
|
||||||
|
break;
|
||||||
|
case UPDATE:
|
||||||
|
physicalRow =
|
||||||
|
(GenericRowData)
|
||||||
|
physicalConverter.convert(record.getLogMessageFieldsBefore());
|
||||||
|
physicalRow.setRowKind(RowKind.UPDATE_BEFORE);
|
||||||
|
emit(record, physicalRow, out);
|
||||||
|
physicalRow =
|
||||||
|
(GenericRowData)
|
||||||
|
physicalConverter.convert(record.getLogMessageFieldsAfter());
|
||||||
|
physicalRow.setRowKind(RowKind.UPDATE_AFTER);
|
||||||
|
emit(record, physicalRow, out);
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw new IllegalArgumentException(
|
||||||
|
"Unsupported log message record type: " + record.getOpt());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void emit(OceanBaseRecord row, RowData physicalRow, Collector<RowData> collector) {
|
||||||
|
if (!hasMetadata) {
|
||||||
|
collector.collect(physicalRow);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
appendMetadataCollector.inputRecord = row;
|
||||||
|
appendMetadataCollector.outputCollector = collector;
|
||||||
|
appendMetadataCollector.collect(physicalRow);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public TypeInformation<RowData> getProducedType() {
|
||||||
|
return resultTypeInfo;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Builder class of {@link RowDataOceanBaseDeserializationSchema}. */
|
||||||
|
public static class Builder {
|
||||||
|
private RowType physicalRowType;
|
||||||
|
private TypeInformation<RowData> resultTypeInfo;
|
||||||
|
private OceanBaseMetadataConverter[] metadataConverters = new OceanBaseMetadataConverter[0];
|
||||||
|
private ZoneId serverTimeZone = ZoneId.of("UTC");
|
||||||
|
|
||||||
|
public RowDataOceanBaseDeserializationSchema.Builder setPhysicalRowType(
|
||||||
|
RowType physicalRowType) {
|
||||||
|
this.physicalRowType = physicalRowType;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public RowDataOceanBaseDeserializationSchema.Builder setMetadataConverters(
|
||||||
|
OceanBaseMetadataConverter[] metadataConverters) {
|
||||||
|
this.metadataConverters = metadataConverters;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public RowDataOceanBaseDeserializationSchema.Builder setResultTypeInfo(
|
||||||
|
TypeInformation<RowData> resultTypeInfo) {
|
||||||
|
this.resultTypeInfo = resultTypeInfo;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public RowDataOceanBaseDeserializationSchema.Builder setServerTimeZone(
|
||||||
|
ZoneId serverTimeZone) {
|
||||||
|
this.serverTimeZone = serverTimeZone;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public RowDataOceanBaseDeserializationSchema build() {
|
||||||
|
return new RowDataOceanBaseDeserializationSchema(
|
||||||
|
physicalRowType, metadataConverters, resultTypeInfo, serverTimeZone);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter createConverter(
|
||||||
|
LogicalType type, ZoneId serverTimeZone) {
|
||||||
|
return wrapIntoNullableConverter(createNotNullConverter(type, serverTimeZone));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter wrapIntoNullableConverter(
|
||||||
|
OceanBaseDeserializationRuntimeConverter converter) {
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convert(Object object) throws Exception {
|
||||||
|
if (object == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
return converter.convert(object);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
public static OceanBaseDeserializationRuntimeConverter createNotNullConverter(
|
||||||
|
LogicalType type, ZoneId serverTimeZone) {
|
||||||
|
switch (type.getTypeRoot()) {
|
||||||
|
case ROW:
|
||||||
|
return createRowConverter((RowType) type, serverTimeZone);
|
||||||
|
case NULL:
|
||||||
|
return convertToNull();
|
||||||
|
case BOOLEAN:
|
||||||
|
return convertToBoolean();
|
||||||
|
case TINYINT:
|
||||||
|
return convertToTinyInt();
|
||||||
|
case SMALLINT:
|
||||||
|
return convertToSmallInt();
|
||||||
|
case INTEGER:
|
||||||
|
case INTERVAL_YEAR_MONTH:
|
||||||
|
return convertToInt();
|
||||||
|
case BIGINT:
|
||||||
|
case INTERVAL_DAY_TIME:
|
||||||
|
return convertToLong();
|
||||||
|
case DATE:
|
||||||
|
return convertToDate();
|
||||||
|
case TIME_WITHOUT_TIME_ZONE:
|
||||||
|
return convertToTime();
|
||||||
|
case TIMESTAMP_WITHOUT_TIME_ZONE:
|
||||||
|
return convertToTimestamp();
|
||||||
|
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
|
||||||
|
return convertToLocalTimeZoneTimestamp(serverTimeZone);
|
||||||
|
case FLOAT:
|
||||||
|
return convertToFloat();
|
||||||
|
case DOUBLE:
|
||||||
|
return convertToDouble();
|
||||||
|
case CHAR:
|
||||||
|
case VARCHAR:
|
||||||
|
return convertToString();
|
||||||
|
case BINARY:
|
||||||
|
return convertToBinary();
|
||||||
|
case VARBINARY:
|
||||||
|
return convertToBytes();
|
||||||
|
case DECIMAL:
|
||||||
|
return createDecimalConverter((DecimalType) type);
|
||||||
|
case ARRAY:
|
||||||
|
return createArrayConverter();
|
||||||
|
default:
|
||||||
|
throw new UnsupportedOperationException("Unsupported type: " + type);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter createRowConverter(
|
||||||
|
RowType rowType, ZoneId serverTimeZone) {
|
||||||
|
final OceanBaseDeserializationRuntimeConverter[] fieldConverters =
|
||||||
|
rowType.getFields().stream()
|
||||||
|
.map(RowType.RowField::getType)
|
||||||
|
.map(logicType -> createConverter(logicType, serverTimeZone))
|
||||||
|
.toArray(OceanBaseDeserializationRuntimeConverter[]::new);
|
||||||
|
final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convert(Object object) throws Exception {
|
||||||
|
int arity = fieldNames.length;
|
||||||
|
GenericRowData row = new GenericRowData(arity);
|
||||||
|
Map<String, Object> fieldMap = (Map<String, Object>) object;
|
||||||
|
for (int i = 0; i < arity; i++) {
|
||||||
|
String fieldName = fieldNames[i];
|
||||||
|
row.setField(i, fieldConverters[i].convert(fieldMap.get(fieldName)));
|
||||||
|
}
|
||||||
|
return row;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter convertToNull() {
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convert(Object object) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter convertToBoolean() {
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertSnapshotEvent(Object object) {
|
||||||
|
if (object instanceof byte[]) {
|
||||||
|
return "1".equals(new String((byte[]) object, StandardCharsets.UTF_8));
|
||||||
|
}
|
||||||
|
return Boolean.parseBoolean(object.toString()) || "1".equals(object.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertChangeEvent(String string) {
|
||||||
|
return "1".equals(string);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter convertToTinyInt() {
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertSnapshotEvent(Object object) {
|
||||||
|
return Byte.parseByte(object.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertChangeEvent(String string) {
|
||||||
|
return Byte.parseByte(string);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter convertToSmallInt() {
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertSnapshotEvent(Object object) {
|
||||||
|
return Short.parseShort(object.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertChangeEvent(String string) {
|
||||||
|
return Short.parseShort(string);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter convertToInt() {
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertSnapshotEvent(Object object) {
|
||||||
|
if (object instanceof Integer) {
|
||||||
|
return object;
|
||||||
|
} else if (object instanceof Long) {
|
||||||
|
return ((Long) object).intValue();
|
||||||
|
} else if (object instanceof Date) {
|
||||||
|
return ((Date) object).toLocalDate().getYear();
|
||||||
|
} else {
|
||||||
|
return Integer.parseInt(object.toString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertChangeEvent(String string) {
|
||||||
|
return Integer.parseInt(string);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter convertToLong() {
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertSnapshotEvent(Object object) {
|
||||||
|
if (object instanceof Integer) {
|
||||||
|
return ((Integer) object).longValue();
|
||||||
|
} else if (object instanceof Long) {
|
||||||
|
return object;
|
||||||
|
} else {
|
||||||
|
return Long.parseLong(object.toString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertChangeEvent(String string) {
|
||||||
|
return Long.parseLong(string);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter convertToDouble() {
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertSnapshotEvent(Object object) {
|
||||||
|
if (object instanceof Float) {
|
||||||
|
return ((Float) object).doubleValue();
|
||||||
|
} else if (object instanceof Double) {
|
||||||
|
return object;
|
||||||
|
} else {
|
||||||
|
return Double.parseDouble(object.toString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertChangeEvent(String string) {
|
||||||
|
return Double.parseDouble(string);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter convertToFloat() {
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertSnapshotEvent(Object object) {
|
||||||
|
if (object instanceof Float) {
|
||||||
|
return object;
|
||||||
|
} else if (object instanceof Double) {
|
||||||
|
return ((Double) object).floatValue();
|
||||||
|
} else {
|
||||||
|
return Float.parseFloat(object.toString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertChangeEvent(String string) {
|
||||||
|
return Float.parseFloat(string);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter convertToDate() {
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertSnapshotEvent(Object object) {
|
||||||
|
return (int) TemporalConversions.toLocalDate(object).toEpochDay();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertChangeEvent(String string) {
|
||||||
|
return (int) Date.valueOf(string).toLocalDate().toEpochDay();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter convertToTime() {
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertSnapshotEvent(Object object) {
|
||||||
|
if (object instanceof Long) {
|
||||||
|
return (int) ((Long) object / 1000_000);
|
||||||
|
}
|
||||||
|
return TemporalConversions.toLocalTime(object).toSecondOfDay() * 1000;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertChangeEvent(String string) {
|
||||||
|
return TemporalConversions.toLocalTime(Time.valueOf(string)).toSecondOfDay() * 1000;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter convertToTimestamp() {
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertSnapshotEvent(Object object) {
|
||||||
|
if (object instanceof Timestamp) {
|
||||||
|
return TimestampData.fromLocalDateTime(((Timestamp) object).toLocalDateTime());
|
||||||
|
}
|
||||||
|
throw new IllegalArgumentException(
|
||||||
|
"Unable to convert to TimestampData from unexpected value '"
|
||||||
|
+ object
|
||||||
|
+ "' of type "
|
||||||
|
+ object.getClass().getName());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertChangeEvent(String string) {
|
||||||
|
return TimestampData.fromLocalDateTime(Timestamp.valueOf(string).toLocalDateTime());
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter convertToLocalTimeZoneTimestamp(
|
||||||
|
ZoneId serverTimeZone) {
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertSnapshotEvent(Object object) {
|
||||||
|
if (object instanceof Timestamp) {
|
||||||
|
return TimestampData.fromInstant(
|
||||||
|
((Timestamp) object)
|
||||||
|
.toLocalDateTime()
|
||||||
|
.atZone(serverTimeZone)
|
||||||
|
.toInstant());
|
||||||
|
}
|
||||||
|
throw new IllegalArgumentException(
|
||||||
|
"Unable to convert to TimestampData from unexpected value '"
|
||||||
|
+ object
|
||||||
|
+ "' of type "
|
||||||
|
+ object.getClass().getName());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertChangeEvent(String string) {
|
||||||
|
return TimestampData.fromInstant(
|
||||||
|
Timestamp.valueOf(string)
|
||||||
|
.toLocalDateTime()
|
||||||
|
.atZone(serverTimeZone)
|
||||||
|
.toInstant());
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter convertToString() {
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertSnapshotEvent(Object object) {
|
||||||
|
return StringData.fromString(object.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertChangeEvent(String string) {
|
||||||
|
return StringData.fromString(string);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter convertToBinary() {
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertSnapshotEvent(Object object) {
|
||||||
|
if (object instanceof byte[]) {
|
||||||
|
String str = new String((byte[]) object, StandardCharsets.US_ASCII);
|
||||||
|
return str.getBytes(StandardCharsets.UTF_8);
|
||||||
|
} else if (object instanceof ByteBuffer) {
|
||||||
|
ByteBuffer byteBuffer = (ByteBuffer) object;
|
||||||
|
byte[] bytes = new byte[byteBuffer.remaining()];
|
||||||
|
byteBuffer.get(bytes);
|
||||||
|
return bytes;
|
||||||
|
} else {
|
||||||
|
throw new UnsupportedOperationException(
|
||||||
|
"Unsupported BINARY value type: " + object.getClass().getSimpleName());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertChangeEvent(String string) {
|
||||||
|
try {
|
||||||
|
long v = Long.parseLong(string);
|
||||||
|
byte[] bytes = ByteBuffer.allocate(8).putLong(v).array();
|
||||||
|
int i = 0;
|
||||||
|
while (i < Long.BYTES - 1 && bytes[i] == 0) {
|
||||||
|
i++;
|
||||||
|
}
|
||||||
|
return Arrays.copyOfRange(bytes, i, Long.BYTES);
|
||||||
|
} catch (NumberFormatException e) {
|
||||||
|
return string.getBytes(StandardCharsets.UTF_8);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter convertToBytes() {
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertSnapshotEvent(Object object) {
|
||||||
|
if (object instanceof byte[]) {
|
||||||
|
return object;
|
||||||
|
} else if (object instanceof ByteBuffer) {
|
||||||
|
ByteBuffer byteBuffer = (ByteBuffer) object;
|
||||||
|
byte[] bytes = new byte[byteBuffer.remaining()];
|
||||||
|
byteBuffer.get(bytes);
|
||||||
|
return bytes;
|
||||||
|
} else {
|
||||||
|
throw new UnsupportedOperationException(
|
||||||
|
"Unsupported BYTES value type: " + object.getClass().getSimpleName());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertChangeEvent(String string) {
|
||||||
|
return string.getBytes(StandardCharsets.UTF_8);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter createDecimalConverter(
|
||||||
|
DecimalType decimalType) {
|
||||||
|
final int precision = decimalType.getPrecision();
|
||||||
|
final int scale = decimalType.getScale();
|
||||||
|
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertSnapshotEvent(Object object) {
|
||||||
|
BigDecimal bigDecimal;
|
||||||
|
if (object instanceof String) {
|
||||||
|
bigDecimal = new BigDecimal((String) object);
|
||||||
|
} else if (object instanceof Long) {
|
||||||
|
bigDecimal = new BigDecimal((Long) object);
|
||||||
|
} else if (object instanceof BigInteger) {
|
||||||
|
bigDecimal = new BigDecimal((BigInteger) object);
|
||||||
|
} else if (object instanceof Double) {
|
||||||
|
bigDecimal = BigDecimal.valueOf((Double) object);
|
||||||
|
} else if (object instanceof BigDecimal) {
|
||||||
|
bigDecimal = (BigDecimal) object;
|
||||||
|
} else {
|
||||||
|
throw new IllegalArgumentException(
|
||||||
|
"Unable to convert to decimal from unexpected value '"
|
||||||
|
+ object
|
||||||
|
+ "' of type "
|
||||||
|
+ object.getClass());
|
||||||
|
}
|
||||||
|
return DecimalData.fromBigDecimal(bigDecimal, precision, scale);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convertChangeEvent(String string) {
|
||||||
|
return DecimalData.fromBigDecimal(new BigDecimal(string), precision, scale);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static OceanBaseDeserializationRuntimeConverter createArrayConverter() {
|
||||||
|
return new OceanBaseDeserializationRuntimeConverter() {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object convert(Object object) {
|
||||||
|
String s;
|
||||||
|
if (object instanceof ByteString) {
|
||||||
|
s = ((ByteString) object).toString(StandardCharsets.UTF_8.name());
|
||||||
|
} else {
|
||||||
|
s = object.toString();
|
||||||
|
}
|
||||||
|
String[] strArray = s.split(",");
|
||||||
|
StringData[] stringDataArray = new StringData[strArray.length];
|
||||||
|
for (int i = 0; i < strArray.length; i++) {
|
||||||
|
stringDataArray[i] = StringData.fromString(strArray[i]);
|
||||||
|
}
|
||||||
|
return new GenericArrayData(stringDataArray);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,56 @@
|
|||||||
|
/*
|
||||||
|
* Copyright 2022 Ververica Inc.
|
||||||
|
*
|
||||||
|
* Licensed 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.ververica.cdc.connectors.oceanbase.table;
|
||||||
|
|
||||||
|
import org.apache.flink.annotation.Internal;
|
||||||
|
import org.apache.flink.table.data.GenericRowData;
|
||||||
|
import org.apache.flink.table.data.RowData;
|
||||||
|
import org.apache.flink.table.data.utils.JoinedRowData;
|
||||||
|
import org.apache.flink.util.Collector;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
|
||||||
|
/** Emits a row with physical fields and metadata fields. */
|
||||||
|
@Internal
|
||||||
|
public class OceanBaseAppendMetadataCollector implements Collector<RowData>, Serializable {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
private final OceanBaseMetadataConverter[] metadataConverters;
|
||||||
|
|
||||||
|
public transient OceanBaseRecord inputRecord;
|
||||||
|
public transient Collector<RowData> outputCollector;
|
||||||
|
|
||||||
|
public OceanBaseAppendMetadataCollector(OceanBaseMetadataConverter[] metadataConverters) {
|
||||||
|
this.metadataConverters = metadataConverters;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void collect(RowData physicalRow) {
|
||||||
|
GenericRowData metaRow = new GenericRowData(metadataConverters.length);
|
||||||
|
for (int i = 0; i < metadataConverters.length; i++) {
|
||||||
|
Object meta = metadataConverters[i].read(inputRecord);
|
||||||
|
metaRow.setField(i, meta);
|
||||||
|
}
|
||||||
|
RowData outRow = new JoinedRowData(physicalRow.getRowKind(), physicalRow, metaRow);
|
||||||
|
outputCollector.collect(outRow);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
// nothing to do
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,36 @@
|
|||||||
|
/*
|
||||||
|
* Copyright 2022 Ververica Inc.
|
||||||
|
*
|
||||||
|
* Licensed 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.ververica.cdc.connectors.oceanbase.table;
|
||||||
|
|
||||||
|
import org.apache.flink.annotation.PublicEvolving;
|
||||||
|
import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
|
||||||
|
import org.apache.flink.util.Collector;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The deserialization schema describes how to turn the OceanBase record into data types (Java/Scala
|
||||||
|
* objects) that are processed by Flink.
|
||||||
|
*
|
||||||
|
* @param <T> The type created by the deserialization schema.
|
||||||
|
*/
|
||||||
|
@PublicEvolving
|
||||||
|
public interface OceanBaseDeserializationSchema<T> extends Serializable, ResultTypeQueryable<T> {
|
||||||
|
|
||||||
|
/** Deserialize the OceanBase record, it is represented in {@link OceanBaseRecord}. */
|
||||||
|
void deserialize(OceanBaseRecord record, Collector<T> out) throws Exception;
|
||||||
|
}
|
@ -0,0 +1,28 @@
|
|||||||
|
/*
|
||||||
|
* Copyright 2022 Ververica Inc.
|
||||||
|
*
|
||||||
|
* Licensed 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.ververica.cdc.connectors.oceanbase.table;
|
||||||
|
|
||||||
|
import org.apache.flink.annotation.Internal;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
|
||||||
|
/** A converter converts OceanBase record metadata into Flink internal data structures. */
|
||||||
|
@FunctionalInterface
|
||||||
|
@Internal
|
||||||
|
public interface OceanBaseMetadataConverter extends Serializable {
|
||||||
|
Object read(OceanBaseRecord record);
|
||||||
|
}
|
@ -0,0 +1,118 @@
|
|||||||
|
/*
|
||||||
|
* Copyright 2022 Ververica Inc.
|
||||||
|
*
|
||||||
|
* Licensed 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.ververica.cdc.connectors.oceanbase.table;
|
||||||
|
|
||||||
|
import com.oceanbase.oms.logmessage.ByteString;
|
||||||
|
import com.oceanbase.oms.logmessage.DataMessage;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/** An internal data structure representing record of OceanBase. */
|
||||||
|
public class OceanBaseRecord implements Serializable {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
private final SourceInfo sourceInfo;
|
||||||
|
private final boolean isSnapshotRecord;
|
||||||
|
private Map<String, Object> jdbcFields;
|
||||||
|
private DataMessage.Record.Type opt;
|
||||||
|
private Map<String, ByteString> logMessageFieldsBefore;
|
||||||
|
private Map<String, ByteString> logMessageFieldsAfter;
|
||||||
|
|
||||||
|
public OceanBaseRecord(SourceInfo sourceInfo, Map<String, Object> jdbcFields) {
|
||||||
|
this.sourceInfo = sourceInfo;
|
||||||
|
this.isSnapshotRecord = true;
|
||||||
|
this.jdbcFields = jdbcFields;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OceanBaseRecord(
|
||||||
|
SourceInfo sourceInfo,
|
||||||
|
DataMessage.Record.Type opt,
|
||||||
|
List<DataMessage.Record.Field> logMessageFieldList) {
|
||||||
|
this.sourceInfo = sourceInfo;
|
||||||
|
this.isSnapshotRecord = false;
|
||||||
|
this.opt = opt;
|
||||||
|
this.logMessageFieldsBefore = new HashMap<>();
|
||||||
|
this.logMessageFieldsAfter = new HashMap<>();
|
||||||
|
for (DataMessage.Record.Field field : logMessageFieldList) {
|
||||||
|
if (field.isPrev()) {
|
||||||
|
logMessageFieldsBefore.put(field.getFieldname(), field.getValue());
|
||||||
|
} else {
|
||||||
|
logMessageFieldsAfter.put(field.getFieldname(), field.getValue());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public SourceInfo getSourceInfo() {
|
||||||
|
return sourceInfo;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isSnapshotRecord() {
|
||||||
|
return isSnapshotRecord;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Map<String, Object> getJdbcFields() {
|
||||||
|
return jdbcFields;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataMessage.Record.Type getOpt() {
|
||||||
|
return opt;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Map<String, ByteString> getLogMessageFieldsBefore() {
|
||||||
|
return logMessageFieldsBefore;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Map<String, ByteString> getLogMessageFieldsAfter() {
|
||||||
|
return logMessageFieldsAfter;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Information about the source of record. */
|
||||||
|
public static class SourceInfo implements Serializable {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
private final String tenant;
|
||||||
|
private final String database;
|
||||||
|
private final String table;
|
||||||
|
private final long timestampS;
|
||||||
|
|
||||||
|
public SourceInfo(String tenant, String database, String table, long timestampS) {
|
||||||
|
this.tenant = tenant;
|
||||||
|
this.database = database;
|
||||||
|
this.table = table;
|
||||||
|
this.timestampS = timestampS;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getTenant() {
|
||||||
|
return tenant;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getDatabase() {
|
||||||
|
return database;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getTable() {
|
||||||
|
return table;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getTimestampS() {
|
||||||
|
return timestampS;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue