lvyanquan commented on code in PR #3434: URL: https://github.com/apache/flink-cdc/pull/3434#discussion_r1853759687
########## flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/MapDataSerializer.java: ########## @@ -0,0 +1,327 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.runtime.serializer.data; + +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.java.typeutils.runtime.DataInputViewStream; +import org.apache.flink.api.java.typeutils.runtime.DataOutputViewStream; +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.annotation.VisibleForTesting; +import org.apache.flink.cdc.common.data.ArrayData; +import org.apache.flink.cdc.common.data.MapData; +import org.apache.flink.cdc.common.data.binary.BinaryArrayData; +import org.apache.flink.cdc.common.data.binary.BinaryMapData; +import org.apache.flink.cdc.common.data.binary.BinarySegmentUtils; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.runtime.serializer.InternalSerializers; +import org.apache.flink.cdc.runtime.serializer.data.writer.BinaryArrayWriter; +import org.apache.flink.cdc.runtime.serializer.data.writer.BinaryWriter; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.util.InstantiationUtil; + +import java.io.IOException; + +/** Serializer for {@link MapData}. */ +@Internal +public class MapDataSerializer extends TypeSerializer<MapData> { + + private final DataType keyType; + private final DataType valueType; + + private final TypeSerializer keySerializer; + private final TypeSerializer valueSerializer; + + private final ArrayData.ElementGetter keyGetter; + private final ArrayData.ElementGetter valueGetter; + + private transient BinaryArrayData reuseKeyArray; + private transient BinaryArrayData reuseValueArray; + private transient BinaryArrayWriter reuseKeyWriter; + private transient BinaryArrayWriter reuseValueWriter; + + public MapDataSerializer(DataType keyType, DataType valueType) { + this( + keyType, + valueType, + InternalSerializers.create(keyType), + InternalSerializers.create(valueType)); + } + + private MapDataSerializer( + DataType keyType, + DataType valueType, + TypeSerializer keySerializer, + TypeSerializer valueSerializer) { + this.keyType = keyType; + this.valueType = valueType; + + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + + this.keyGetter = ArrayData.createElementGetter(keyType); + this.valueGetter = ArrayData.createElementGetter(valueType); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer<MapData> duplicate() { + return new MapDataSerializer( + keyType, valueType, keySerializer.duplicate(), valueSerializer.duplicate()); + } + + @Override + public MapData createInstance() { + return new BinaryMapData(); + } + + /** + * NOTE: Map should be a HashMap, when we insert the key/value pairs of the TreeMap into a + * HashMap, problems maybe occur. + */ + @Override + public MapData copy(MapData from) { + if (from instanceof BinaryMapData) { + return ((BinaryMapData) from).copy(); + } else { + return toBinaryMap(from); + } + } + + @Override + public MapData copy(MapData from, MapData reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(MapData record, DataOutputView target) throws IOException { + BinaryMapData binaryMap = toBinaryMap(record); + target.writeInt(binaryMap.getSizeInBytes()); + BinarySegmentUtils.copyToView( + binaryMap.getSegments(), binaryMap.getOffset(), binaryMap.getSizeInBytes(), target); + } + + public BinaryMapData toBinaryMap(MapData from) { + if (from instanceof BinaryMapData) { + return (BinaryMapData) from; + } + + int numElements = from.size(); + if (reuseKeyArray == null) { + reuseKeyArray = new BinaryArrayData(); + } + if (reuseValueArray == null) { + reuseValueArray = new BinaryArrayData(); + } + if (reuseKeyWriter == null || reuseKeyWriter.getNumElements() != numElements) { + reuseKeyWriter = + new BinaryArrayWriter( + reuseKeyArray, + numElements, + BinaryArrayData.calculateFixLengthPartSize(keyType)); + } else { + reuseKeyWriter.reset(); + } + if (reuseValueWriter == null || reuseValueWriter.getNumElements() != numElements) { + reuseValueWriter = + new BinaryArrayWriter( + reuseValueArray, + numElements, + BinaryArrayData.calculateFixLengthPartSize(valueType)); + } else { + reuseValueWriter.reset(); + } + + ArrayData keyArray = from.keyArray(); + ArrayData valueArray = from.valueArray(); + for (int i = 0; i < from.size(); i++) { + Object key = keyGetter.getElementOrNull(keyArray, i); + Object value = valueGetter.getElementOrNull(valueArray, i); + if (key == null) { + reuseKeyWriter.setNullAt(i, keyType); + } else { + BinaryWriter.write(reuseKeyWriter, i, key, keyType, keySerializer); + } + if (value == null) { + reuseValueWriter.setNullAt(i, valueType); + } else { + BinaryWriter.write(reuseValueWriter, i, value, valueType, valueSerializer); + } + } + + reuseKeyWriter.complete(); + reuseValueWriter.complete(); + + return BinaryMapData.valueOf(reuseKeyArray, reuseValueArray); + } + + @Override + public MapData deserialize(DataInputView source) throws IOException { + return deserializeReuse(new BinaryMapData(), source); + } + + @Override + public MapData deserialize(MapData reuse, DataInputView source) throws IOException { + return deserializeReuse( + reuse instanceof BinaryMapData ? (BinaryMapData) reuse : new BinaryMapData(), + source); + } + + private BinaryMapData deserializeReuse(BinaryMapData reuse, DataInputView source) + throws IOException { + int length = source.readInt(); + byte[] bytes = new byte[length]; + source.readFully(bytes); + reuse.pointTo(MemorySegmentFactory.wrap(bytes), 0, bytes.length); + return reuse; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + int length = source.readInt(); + target.writeInt(length); + target.write(source, length); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + MapDataSerializer that = (MapDataSerializer) o; + + return keyType.equals(that.keyType) && valueType.equals(that.valueType); + } + + @Override + public int hashCode() { + int result = keyType.hashCode(); + result = 31 * result + valueType.hashCode(); + return result; + } + + @VisibleForTesting + public TypeSerializer getKeySerializer() { + return keySerializer; + } + + @VisibleForTesting + public TypeSerializer getValueSerializer() { + return valueSerializer; + } + + @Override + public TypeSerializerSnapshot<MapData> snapshotConfiguration() { + return new MapDataSerializer.MapDataSerializerSnapshot( + keyType, valueType, keySerializer, valueSerializer); + } + + /** {@link TypeSerializerSnapshot} for {@link MapDataSerializer}. */ + public static final class MapDataSerializerSnapshot implements TypeSerializerSnapshot<MapData> { + private static final int CURRENT_VERSION = 3; Review Comment: Is 0 or 1 will be better? ########## flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/data/writer/BinaryArrayWriterTest.java: ########## @@ -0,0 +1,565 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.runtime.serializer.data.writer; + +import org.apache.flink.cdc.common.data.DecimalData; +import org.apache.flink.cdc.common.data.RecordData; +import org.apache.flink.cdc.common.data.TimestampData; +import org.apache.flink.cdc.common.data.binary.BinaryArrayData; +import org.apache.flink.cdc.common.data.binary.BinaryMapData; +import org.apache.flink.cdc.common.data.binary.BinaryRecordData; +import org.apache.flink.cdc.common.data.binary.BinarySegmentUtils; +import org.apache.flink.cdc.common.data.binary.BinaryStringData; +import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.runtime.serializer.data.ArrayDataSerializer; +import org.apache.flink.cdc.runtime.serializer.data.MapDataSerializer; +import org.apache.flink.cdc.runtime.serializer.data.RecordDataSerializer; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; + +import org.junit.Test; + +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.time.LocalDateTime; + +import static org.apache.flink.cdc.common.data.binary.BinaryStringData.fromString; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test of {@link BinaryArrayData} and {@link BinaryArrayWriter}. */ +public class BinaryArrayWriterTest { + @Test + public void testArray() { + // 1.array test + BinaryArrayData array = new BinaryArrayData(); + BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 4); + + writer.writeInt(0, 6); + writer.setNullInt(1); + writer.writeInt(2, 666); + writer.complete(); + + assertThat(6).isEqualTo(array.getInt(0)); + assertThat(array.isNullAt(1)).isTrue(); + assertThat(666).isEqualTo(array.getInt(2)); + + // 2.test write to binary row. + { + BinaryRecordData row2 = new BinaryRecordData(1); + BinaryRecordDataWriter writer2 = new BinaryRecordDataWriter(row2); + writer2.writeArray(0, array, new ArrayDataSerializer(DataTypes.INT())); + writer2.complete(); + + BinaryArrayData array2 = (BinaryArrayData) row2.getArray(0); + assertThat(array).isEqualTo(array2); + assertThat(6).isEqualTo(array2.getInt(0)); + assertThat(array2.isNullAt(1)).isTrue(); + assertThat(666).isEqualTo(array2.getInt(2)); + } + + // 3.test write var seg array to binary row. + { + BinaryArrayData array3 = splitArray(array); + + BinaryRecordData row2 = new BinaryRecordData(1); + BinaryRecordDataWriter writer2 = new BinaryRecordDataWriter(row2); + writer2.writeArray(0, array3, new ArrayDataSerializer(DataTypes.INT())); + writer2.complete(); + + BinaryArrayData array2 = (BinaryArrayData) row2.getArray(0); + assertThat(array).isEqualTo(array2); + assertThat(6).isEqualTo(array2.getInt(0)); + assertThat(array2.isNullAt(1)).isTrue(); + assertThat(666).isEqualTo(array2.getInt(2)); + } + } + + @Test + public void testArrayTypes() { + { + // test bool + BinaryArrayData array = new BinaryArrayData(); + BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 1); + writer.setNullBoolean(0); + writer.writeBoolean(1, true); + writer.complete(); + + assertThat(array.isNullAt(0)).isTrue(); + assertThat(array.getBoolean(1)).isTrue(); + array.setBoolean(0, true); + assertThat(array.getBoolean(0)).isTrue(); + array.setNullBoolean(0); + assertThat(array.isNullAt(0)).isTrue(); + + BinaryArrayData newArray = splitArray(array); + assertThat(newArray.isNullAt(0)).isTrue(); + assertThat(newArray.getBoolean(1)).isTrue(); + newArray.setBoolean(0, true); + assertThat(newArray.getBoolean(0)).isTrue(); + newArray.setNullBoolean(0); + assertThat(newArray.isNullAt(0)).isTrue(); + + newArray.setBoolean(0, true); + assertThat(BinaryArrayData.fromPrimitiveArray(newArray.toBooleanArray())) + .isEqualTo(newArray); + } + + { + // test byte + BinaryArrayData array = new BinaryArrayData(); + BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 1); + writer.setNullByte(0); + writer.writeByte(1, (byte) 25); + writer.complete(); + + assertThat(array.isNullAt(0)).isTrue(); + assertThat(array.getByte(1)).isEqualTo((byte) 25); + array.setByte(0, (byte) 5); + assertThat(array.getByte(0)).isEqualTo((byte) 5); + array.setNullByte(0); + assertThat(array.isNullAt(0)).isTrue(); + + BinaryArrayData newArray = splitArray(array); + assertThat(newArray.isNullAt(0)).isTrue(); + assertThat(newArray.getByte(1)).isEqualTo((byte) 25); + newArray.setByte(0, (byte) 5); + assertThat(newArray.getByte(0)).isEqualTo((byte) 5); + newArray.setNullByte(0); + assertThat(newArray.isNullAt(0)).isTrue(); + + newArray.setByte(0, (byte) 3); + assertThat(BinaryArrayData.fromPrimitiveArray(newArray.toByteArray())) + .isEqualTo(newArray); + } + + { + // test short + BinaryArrayData array = new BinaryArrayData(); + BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 2); + writer.setNullShort(0); + writer.writeShort(1, (short) 25); + writer.complete(); + + assertThat(array.isNullAt(0)).isTrue(); + assertThat(array.getShort(1)).isEqualTo((short) 25); + array.setShort(0, (short) 5); + assertThat(array.getShort(0)).isEqualTo((short) 5); + array.setNullShort(0); + assertThat(array.isNullAt(0)).isTrue(); + + BinaryArrayData newArray = splitArray(array); + assertThat(newArray.isNullAt(0)).isTrue(); + assertThat(newArray.getShort(1)).isEqualTo((short) 25); + newArray.setShort(0, (short) 5); + assertThat(newArray.getShort(0)).isEqualTo((short) 5); + newArray.setNullShort(0); + assertThat(newArray.isNullAt(0)).isTrue(); + + newArray.setShort(0, (short) 3); + assertThat(BinaryArrayData.fromPrimitiveArray(newArray.toShortArray())) + .isEqualTo(newArray); + } + + { + // test int + BinaryArrayData array = new BinaryArrayData(); + BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 4); + writer.setNullInt(0); + writer.writeInt(1, 25); + writer.complete(); + + assertThat(array.isNullAt(0)).isTrue(); + assertThat(array.getInt(1)).isEqualTo(25); + array.setInt(0, 5); + assertThat(array.getInt(0)).isEqualTo(5); + array.setNullInt(0); + assertThat(array.isNullAt(0)).isTrue(); + + BinaryArrayData newArray = splitArray(array); + assertThat(newArray.isNullAt(0)).isTrue(); + assertThat(newArray.getInt(1)).isEqualTo(25); + newArray.setInt(0, 5); + assertThat(newArray.getInt(0)).isEqualTo(5); + newArray.setNullInt(0); + assertThat(newArray.isNullAt(0)).isTrue(); + + newArray.setInt(0, 3); + assertThat(BinaryArrayData.fromPrimitiveArray(newArray.toIntArray())) + .isEqualTo(newArray); + } + + { + // test long + BinaryArrayData array = new BinaryArrayData(); + BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); + writer.setNullLong(0); + writer.writeLong(1, 25); + writer.complete(); + + assertThat(array.isNullAt(0)).isTrue(); + assertThat(array.getLong(1)).isEqualTo(25); + array.setLong(0, 5); + assertThat(array.getLong(0)).isEqualTo(5); + array.setNullLong(0); + assertThat(array.isNullAt(0)).isTrue(); + + BinaryArrayData newArray = splitArray(array); + assertThat(newArray.isNullAt(0)).isTrue(); + assertThat(newArray.getLong(1)).isEqualTo(25); + newArray.setLong(0, 5); + assertThat(newArray.getLong(0)).isEqualTo(5); + newArray.setNullLong(0); + assertThat(newArray.isNullAt(0)).isTrue(); + + newArray.setLong(0, 3); + assertThat(BinaryArrayData.fromPrimitiveArray(newArray.toLongArray())) + .isEqualTo(newArray); + } + + { + // test float + BinaryArrayData array = new BinaryArrayData(); + BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 4); + writer.setNullFloat(0); + writer.writeFloat(1, 25); + writer.complete(); + + assertThat(array.isNullAt(0)).isTrue(); + assertThat(array.getFloat(1)).isEqualTo(25f); + array.setFloat(0, 5); + assertThat(array.getFloat(0)).isEqualTo(5f); + array.setNullFloat(0); + assertThat(array.isNullAt(0)).isTrue(); + + BinaryArrayData newArray = splitArray(array); + assertThat(newArray.isNullAt(0)).isTrue(); + assertThat(newArray.getFloat(1)).isEqualTo(25f); + newArray.setFloat(0, 5); + assertThat(newArray.getFloat(0)).isEqualTo(5f); + newArray.setNullFloat(0); + assertThat(newArray.isNullAt(0)).isTrue(); + + newArray.setFloat(0, 3); + assertThat(BinaryArrayData.fromPrimitiveArray(newArray.toFloatArray())) + .isEqualTo(newArray); + } + + { + // test double + BinaryArrayData array = new BinaryArrayData(); + BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); + writer.setNullDouble(0); + writer.writeDouble(1, 25); + writer.complete(); + + assertThat(array.isNullAt(0)).isTrue(); + assertThat(array.getDouble(1)).isEqualTo(25d); + array.setDouble(0, 5); + assertThat(array.getDouble(0)).isEqualTo(5d); + array.setNullDouble(0); + assertThat(array.isNullAt(0)).isTrue(); + + BinaryArrayData newArray = splitArray(array); + assertThat(newArray.isNullAt(0)).isTrue(); + assertThat(newArray.getDouble(1)).isEqualTo(25d); + newArray.setDouble(0, 5); + assertThat(newArray.getDouble(0)).isEqualTo(5d); + newArray.setNullDouble(0); + assertThat(newArray.isNullAt(0)).isTrue(); + + newArray.setDouble(0, 3); + assertThat(BinaryArrayData.fromPrimitiveArray(newArray.toDoubleArray())) + .isEqualTo(newArray); + } + + { + // test string + BinaryArrayData array = new BinaryArrayData(); + BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); + writer.setNullAt(0); + writer.writeString(1, fromString("jaja")); + writer.complete(); + + assertThat(array.isNullAt(0)).isTrue(); + assertThat(array.getString(1)).isEqualTo(fromString("jaja")); + + BinaryArrayData newArray = splitArray(array); + assertThat(newArray.isNullAt(0)).isTrue(); + assertThat(newArray.getString(1)).isEqualTo(fromString("jaja")); + } + + BinaryArrayData subArray = new BinaryArrayData(); + BinaryArrayWriter subWriter = new BinaryArrayWriter(subArray, 2, 8); + subWriter.setNullAt(0); + subWriter.writeString(1, fromString("hehehe")); + subWriter.complete(); + + { + // test array + BinaryArrayData array = new BinaryArrayData(); + BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); + writer.setNullAt(0); + writer.writeArray(1, subArray, new ArrayDataSerializer(DataTypes.INT())); + writer.complete(); + + assertThat(array.isNullAt(0)).isTrue(); + assertThat(array.getArray(1)).isEqualTo(subArray); + + BinaryArrayData newArray = splitArray(array); + assertThat(newArray.isNullAt(0)).isTrue(); + assertThat(newArray.getArray(1)).isEqualTo(subArray); + } + + { + // test map + BinaryArrayData array = new BinaryArrayData(); + BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); + writer.setNullAt(0); + writer.writeMap( + 1, + BinaryMapData.valueOf(subArray, subArray), + new MapDataSerializer(DataTypes.INT(), DataTypes.INT())); + writer.complete(); + + assertThat(array.isNullAt(0)).isTrue(); + assertThat(array.getMap(1)).isEqualTo(BinaryMapData.valueOf(subArray, subArray)); + + BinaryArrayData newArray = splitArray(array); + assertThat(newArray.isNullAt(0)).isTrue(); + assertThat(newArray.getMap(1)).isEqualTo(BinaryMapData.valueOf(subArray, subArray)); + } + } + + @Test + public void testMap() { + BinaryArrayData array1 = new BinaryArrayData(); + BinaryArrayWriter writer1 = new BinaryArrayWriter(array1, 3, 4); + writer1.writeInt(0, 6); + writer1.writeInt(1, 5); + writer1.writeInt(2, 666); + writer1.complete(); + + BinaryArrayData array2 = new BinaryArrayData(); + BinaryArrayWriter writer2 = new BinaryArrayWriter(array2, 3, 8); + writer2.writeString(0, BinaryStringData.fromString("6")); + writer2.writeString(1, BinaryStringData.fromString("5")); + writer2.writeString(2, BinaryStringData.fromString("666")); + writer2.complete(); + + BinaryMapData binaryMap = BinaryMapData.valueOf(array1, array2); + + BinaryRecordData row = new BinaryRecordData(1); + BinaryRecordDataWriter rowWriter = new BinaryRecordDataWriter(row); + rowWriter.writeMap(0, binaryMap, new MapDataSerializer(DataTypes.INT(), DataTypes.INT())); + rowWriter.complete(); + + BinaryMapData map = (BinaryMapData) row.getMap(0); + BinaryArrayData key = map.keyArray(); + BinaryArrayData value = map.valueArray(); + + assertThat(map).isEqualTo(binaryMap); + assertThat(key).isEqualTo(array1); + assertThat(value).isEqualTo(array2); + + assertThat(5).isEqualTo(key.getInt(1)); + assertThat(BinaryStringData.fromString("5")).isEqualTo(value.getString(1)); + } + + private static BinaryArrayData splitArray(BinaryArrayData array) { + BinaryArrayData ret = new BinaryArrayData(); + MemorySegment[] segments = + splitBytes( + BinarySegmentUtils.copyToBytes( + array.getSegments(), 0, array.getSizeInBytes()), + 0); + ret.pointTo(segments, 0, array.getSizeInBytes()); + return ret; + } + + private static MemorySegment[] splitBytes(byte[] bytes, int baseOffset) { + int newSize = (bytes.length + 1) / 2 + baseOffset; + MemorySegment[] ret = new MemorySegment[2]; + ret[0] = MemorySegmentFactory.wrap(new byte[newSize]); + ret[1] = MemorySegmentFactory.wrap(new byte[newSize]); + + ret[0].put(baseOffset, bytes, 0, newSize - baseOffset); + ret[1].put(0, bytes, newSize - baseOffset, bytes.length - (newSize - baseOffset)); + return ret; + } + + @Test + public void testToArray() { + BinaryArrayData array = new BinaryArrayData(); + BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 2); + writer.writeShort(0, (short) 5); + writer.writeShort(1, (short) 10); + writer.writeShort(2, (short) 15); + writer.complete(); + + short[] shorts = array.toShortArray(); + assertThat(shorts[0]).isEqualTo((short) 5); + assertThat(shorts[1]).isEqualTo((short) 10); + assertThat(shorts[2]).isEqualTo((short) 15); + + MemorySegment[] segments = splitBytes(writer.getSegments().getArray(), 3); + array.pointTo(segments, 3, array.getSizeInBytes()); + assertThat(array.getShort(0)).isEqualTo((short) 5); + assertThat(array.getShort(1)).isEqualTo((short) 10); + assertThat(array.getShort(2)).isEqualTo((short) 15); + short[] shorts2 = array.toShortArray(); + assertThat(shorts2[0]).isEqualTo((short) 5); + assertThat(shorts2[1]).isEqualTo((short) 10); + assertThat(shorts2[2]).isEqualTo((short) 15); + } + + @Test + public void testDecimal() { + + BinaryArrayData array = new BinaryArrayData(); + BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); + + // 1.compact + { + int precision = 4; + int scale = 2; + writer.reset(); + writer.writeDecimal(0, DecimalData.fromUnscaledLong(5, precision, scale), precision); + writer.setNullAt(1); + writer.complete(); + + assertThat(array.getDecimal(0, precision, scale).toString()).isEqualTo("0.05"); + assertThat(array.isNullAt(1)).isTrue(); + array.setDecimal(0, DecimalData.fromUnscaledLong(6, precision, scale), precision); + assertThat(array.getDecimal(0, precision, scale).toString()).isEqualTo("0.06"); + } + + // 2.not compact + { + int precision = 25; + int scale = 5; + DecimalData decimal1 = + DecimalData.fromBigDecimal(BigDecimal.valueOf(5.55), precision, scale); + DecimalData decimal2 = + DecimalData.fromBigDecimal(BigDecimal.valueOf(6.55), precision, scale); + + writer.reset(); + writer.writeDecimal(0, decimal1, precision); + writer.writeDecimal(1, null, precision); + writer.complete(); + + assertThat(array.getDecimal(0, precision, scale).toString()).isEqualTo("5.55000"); + assertThat(array.isNullAt(1)).isTrue(); + array.setDecimal(0, decimal2, precision); + assertThat(array.getDecimal(0, precision, scale).toString()).isEqualTo("6.55000"); + } + } + + // @Test + // public void testGeneric() { + // BinaryArrayData array = new BinaryArrayData(); + // BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); + // RawValueData<String> generic = RawValueData.fromObject("hahah"); + // RawValueDataSerializer<String> serializer = + // new RawValueDataSerializer<>(StringSerializer.INSTANCE); + // writer.writeRawValue(0, generic, serializer); + // writer.setNullAt(1); + // writer.complete(); + // + // RawValueData<String> newGeneric = array.getRawValue(0); + // assertThat(newGeneric).satisfies(matching(equivalent(generic, serializer))); + // assertThat(array.isNullAt(1)).isTrue(); + // } Review Comment: What's the reason to comment this Test? ########## flink-cdc-common/src/test/java/org/apache/flink/cdc/common/data/binary/BinarySegmentUtilsTest.java: ########## @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.common.data.binary; + +import org.apache.flink.cdc.common.data.util.BinaryRecordDataDataUtil; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; + +import org.junit.Test; + +import static org.apache.flink.cdc.common.data.util.BinaryRecordDataDataUtil.BYTE_ARRAY_BASE_OFFSET; +import static org.assertj.core.api.Assertions.assertThat; + +/** Utilities for binary data segments which heavily uses {@link MemorySegment}. */ +public class BinarySegmentUtilsTest { + @Test + public void testCopy() { + // test copy the content of the latter Seg + MemorySegment[] segments = new MemorySegment[2]; + segments[0] = MemorySegmentFactory.wrap(new byte[] {0, 2, 5}); + segments[1] = MemorySegmentFactory.wrap(new byte[] {6, 12, 15}); + + byte[] bytes = BinarySegmentUtils.copyToBytes(segments, 4, 2); + assertThat(bytes).isEqualTo(new byte[] {12, 15}); + } + + @Test + public void testEquals() { + // test copy the content of the latter Seg + MemorySegment[] segments1 = new MemorySegment[3]; + segments1[0] = MemorySegmentFactory.wrap(new byte[] {0, 2, 5}); + segments1[1] = MemorySegmentFactory.wrap(new byte[] {6, 12, 15}); + segments1[2] = MemorySegmentFactory.wrap(new byte[] {1, 1, 1}); + + MemorySegment[] segments2 = new MemorySegment[2]; + segments2[0] = MemorySegmentFactory.wrap(new byte[] {6, 0, 2, 5}); + segments2[1] = MemorySegmentFactory.wrap(new byte[] {6, 12, 15, 18}); + + assertThat(BinarySegmentUtils.equalsMultiSegments(segments1, 0, segments2, 0, 0)).isTrue(); + assertThat(BinarySegmentUtils.equals(segments1, 0, segments2, 1, 3)).isTrue(); + assertThat(BinarySegmentUtils.equals(segments1, 0, segments2, 1, 6)).isTrue(); + assertThat(BinarySegmentUtils.equals(segments1, 0, segments2, 1, 7)).isFalse(); + } + + @Test + public void testBoundaryByteArrayEquals() { + byte[] bytes1 = new byte[5]; + bytes1[3] = 81; + byte[] bytes2 = new byte[100]; + bytes2[3] = 81; + bytes2[4] = 81; + + assertThat(BinaryRecordDataDataUtil.byteArrayEquals(bytes1, bytes2, 4)).isTrue(); + assertThat(BinaryRecordDataDataUtil.byteArrayEquals(bytes1, bytes2, 5)).isFalse(); + assertThat(BinaryRecordDataDataUtil.byteArrayEquals(bytes1, bytes2, 0)).isTrue(); + } + + // @Test + // public void testBoundaryEquals() { Review Comment: Avoid remaining dead code. ########## flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/schema/DataFieldSerializer.java: ########## @@ -39,7 +39,14 @@ public class DataFieldSerializer extends TypeSerializerSingleton<DataField> { public static final DataFieldSerializer INSTANCE = new DataFieldSerializer(); private final StringSerializer stringSerializer = StringSerializer.INSTANCE; - private final DataTypeSerializer dataTypeSerializer = new DataTypeSerializer(); + private DataTypeSerializer dataTypeSerializer; + + private DataTypeSerializer getDataTypeSerializer() { Review Comment: This change doesn't seem so necessary. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
