syhily commented on a change in pull request #17452:
URL: https://github.com/apache/flink/pull/17452#discussion_r799731961



##########
File path: 
flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/serializer/PulsarSchemaWrapper.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.connector.pulsar.sink.writer.serializer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.pulsar.common.schema.PulsarSchema;
+import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext;
+import org.apache.flink.connector.pulsar.sink.writer.message.RawMessage;
+
+import org.apache.pulsar.client.api.Schema;
+
+/** Wrap the Pulsar's Schema into PulsarSerializationSchema. */
+@Internal
+public class PulsarSchemaWrapper<IN> implements PulsarSerializationSchema<IN> {
+    private static final long serialVersionUID = -2567052498398184194L;
+
+    private static final byte[] EMPTY_BYTES = new byte[0];
+    private final PulsarSchema<IN> pulsarSchema;
+
+    public PulsarSchemaWrapper(PulsarSchema<IN> pulsarSchema) {
+        this.pulsarSchema = pulsarSchema;
+    }
+
+    @Override
+    public RawMessage<byte[]> serialize(IN element, PulsarSinkContext 
sinkContext) {
+        RawMessage<byte[]> message;
+
+        if (sinkContext.isEnableSchemaEvolution()) {
+            // We don't need to serialize incoming records in schema evolution.
+            message = new RawMessage<>(EMPTY_BYTES);

Review comment:
       This is a confusing design. We provide two types of serialization.
   
   1. The first one is based on Flink serialization logic. We would serialize 
the message into bytes by using `Schema` (Pulsar class) or 
`SerializationSchema` (Flink class). The bytes would be sent to Pulsar in a 
`Schema.BYTES` which Pulsar wouldn't validate the correctness.
   2. The second one is only based on Pulsar Schema. We would send the original 
message with a valid Pulsar `Schema`. The message would be serialized in Pulsar 
Client and the Schema would also be sent to Pulsar for validation.
   
   The second mechanism is called schema evolution in Pulsar. It could be 
enabled by `PulsarSinkOptions.PULSAR_WRITE_SCHEMA_EVOLUTION`. Since the message 
would be serialized in Client. It's no need to serialize in 
`PulsarSchemaWrapper`.
   
   But I admit that the design should be improved.




-- 
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


Reply via email to