yuxiqian commented on code in PR #3652: URL: https://github.com/apache/flink-cdc/pull/3652#discussion_r1832107558
########## flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonWriterStateSerializer.java: ########## @@ -0,0 +1,62 @@ +/* + * 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.connectors.paimon.sink.v2; + +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.table.api.TableException; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +/** A serializer for the {@link PaimonWriterState}. */ +public class PaimonWriterStateSerializer implements SimpleVersionedSerializer<PaimonWriterState> { + + private static final int currentVersion = 0; + + @Override + public int getVersion() { + return currentVersion; + } + + @Override + public byte[] serialize(PaimonWriterState paimonWriterState) throws IOException { + try (ByteArrayOutputStream bao = new ByteArrayOutputStream(256)) { + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(bao); + view.writeLong(paimonWriterState.getCheckpointId()); + return bao.toByteArray(); + } + } + + @Override + public PaimonWriterState deserialize(int version, byte[] serialized) throws IOException { + if (version == 0) { + try (ByteArrayInputStream bis = new ByteArrayInputStream(serialized)) { + DataInputViewStreamWrapper view = new DataInputViewStreamWrapper(bis); + long check = view.readLong(); + return new PaimonWriterState(check); + } + } + throw new TableException( + String.format( + "Can't serialized data with version %d because the max support version is %d.", Review Comment: ```suggestion "Can't deserialize data with version %d because the max support version is %d.", ``` ########## flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonWriter.java: ########## @@ -85,6 +92,28 @@ public PaimonWriter( new ExecutorThreadFactory( Thread.currentThread().getName() + "-CdcMultiWrite-Compaction")); this.serializer = serializer; + this.checkpointId = CheckpointIDCounter.INITIAL_CHECKPOINT_ID - 1; Review Comment: Is it possible to use `sinkInitContext.getRestoredCheckpointId()` to fetch last succesfully executed checkpointId, instead of saving it in the `PaimonWriterState`? ########## flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonWriter.java: ########## @@ -186,4 +216,10 @@ public void close() throws Exception { compactExecutor.shutdownNow(); } } + + @Override + public List<PaimonWriterState> snapshotState(long checkpointId) throws IOException { + this.checkpointId = checkpointId; Review Comment: Seems `this.checkpointId` and `long checkpointId` have different meanings, and identical name might be confusing. Maybe we can rename field to `lastCheckpointId`, just like what `KafkaWriter` does? ########## flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonWriterStateSerializer.java: ########## @@ -0,0 +1,62 @@ +/* + * 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.connectors.paimon.sink.v2; + +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.table.api.TableException; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +/** A serializer for the {@link PaimonWriterState}. */ +public class PaimonWriterStateSerializer implements SimpleVersionedSerializer<PaimonWriterState> { + + private static final int currentVersion = 0; + + @Override + public int getVersion() { + return currentVersion; + } + + @Override + public byte[] serialize(PaimonWriterState paimonWriterState) throws IOException { + try (ByteArrayOutputStream bao = new ByteArrayOutputStream(256)) { Review Comment: Is it necessary to specify 256-byte size here? Seems a Long value consumes 8 bytes at most. -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org