You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/08/24 08:04:53 UTC

[GitHub] [pulsar] eolivelli commented on a diff in pull request #16740: [PIP193] [feature][connectors] Add support for a preprocess Function in Sinks

eolivelli commented on code in PR #16740:
URL: https://github.com/apache/pulsar/pull/16740#discussion_r953477852


##########
pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java:
##########
@@ -383,11 +411,64 @@ void handleResult(Record srcRecord, JavaExecutionResult result) throws Exception
 
     private void sendOutputMessage(Record srcRecord, Object output) throws Exception {
         if (componentType == org.apache.pulsar.functions.proto.Function.FunctionDetails.ComponentType.SINK) {
-            Thread.currentThread().setContextClassLoader(functionClassLoader);
+            Thread.currentThread().setContextClassLoader(componentClassLoader);
         }
         AbstractSinkRecord<?> sinkRecord;
         if (output instanceof Record) {
-            sinkRecord = new OutputRecordSinkRecord<>(srcRecord, (Record) output);
+            Record record = (Record) output;
+            if (sinkSchemaInfoProvider != null) {
+                // Function and Sink coupled together so we need to encode with the Function Schema
+                // and decode with the Sink schema
+                Schema encodingSchema = record.getSchema();
+                boolean isKeyValueSeparated = false;
+                if (encodingSchema instanceof KeyValueSchema) {
+                    KeyValueSchema<?, ?> kvSchema = (KeyValueSchema<?, ?>) encodingSchema;
+                    // If the encoding is SEPARATED, it's easier to encode/decode with INLINE
+                    // and rebuild the SEPARATED KeyValueSchema after decoding
+                    if (kvSchema.getKeyValueEncodingType() == KeyValueEncodingType.SEPARATED) {
+                        encodingSchema = KeyValueSchemaImpl.of(kvSchema.getKeySchema(), kvSchema.getValueSchema());
+                        isKeyValueSeparated = true;
+                    }
+                }
+                byte[] encoded = encodingSchema.encode(record.getValue());
+
+                if (sinkSchema.get() == null) {
+                    Schema<?> schema = getSinkSchema(record, sinkTypeArg);
+                    schema.setSchemaInfoProvider(sinkSchemaInfoProvider);
+                    sinkSchema.compareAndSet(null, schema);
+                }
+                Schema<?> schema = sinkSchema.get();
+                SchemaVersion schemaVersion = sinkSchemaInfoProvider.getSchemaVersion(encodingSchema);
+                final byte[] schemaVersionBytes = schemaVersion.bytes();
+                Object decoded = schema.decode(encoded, schemaVersionBytes);
+
+                if (schema instanceof AutoConsumeSchema) {
+                    schema = ((AutoConsumeSchema) schema).getInternalSchema(schemaVersionBytes);
+                }
+
+                final Schema<?> finalSchema;
+                if (isKeyValueSeparated && schema instanceof KeyValueSchema) {
+                    KeyValueSchema<?, ?> kvSchema = (KeyValueSchema<?, ?>) schema;
+                    finalSchema = KeyValueSchemaImpl.of(kvSchema.getKeySchema(), kvSchema.getValueSchema(),
+                        KeyValueEncodingType.SEPARATED);
+                } else {
+                    finalSchema = schema;
+                }
+
+                sinkRecord = new OutputRecordSinkRecord<>(srcRecord, record) {

Review Comment:
   what about creating a named class ? it will help Sink developers when they dump the "Record" object (and especially us when we will debug problems about this feature :-)  )



##########
pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java:
##########
@@ -383,11 +411,64 @@ void handleResult(Record srcRecord, JavaExecutionResult result) throws Exception
 
     private void sendOutputMessage(Record srcRecord, Object output) throws Exception {
         if (componentType == org.apache.pulsar.functions.proto.Function.FunctionDetails.ComponentType.SINK) {
-            Thread.currentThread().setContextClassLoader(functionClassLoader);
+            Thread.currentThread().setContextClassLoader(componentClassLoader);
         }
         AbstractSinkRecord<?> sinkRecord;
         if (output instanceof Record) {
-            sinkRecord = new OutputRecordSinkRecord<>(srcRecord, (Record) output);
+            Record record = (Record) output;

Review Comment:
   what about moving this block to a method ?



##########
pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/SinkSchemaInfoProvider.java:
##########
@@ -0,0 +1,68 @@
+/**
+ * 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.pulsar.functions.instance;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.schema.SchemaInfoProvider;
+import org.apache.pulsar.common.protocol.schema.BytesSchemaVersion;
+import org.apache.pulsar.common.protocol.schema.SchemaHash;
+import org.apache.pulsar.common.protocol.schema.SchemaVersion;
+import org.apache.pulsar.common.schema.SchemaInfo;
+
+class SinkSchemaInfoProvider implements SchemaInfoProvider {
+
+  AtomicLong latestVersion = new AtomicLong(0);
+  ConcurrentHashMap<SchemaVersion, SchemaInfo> schemaInfos = new ConcurrentHashMap<>();

Review Comment:
   it is not very clear to me (by reading the code) who is populating these maps.
   maybe we could add explicit accessors ?



##########
pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java:
##########
@@ -881,4 +963,98 @@ private void setupOutput(ContextImpl contextImpl) throws Exception {
             Thread.currentThread().setContextClassLoader(this.instanceClassLoader);
         }
     }
+
+    private static <T> Schema<T> getSinkSchema(Record<?> record, Class<T> clazz) {

Review Comment:
   we could move these 3 methods to some other class that does similar things, like TopicSchema.
   but I have no strong opinion



##########
pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/SinkSchemaInfoProvider.java:
##########
@@ -0,0 +1,68 @@
+/**
+ * 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.pulsar.functions.instance;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.schema.SchemaInfoProvider;
+import org.apache.pulsar.common.protocol.schema.BytesSchemaVersion;
+import org.apache.pulsar.common.protocol.schema.SchemaHash;
+import org.apache.pulsar.common.protocol.schema.SchemaVersion;
+import org.apache.pulsar.common.schema.SchemaInfo;
+
+class SinkSchemaInfoProvider implements SchemaInfoProvider {

Review Comment:
   what about adding a javadoc here ?



-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org