sijie commented on a change in pull request #9927:
URL: https://github.com/apache/pulsar/pull/9927#discussion_r621335621



##########
File path: pulsar-io/kafka-connect-adaptor-nar/pom.xml
##########
@@ -30,13 +30,40 @@
   <artifactId>pulsar-io-kafka-connect-adaptor-nar</artifactId>
   <name>Pulsar IO :: Kafka Connect Adaptor NAR</name>
 
-  <dependencies>
-    <dependency>
-      <groupId>${project.groupId}</groupId>
-      <artifactId>pulsar-io-kafka-connect-adaptor</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-  </dependencies>
+  <profiles>
+    <profile>
+      <id>main</id>
+      <activation>
+        <activeByDefault>true</activeByDefault>
+      </activation>
+      <dependencies>
+        <dependency>
+          <groupId>${project.groupId}</groupId>
+          <artifactId>pulsar-io-kafka-connect-adaptor</artifactId>
+          <version>${project.version}</version>
+        </dependency>
+      </dependencies>
+    </profile>
+    <profile>

Review comment:
       I am not sure it is a good idea to add an example to package a file 
connect here. Isn't it better to add a separate example module for 
demonstration purposes?

##########
File path: 
pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/KafkaConnectSink.java
##########
@@ -0,0 +1,325 @@
+/**
+ * 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.io.kafka.connect;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.TimestampType;
+import org.apache.kafka.connect.connector.Task;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.sink.SinkConnector;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.pulsar.client.api.schema.GenericObject;
+import org.apache.pulsar.client.impl.schema.KeyValueSchema;
+import org.apache.pulsar.common.schema.SchemaType;
+import org.apache.pulsar.functions.api.Record;
+import org.apache.pulsar.io.core.KeyValue;
+import org.apache.pulsar.io.core.Sink;
+import org.apache.pulsar.io.core.SinkContext;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig.OFFSET_STORAGE_TOPIC_CONFIG;
+import static 
org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig.PULSAR_SERVICE_URL_CONFIG;
+
+@Slf4j
+public class KafkaConnectSink implements Sink<GenericObject> {
+
+    private boolean unwrapKeyValueIfAvailable;
+
+    private final static ImmutableMap<Class<?>, Schema> primitiveTypeToSchema;
+    private final static ImmutableMap<SchemaType, Schema> 
pulsarSchemaTypeTypeToKafkaSchema;
+
+    static {
+        primitiveTypeToSchema = ImmutableMap.<Class<?>, Schema>builder()
+                .put(Boolean.class, Schema.BOOLEAN_SCHEMA)
+                .put(Byte.class, Schema.INT8_SCHEMA)
+                .put(Short.class, Schema.INT16_SCHEMA)
+                .put(Integer.class, Schema.INT32_SCHEMA)
+                .put(Long.class, Schema.INT64_SCHEMA)
+                .put(Float.class, Schema.FLOAT32_SCHEMA)
+                .put(Double.class, Schema.FLOAT64_SCHEMA)
+                .put(String.class, Schema.STRING_SCHEMA)
+                .put(byte[].class, Schema.BYTES_SCHEMA)
+                .build();
+        pulsarSchemaTypeTypeToKafkaSchema = ImmutableMap.<SchemaType, 
Schema>builder()
+                .put(SchemaType.BOOLEAN, Schema.BOOLEAN_SCHEMA)
+                .put(SchemaType.INT8, Schema.INT8_SCHEMA)
+                .put(SchemaType.INT16, Schema.INT16_SCHEMA)
+                .put(SchemaType.INT32, Schema.INT32_SCHEMA)
+                .put(SchemaType.INT64, Schema.INT64_SCHEMA)
+                .put(SchemaType.FLOAT, Schema.FLOAT32_SCHEMA)
+                .put(SchemaType.DOUBLE, Schema.FLOAT64_SCHEMA)
+                .put(SchemaType.STRING, Schema.STRING_SCHEMA)
+                .put(SchemaType.BYTES, Schema.BYTES_SCHEMA)
+                .build();
+    }
+
+    private PulsarKafkaSinkContext sinkContext;
+    private PulsarKafkaSinkTaskContext taskContext;
+    private SinkConnector connector;
+    private SinkTask task;
+
+    private int batchSize;
+    private long lingerMs;
+    private final ScheduledExecutorService scheduledExecutor =
+            Executors.newSingleThreadScheduledExecutor(new 
ThreadFactoryBuilder()
+                    .setNameFormat("pulsar-io-kafka-adaptor-sink-flush-%d")
+                    .build());
+    private final ConcurrentLinkedDeque<Record<GenericObject>> 
pendingFlushQueue = new ConcurrentLinkedDeque<>();
+    private volatile boolean isRunning = false;
+
+    private final Properties props = new Properties();
+    private PulsarKafkaConnectSinkConfig kafkaSinkConfig;
+
+    protected String topicName;
+
+    @Override
+    public void write(Record<GenericObject> sourceRecord) {
+        if (log.isDebugEnabled()) {
+            log.debug("Record sending to kafka, record={}.", sourceRecord);
+        }
+
+        if (!isRunning) {
+            log.error("Sink is stopped. Cannot send the record {}", 
sourceRecord);
+            sourceRecord.fail();
+            return;
+        }
+
+        try {
+            SinkRecord record = toSinkRecord(sourceRecord);
+            task.put(Lists.newArrayList(record));
+        } catch (Exception ex) {
+            log.error("Error sending the record {}", sourceRecord, ex);
+            sourceRecord.fail();
+            return;
+        }
+        pendingFlushQueue.add(sourceRecord);
+        flushIfNeeded(false);
+    }
+
+    @Override
+    public void close() throws Exception {
+        isRunning = false;
+        flushIfNeeded(true);
+        scheduledExecutor.shutdown();
+        if (!scheduledExecutor.awaitTermination(10 * lingerMs, 
TimeUnit.MILLISECONDS)) {
+            log.error("scheduledExecutor did not terminate in {} ms", 10 * 
lingerMs);
+        }
+
+        task.stop();
+        connector.stop();
+        taskContext.close();
+
+        log.info("Kafka sink stopped.");
+    }
+
+    @Override
+    public void open(Map<String, Object> config, SinkContext ctx) throws 
Exception {
+        kafkaSinkConfig = PulsarKafkaConnectSinkConfig.load(config);
+        Objects.requireNonNull(kafkaSinkConfig.getTopic(), "Kafka topic is not 
set");
+        topicName = kafkaSinkConfig.getTopic();
+        unwrapKeyValueIfAvailable = 
kafkaSinkConfig.isUnwrapKeyValueIfAvailable();
+
+        String kafkaConnectorFQClassName = 
kafkaSinkConfig.getKafkaConnectorSinkClass();
+        
kafkaSinkConfig.getKafkaConnectorConfigProperties().forEach(props::put);
+
+        Class<?> clazz = Class.forName(kafkaConnectorFQClassName);
+        connector = (SinkConnector) clazz.getConstructor().newInstance();
+
+        Class<? extends Task> taskClass = connector.taskClass();
+        sinkContext = new PulsarKafkaSinkContext();
+        connector.initialize(sinkContext);
+        connector.start(Maps.fromProperties(props));
+
+        List<Map<String, String>> configs = connector.taskConfigs(1);
+        Preconditions.checkNotNull(configs);
+        Preconditions.checkArgument(configs.size() == 1);
+
+        configs.forEach(x -> {
+            x.put(OFFSET_STORAGE_TOPIC_CONFIG, 
kafkaSinkConfig.getOffsetStorageTopic());
+            x.put(PULSAR_SERVICE_URL_CONFIG, 
kafkaSinkConfig.getPulsarServiceUrl());
+        });
+        task = (SinkTask) taskClass.getConstructor().newInstance();
+        taskContext =
+                new PulsarKafkaSinkTaskContext(configs.get(0), ctx, 
task::open);
+        task.initialize(taskContext);
+        task.start(configs.get(0));
+
+        batchSize = kafkaSinkConfig.getBatchSize();
+        lingerMs = kafkaSinkConfig.getLingerTimeMs();
+        scheduledExecutor.scheduleAtFixedRate(() ->
+                this.flushIfNeeded(true), lingerMs, lingerMs, 
TimeUnit.MILLISECONDS);
+
+
+        isRunning = true;
+        log.info("Kafka sink started : {}.", props);
+    }
+
+    private void flushIfNeeded(boolean force) {
+        if (force || pendingFlushQueue.stream().limit(batchSize).count() >= 
batchSize) {

Review comment:
       two questions here:
   
   1) why do we need this transformation here? Can't we just use 
`pendingFlushQueue.size()`?
   
   2) why do we use link queue not other queue implementation which provides 
more efficient `size()` implementation?

##########
File path: 
pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/PulsarKafkaSinkContext.java
##########
@@ -0,0 +1,38 @@
+/**
+ * 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.io.kafka.connect;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.kafka.connect.connector.ConnectorContext;
+
+@Slf4j
+public class PulsarKafkaSinkContext implements ConnectorContext {
+
+    @Override
+    public void requestTaskReconfiguration() {
+        throw new UnsupportedOperationException("not implemented");
+    }
+
+    @Override
+    public void raiseError(Exception e) {
+        log.error("raiseError requested", e);

Review comment:
       If it is not supported, why do you want to log this message?

##########
File path: 
pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/PulsarKafkaSinkTaskContext.java
##########
@@ -0,0 +1,219 @@
+/**
+ * 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.io.kafka.connect;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.connect.sink.SinkTaskContext;
+import org.apache.kafka.connect.storage.OffsetBackingStore;
+import org.apache.pulsar.io.core.SinkContext;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static 
org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig.TOPIC_NAMESPACE_CONFIG;
+
+@Slf4j
+public class PulsarKafkaSinkTaskContext implements SinkTaskContext {
+
+    private final Map<String, String> config;
+
+    private final OffsetBackingStore offsetStore;
+    private final String topicNamespace;
+    private final Consumer<Collection<TopicPartition>> onPartitionChange;
+    private final AtomicBoolean runRepartition = new AtomicBoolean(false);
+
+    private final ConcurrentHashMap<TopicPartition, AtomicLong> currentOffsets 
= new ConcurrentHashMap<>();
+
+    public PulsarKafkaSinkTaskContext(Map<String, String> config,
+                                      SinkContext ctx,
+                                      Consumer<Collection<TopicPartition>> 
onPartitionChange) {
+        this.config = config;
+
+        offsetStore = new PulsarOffsetBackingStore();
+        PulsarKafkaWorkerConfig pulsarKafkaWorkerConfig = new 
PulsarKafkaWorkerConfig(config);
+        offsetStore.configure(pulsarKafkaWorkerConfig);
+        offsetStore.start();
+
+        this.onPartitionChange = onPartitionChange;
+        this.topicNamespace = 
pulsarKafkaWorkerConfig.getString(TOPIC_NAMESPACE_CONFIG);
+    }
+
+    public void close() {
+        offsetStore.stop();
+    }
+
+    @Override
+    public Map<String, String> configs() {
+        return config;
+    }
+
+    public AtomicLong currentOffset(String topic, int partition) {
+        return currentOffset(new TopicPartition(topic, partition));
+    }
+
+    public AtomicLong currentOffset(TopicPartition topicPartition) {
+        AtomicLong offset = currentOffsets.computeIfAbsent(topicPartition, kv 
-> {
+            List<ByteBuffer> req = Lists.newLinkedList();
+            ByteBuffer key = topicPartitionAsKey(topicPartition);
+            req.add(key);
+            CompletableFuture<Long> offsetFuture = new CompletableFuture<>();
+            offsetStore.get(req, (Throwable ex, Map<ByteBuffer, ByteBuffer> 
result) -> {
+                if (ex == null) {
+                    if (result != null && result.size() != 0) {
+                        Optional<ByteBuffer> val = result.entrySet().stream()
+                                .filter(entry -> entry.getKey().equals(key))
+                                .findFirst().map(entry -> entry.getValue());
+                        if (val.isPresent()) {
+                            long received = val.get().getLong();
+                            if (log.isDebugEnabled()) {
+                                log.debug("read initial offset for {} == {}", 
topicPartition, received);
+                            }
+                            offsetFuture.complete(received);
+                            return;
+                        }
+                    }
+                    offsetFuture.complete(-1L);
+                } else {
+                    offsetFuture.completeExceptionally(ex);
+                }
+            });
+
+            runRepartition.set(true);
+            try {
+                return new AtomicLong(offsetFuture.get());
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                log.error("error getting initial state of {}", topicPartition, 
e);
+                throw new RuntimeException("error getting initial state of " + 
topicPartition, e);
+            } catch (ExecutionException e) {
+                log.error("error getting initial state of {}", topicPartition, 
e);
+                throw new RuntimeException("error getting initial state of " + 
topicPartition, e);            }
+        });
+        if (runRepartition.compareAndSet(true, false)) {
+            onPartitionChange.accept(currentOffsets.keySet());
+        }
+        return offset;
+    }
+
+    public Map<TopicPartition, OffsetAndMetadata> currentOffsets() {
+        Map<TopicPartition, OffsetAndMetadata> snapshot = 
Maps.newHashMapWithExpectedSize(currentOffsets.size());
+        currentOffsets.forEach((topicPartition, offset) -> {
+            if (offset.get() > 0) {
+                snapshot.put(topicPartition,
+                        new OffsetAndMetadata(offset.get(), Optional.empty(), 
null));
+            }
+        });
+        return snapshot;
+    }
+
+    private ByteBuffer topicPartitionAsKey(TopicPartition topicPartition) {
+        return ByteBuffer.wrap((topicNamespace + "/" + 
topicPartition.toString()).getBytes(UTF_8));
+
+    }
+
+    private void fillOffsetMap(Map<ByteBuffer, ByteBuffer> offsetMap, 
TopicPartition topicPartition, long l) {
+        ByteBuffer key = topicPartitionAsKey(topicPartition);
+        ByteBuffer value = ByteBuffer.allocate(Long.BYTES);
+        value.putLong(l);
+        value.flip();
+        offsetMap.put(key, value);
+    }
+
+    @SneakyThrows
+    @Override
+    public void offset(Map<TopicPartition, Long> map) {
+        map.forEach((key, value) -> {
+            if (!currentOffsets.containsKey(key)) {
+                runRepartition.set(true);
+            }
+            currentOffsets.put(key, new AtomicLong(value));
+        });
+
+        if (runRepartition.compareAndSet(true, false)) {
+            onPartitionChange.accept(currentOffsets.keySet());
+        }
+    }
+
+    @Override
+    public void offset(TopicPartition topicPartition, long l) {
+        Map<TopicPartition, Long> map = Maps.newHashMap();
+        map.put(topicPartition, l);
+        this.offset(map);
+    }
+
+    @Override
+    public void timeout(long l) {
+        log.warn("timeout() is called but is not supported currently.");
+    }
+
+    @Override
+    public Set<TopicPartition> assignment() {
+        return currentOffsets.keySet();
+    }
+
+    @Override
+    public void pause(TopicPartition... topicPartitions) {
+        log.warn("pause() is called but is not supported currently.");
+    }
+
+    @Override
+    public void resume(TopicPartition... topicPartitions) {
+        log.warn("resume() is called but is not supported currently.");
+    }
+
+    @Override
+    public void requestCommit() {
+        log.warn("requestCommit() is called but is not supported currently.");
+    }
+
+    public void flushOffsets(Map<TopicPartition, OffsetAndMetadata> offsets) 
throws Exception {
+        Map<ByteBuffer, ByteBuffer> offsetMap = 
Maps.newHashMapWithExpectedSize(offsets.size());
+
+        offsets.forEach((tp, om) -> fillOffsetMap(offsetMap, tp, om.offset()));
+        CompletableFuture<Void> result = new CompletableFuture();
+        offsetStore.set(offsetMap, (ex, ignore) -> {
+            if (ex == null) {
+                result.complete(null);
+            } else {
+                log.error("error flushing offsets for {}",
+                        
Joiner.on(",").withKeyValueSeparator("=").join(offsets), ex);
+                result.completeExceptionally(ex);
+            }
+        });
+        result.get();

Review comment:
       does this need to be blocking?

##########
File path: 
pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/KafkaConnectSink.java
##########
@@ -0,0 +1,325 @@
+/**
+ * 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.io.kafka.connect;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.TimestampType;
+import org.apache.kafka.connect.connector.Task;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.sink.SinkConnector;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.pulsar.client.api.schema.GenericObject;
+import org.apache.pulsar.client.impl.schema.KeyValueSchema;
+import org.apache.pulsar.common.schema.SchemaType;
+import org.apache.pulsar.functions.api.Record;
+import org.apache.pulsar.io.core.KeyValue;
+import org.apache.pulsar.io.core.Sink;
+import org.apache.pulsar.io.core.SinkContext;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig.OFFSET_STORAGE_TOPIC_CONFIG;
+import static 
org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig.PULSAR_SERVICE_URL_CONFIG;
+
+@Slf4j
+public class KafkaConnectSink implements Sink<GenericObject> {
+
+    private boolean unwrapKeyValueIfAvailable;
+
+    private final static ImmutableMap<Class<?>, Schema> primitiveTypeToSchema;
+    private final static ImmutableMap<SchemaType, Schema> 
pulsarSchemaTypeTypeToKafkaSchema;
+
+    static {
+        primitiveTypeToSchema = ImmutableMap.<Class<?>, Schema>builder()
+                .put(Boolean.class, Schema.BOOLEAN_SCHEMA)
+                .put(Byte.class, Schema.INT8_SCHEMA)
+                .put(Short.class, Schema.INT16_SCHEMA)
+                .put(Integer.class, Schema.INT32_SCHEMA)
+                .put(Long.class, Schema.INT64_SCHEMA)
+                .put(Float.class, Schema.FLOAT32_SCHEMA)
+                .put(Double.class, Schema.FLOAT64_SCHEMA)
+                .put(String.class, Schema.STRING_SCHEMA)
+                .put(byte[].class, Schema.BYTES_SCHEMA)
+                .build();
+        pulsarSchemaTypeTypeToKafkaSchema = ImmutableMap.<SchemaType, 
Schema>builder()
+                .put(SchemaType.BOOLEAN, Schema.BOOLEAN_SCHEMA)
+                .put(SchemaType.INT8, Schema.INT8_SCHEMA)
+                .put(SchemaType.INT16, Schema.INT16_SCHEMA)
+                .put(SchemaType.INT32, Schema.INT32_SCHEMA)
+                .put(SchemaType.INT64, Schema.INT64_SCHEMA)
+                .put(SchemaType.FLOAT, Schema.FLOAT32_SCHEMA)
+                .put(SchemaType.DOUBLE, Schema.FLOAT64_SCHEMA)
+                .put(SchemaType.STRING, Schema.STRING_SCHEMA)
+                .put(SchemaType.BYTES, Schema.BYTES_SCHEMA)
+                .build();
+    }
+
+    private PulsarKafkaSinkContext sinkContext;
+    private PulsarKafkaSinkTaskContext taskContext;
+    private SinkConnector connector;
+    private SinkTask task;
+
+    private int batchSize;
+    private long lingerMs;
+    private final ScheduledExecutorService scheduledExecutor =
+            Executors.newSingleThreadScheduledExecutor(new 
ThreadFactoryBuilder()
+                    .setNameFormat("pulsar-io-kafka-adaptor-sink-flush-%d")
+                    .build());
+    private final ConcurrentLinkedDeque<Record<GenericObject>> 
pendingFlushQueue = new ConcurrentLinkedDeque<>();
+    private volatile boolean isRunning = false;
+
+    private final Properties props = new Properties();
+    private PulsarKafkaConnectSinkConfig kafkaSinkConfig;
+
+    protected String topicName;
+
+    @Override
+    public void write(Record<GenericObject> sourceRecord) {
+        if (log.isDebugEnabled()) {
+            log.debug("Record sending to kafka, record={}.", sourceRecord);
+        }
+
+        if (!isRunning) {
+            log.error("Sink is stopped. Cannot send the record {}", 
sourceRecord);
+            sourceRecord.fail();
+            return;
+        }
+
+        try {
+            SinkRecord record = toSinkRecord(sourceRecord);
+            task.put(Lists.newArrayList(record));
+        } catch (Exception ex) {
+            log.error("Error sending the record {}", sourceRecord, ex);
+            sourceRecord.fail();
+            return;
+        }
+        pendingFlushQueue.add(sourceRecord);
+        flushIfNeeded(false);
+    }
+
+    @Override
+    public void close() throws Exception {
+        isRunning = false;
+        flushIfNeeded(true);
+        scheduledExecutor.shutdown();
+        if (!scheduledExecutor.awaitTermination(10 * lingerMs, 
TimeUnit.MILLISECONDS)) {
+            log.error("scheduledExecutor did not terminate in {} ms", 10 * 
lingerMs);
+        }
+
+        task.stop();
+        connector.stop();
+        taskContext.close();
+
+        log.info("Kafka sink stopped.");
+    }
+
+    @Override
+    public void open(Map<String, Object> config, SinkContext ctx) throws 
Exception {
+        kafkaSinkConfig = PulsarKafkaConnectSinkConfig.load(config);
+        Objects.requireNonNull(kafkaSinkConfig.getTopic(), "Kafka topic is not 
set");
+        topicName = kafkaSinkConfig.getTopic();
+        unwrapKeyValueIfAvailable = 
kafkaSinkConfig.isUnwrapKeyValueIfAvailable();
+
+        String kafkaConnectorFQClassName = 
kafkaSinkConfig.getKafkaConnectorSinkClass();
+        
kafkaSinkConfig.getKafkaConnectorConfigProperties().forEach(props::put);
+
+        Class<?> clazz = Class.forName(kafkaConnectorFQClassName);
+        connector = (SinkConnector) clazz.getConstructor().newInstance();
+
+        Class<? extends Task> taskClass = connector.taskClass();
+        sinkContext = new PulsarKafkaSinkContext();
+        connector.initialize(sinkContext);
+        connector.start(Maps.fromProperties(props));
+
+        List<Map<String, String>> configs = connector.taskConfigs(1);
+        Preconditions.checkNotNull(configs);
+        Preconditions.checkArgument(configs.size() == 1);
+
+        configs.forEach(x -> {
+            x.put(OFFSET_STORAGE_TOPIC_CONFIG, 
kafkaSinkConfig.getOffsetStorageTopic());
+            x.put(PULSAR_SERVICE_URL_CONFIG, 
kafkaSinkConfig.getPulsarServiceUrl());
+        });
+        task = (SinkTask) taskClass.getConstructor().newInstance();
+        taskContext =
+                new PulsarKafkaSinkTaskContext(configs.get(0), ctx, 
task::open);
+        task.initialize(taskContext);
+        task.start(configs.get(0));
+
+        batchSize = kafkaSinkConfig.getBatchSize();
+        lingerMs = kafkaSinkConfig.getLingerTimeMs();
+        scheduledExecutor.scheduleAtFixedRate(() ->
+                this.flushIfNeeded(true), lingerMs, lingerMs, 
TimeUnit.MILLISECONDS);
+
+
+        isRunning = true;
+        log.info("Kafka sink started : {}.", props);
+    }
+
+    private void flushIfNeeded(boolean force) {
+        if (force || pendingFlushQueue.stream().limit(batchSize).count() >= 
batchSize) {
+            scheduledExecutor.submit(this::flush);
+        }
+    }
+
+    // flush always happens on the same thread
+    public void flush() {
+        if (log.isDebugEnabled()) {
+            log.debug("flush requested, pending: {}, batchSize: {}",
+                    pendingFlushQueue.size(), batchSize);
+        }
+
+        if (pendingFlushQueue.isEmpty()) {
+            return;
+        }
+
+        final Record<GenericObject> lastNotFlushed = 
pendingFlushQueue.getLast();
+        Map<TopicPartition, OffsetAndMetadata> currentOffsets = 
taskContext.currentOffsets();
+
+        try {
+            task.flush(currentOffsets);
+            taskContext.flushOffsets(currentOffsets);
+            ackUntil(lastNotFlushed, Record::ack);
+        } catch (Throwable t) {
+            log.error("error flushing pending records", t);
+            ackUntil(lastNotFlushed, Record::fail);
+        }
+    }
+
+    private void ackUntil(Record<GenericObject> lastNotFlushed, 
java.util.function.Consumer<Record<GenericObject>> cb) {
+        while (!pendingFlushQueue.isEmpty()) {
+            Record<GenericObject> r = pendingFlushQueue.pollFirst();
+            cb.accept(r);
+            if (r == lastNotFlushed) {
+                break;
+            }
+        }
+    }
+
+    /**
+     * org.apache.kafka.connect.data.Schema for the object
+     * @param obj - Object to get schema of.
+     * @return org.apache.kafka.connect.data.Schema
+     */
+    private static Schema getKafkaConnectSchemaForObject(Object obj) {
+        if (obj != null && primitiveTypeToSchema.containsKey(obj.getClass())) {
+            return primitiveTypeToSchema.get(obj.getClass());
+        }
+        return null;
+    }
+
+    public static Schema 
getKafkaConnectSchema(org.apache.pulsar.client.api.Schema pulsarSchema, Object 
obj) {
+        if (pulsarSchema != null && pulsarSchema.getSchemaInfo() != null
+                && 
pulsarSchemaTypeTypeToKafkaSchema.containsKey(pulsarSchema.getSchemaInfo().getType()))
 {
+            return 
pulsarSchemaTypeTypeToKafkaSchema.get(pulsarSchema.getSchemaInfo().getType());
+        }
+
+        Schema result = getKafkaConnectSchemaForObject(obj);
+        if (result == null) {
+            throw new IllegalStateException("Unsupported kafka schema for 
Pulsar Schema "
+                    + (pulsarSchema == null || pulsarSchema.getSchemaInfo() == 
null
+                        ? "null"
+                        : pulsarSchema.getSchemaInfo().toString())
+                    + " object class "
+                    + (obj == null ? "null" : 
obj.getClass().getCanonicalName()));
+        }
+        return result;
+    }
+
+
+    @SuppressWarnings("rawtypes")
+    private SinkRecord toSinkRecord(Record<GenericObject> sourceRecord) {
+        final int partition = sourceRecord.getPartitionIndex().orElse(0);
+        final String topic = sourceRecord.getTopicName().orElse(topicName);
+        final Object key;
+        final Object value;
+        final Schema keySchema;
+        final Schema valueSchema;
+
+        // sourceRecord is never instanceof KVRecord
+        // https://github.com/apache/pulsar/pull/10113
+        if (unwrapKeyValueIfAvailable && sourceRecord.getSchema() != null
+                && sourceRecord.getSchema().getSchemaInfo() != null
+                && sourceRecord.getSchema().getSchemaInfo().getType() == 
SchemaType.KEY_VALUE) {
+            KeyValueSchema kvSchema = (KeyValueSchema) 
sourceRecord.getSchema();
+            KeyValue kv = (KeyValue) sourceRecord.getValue().getNativeObject();
+            key = kv.getKey();
+            value = kv.getValue();
+            keySchema = getKafkaConnectSchema(kvSchema.getKeySchema(), key);
+            valueSchema = getKafkaConnectSchema(kvSchema.getValueSchema(), 
value);
+        } else {
+            key = sourceRecord.getKey().orElse(null);
+            value = sourceRecord.getValue().getNativeObject();
+            keySchema = Schema.STRING_SCHEMA;
+            valueSchema = getKafkaConnectSchema(sourceRecord.getSchema(), 
value);
+        }
+
+        long offset = sourceRecord.getRecordSequence()
+                .orElse(-1L);
+        if (offset < 0) {
+            offset = taskContext.currentOffset(topic, partition)
+                    .incrementAndGet();
+        } else {
+            final long curr = offset;
+            taskContext.currentOffset(topic, partition)
+                    .updateAndGet(curMax -> Math.max(curr, curMax));
+        }
+
+        Long timestamp = null;
+        TimestampType timestampType = TimestampType.NO_TIMESTAMP_TYPE;
+        if (sourceRecord.getEventTime().isPresent()) {
+            timestamp = sourceRecord.getEventTime().get();
+            timestampType = TimestampType.CREATE_TIME;
+        } else if (sourceRecord.getMessage().isPresent()) {
+            timestamp = sourceRecord.getMessage().get().getPublishTime();
+            timestampType = TimestampType.LOG_APPEND_TIME;

Review comment:
       Technically publishTime is not log append time. I think we should call 
it out in the documentation.

##########
File path: 
pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/PulsarKafkaSinkTaskContext.java
##########
@@ -0,0 +1,219 @@
+/**
+ * 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.io.kafka.connect;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.connect.sink.SinkTaskContext;
+import org.apache.kafka.connect.storage.OffsetBackingStore;
+import org.apache.pulsar.io.core.SinkContext;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static 
org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig.TOPIC_NAMESPACE_CONFIG;
+
+@Slf4j
+public class PulsarKafkaSinkTaskContext implements SinkTaskContext {
+
+    private final Map<String, String> config;
+
+    private final OffsetBackingStore offsetStore;
+    private final String topicNamespace;
+    private final Consumer<Collection<TopicPartition>> onPartitionChange;
+    private final AtomicBoolean runRepartition = new AtomicBoolean(false);
+
+    private final ConcurrentHashMap<TopicPartition, AtomicLong> currentOffsets 
= new ConcurrentHashMap<>();
+
+    public PulsarKafkaSinkTaskContext(Map<String, String> config,
+                                      SinkContext ctx,
+                                      Consumer<Collection<TopicPartition>> 
onPartitionChange) {
+        this.config = config;
+
+        offsetStore = new PulsarOffsetBackingStore();
+        PulsarKafkaWorkerConfig pulsarKafkaWorkerConfig = new 
PulsarKafkaWorkerConfig(config);
+        offsetStore.configure(pulsarKafkaWorkerConfig);
+        offsetStore.start();
+
+        this.onPartitionChange = onPartitionChange;
+        this.topicNamespace = 
pulsarKafkaWorkerConfig.getString(TOPIC_NAMESPACE_CONFIG);
+    }
+
+    public void close() {
+        offsetStore.stop();
+    }
+
+    @Override
+    public Map<String, String> configs() {
+        return config;
+    }
+
+    public AtomicLong currentOffset(String topic, int partition) {
+        return currentOffset(new TopicPartition(topic, partition));
+    }
+
+    public AtomicLong currentOffset(TopicPartition topicPartition) {
+        AtomicLong offset = currentOffsets.computeIfAbsent(topicPartition, kv 
-> {
+            List<ByteBuffer> req = Lists.newLinkedList();
+            ByteBuffer key = topicPartitionAsKey(topicPartition);
+            req.add(key);
+            CompletableFuture<Long> offsetFuture = new CompletableFuture<>();
+            offsetStore.get(req, (Throwable ex, Map<ByteBuffer, ByteBuffer> 
result) -> {
+                if (ex == null) {
+                    if (result != null && result.size() != 0) {
+                        Optional<ByteBuffer> val = result.entrySet().stream()
+                                .filter(entry -> entry.getKey().equals(key))
+                                .findFirst().map(entry -> entry.getValue());
+                        if (val.isPresent()) {
+                            long received = val.get().getLong();
+                            if (log.isDebugEnabled()) {
+                                log.debug("read initial offset for {} == {}", 
topicPartition, received);
+                            }
+                            offsetFuture.complete(received);
+                            return;
+                        }
+                    }
+                    offsetFuture.complete(-1L);
+                } else {
+                    offsetFuture.completeExceptionally(ex);
+                }
+            });
+
+            runRepartition.set(true);
+            try {
+                return new AtomicLong(offsetFuture.get());
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                log.error("error getting initial state of {}", topicPartition, 
e);
+                throw new RuntimeException("error getting initial state of " + 
topicPartition, e);
+            } catch (ExecutionException e) {
+                log.error("error getting initial state of {}", topicPartition, 
e);
+                throw new RuntimeException("error getting initial state of " + 
topicPartition, e);            }
+        });
+        if (runRepartition.compareAndSet(true, false)) {
+            onPartitionChange.accept(currentOffsets.keySet());
+        }
+        return offset;
+    }
+
+    public Map<TopicPartition, OffsetAndMetadata> currentOffsets() {
+        Map<TopicPartition, OffsetAndMetadata> snapshot = 
Maps.newHashMapWithExpectedSize(currentOffsets.size());
+        currentOffsets.forEach((topicPartition, offset) -> {
+            if (offset.get() > 0) {
+                snapshot.put(topicPartition,
+                        new OffsetAndMetadata(offset.get(), Optional.empty(), 
null));
+            }
+        });
+        return snapshot;
+    }
+
+    private ByteBuffer topicPartitionAsKey(TopicPartition topicPartition) {
+        return ByteBuffer.wrap((topicNamespace + "/" + 
topicPartition.toString()).getBytes(UTF_8));
+
+    }
+
+    private void fillOffsetMap(Map<ByteBuffer, ByteBuffer> offsetMap, 
TopicPartition topicPartition, long l) {
+        ByteBuffer key = topicPartitionAsKey(topicPartition);
+        ByteBuffer value = ByteBuffer.allocate(Long.BYTES);
+        value.putLong(l);
+        value.flip();
+        offsetMap.put(key, value);
+    }
+
+    @SneakyThrows
+    @Override
+    public void offset(Map<TopicPartition, Long> map) {
+        map.forEach((key, value) -> {
+            if (!currentOffsets.containsKey(key)) {
+                runRepartition.set(true);
+            }
+            currentOffsets.put(key, new AtomicLong(value));
+        });
+
+        if (runRepartition.compareAndSet(true, false)) {
+            onPartitionChange.accept(currentOffsets.keySet());
+        }
+    }
+
+    @Override
+    public void offset(TopicPartition topicPartition, long l) {
+        Map<TopicPartition, Long> map = Maps.newHashMap();
+        map.put(topicPartition, l);
+        this.offset(map);
+    }
+
+    @Override
+    public void timeout(long l) {
+        log.warn("timeout() is called but is not supported currently.");
+    }
+
+    @Override
+    public Set<TopicPartition> assignment() {
+        return currentOffsets.keySet();
+    }
+
+    @Override
+    public void pause(TopicPartition... topicPartitions) {
+        log.warn("pause() is called but is not supported currently.");
+    }
+
+    @Override
+    public void resume(TopicPartition... topicPartitions) {
+        log.warn("resume() is called but is not supported currently.");
+    }
+
+    @Override
+    public void requestCommit() {
+        log.warn("requestCommit() is called but is not supported currently.");
+    }
+
+    public void flushOffsets(Map<TopicPartition, OffsetAndMetadata> offsets) 
throws Exception {
+        Map<ByteBuffer, ByteBuffer> offsetMap = 
Maps.newHashMapWithExpectedSize(offsets.size());
+
+        offsets.forEach((tp, om) -> fillOffsetMap(offsetMap, tp, om.offset()));
+        CompletableFuture<Void> result = new CompletableFuture();
+        offsetStore.set(offsetMap, (ex, ignore) -> {
+            if (ex == null) {
+                result.complete(null);
+            } else {
+                log.error("error flushing offsets for {}",
+                        
Joiner.on(",").withKeyValueSeparator("=").join(offsets), ex);

Review comment:
       Do we need to run `Joiner` here?

##########
File path: 
pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/KafkaConnectSink.java
##########
@@ -0,0 +1,325 @@
+/**
+ * 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.io.kafka.connect;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.TimestampType;
+import org.apache.kafka.connect.connector.Task;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.sink.SinkConnector;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.pulsar.client.api.schema.GenericObject;
+import org.apache.pulsar.client.impl.schema.KeyValueSchema;
+import org.apache.pulsar.common.schema.SchemaType;
+import org.apache.pulsar.functions.api.Record;
+import org.apache.pulsar.io.core.KeyValue;
+import org.apache.pulsar.io.core.Sink;
+import org.apache.pulsar.io.core.SinkContext;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig.OFFSET_STORAGE_TOPIC_CONFIG;
+import static 
org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig.PULSAR_SERVICE_URL_CONFIG;
+
+@Slf4j
+public class KafkaConnectSink implements Sink<GenericObject> {
+
+    private boolean unwrapKeyValueIfAvailable;
+
+    private final static ImmutableMap<Class<?>, Schema> primitiveTypeToSchema;
+    private final static ImmutableMap<SchemaType, Schema> 
pulsarSchemaTypeTypeToKafkaSchema;
+
+    static {
+        primitiveTypeToSchema = ImmutableMap.<Class<?>, Schema>builder()
+                .put(Boolean.class, Schema.BOOLEAN_SCHEMA)
+                .put(Byte.class, Schema.INT8_SCHEMA)
+                .put(Short.class, Schema.INT16_SCHEMA)
+                .put(Integer.class, Schema.INT32_SCHEMA)
+                .put(Long.class, Schema.INT64_SCHEMA)
+                .put(Float.class, Schema.FLOAT32_SCHEMA)
+                .put(Double.class, Schema.FLOAT64_SCHEMA)
+                .put(String.class, Schema.STRING_SCHEMA)
+                .put(byte[].class, Schema.BYTES_SCHEMA)
+                .build();
+        pulsarSchemaTypeTypeToKafkaSchema = ImmutableMap.<SchemaType, 
Schema>builder()
+                .put(SchemaType.BOOLEAN, Schema.BOOLEAN_SCHEMA)
+                .put(SchemaType.INT8, Schema.INT8_SCHEMA)
+                .put(SchemaType.INT16, Schema.INT16_SCHEMA)
+                .put(SchemaType.INT32, Schema.INT32_SCHEMA)
+                .put(SchemaType.INT64, Schema.INT64_SCHEMA)
+                .put(SchemaType.FLOAT, Schema.FLOAT32_SCHEMA)
+                .put(SchemaType.DOUBLE, Schema.FLOAT64_SCHEMA)
+                .put(SchemaType.STRING, Schema.STRING_SCHEMA)
+                .put(SchemaType.BYTES, Schema.BYTES_SCHEMA)
+                .build();
+    }
+
+    private PulsarKafkaSinkContext sinkContext;
+    private PulsarKafkaSinkTaskContext taskContext;
+    private SinkConnector connector;
+    private SinkTask task;
+
+    private int batchSize;
+    private long lingerMs;
+    private final ScheduledExecutorService scheduledExecutor =
+            Executors.newSingleThreadScheduledExecutor(new 
ThreadFactoryBuilder()
+                    .setNameFormat("pulsar-io-kafka-adaptor-sink-flush-%d")
+                    .build());
+    private final ConcurrentLinkedDeque<Record<GenericObject>> 
pendingFlushQueue = new ConcurrentLinkedDeque<>();
+    private volatile boolean isRunning = false;
+
+    private final Properties props = new Properties();
+    private PulsarKafkaConnectSinkConfig kafkaSinkConfig;
+
+    protected String topicName;
+
+    @Override
+    public void write(Record<GenericObject> sourceRecord) {
+        if (log.isDebugEnabled()) {
+            log.debug("Record sending to kafka, record={}.", sourceRecord);
+        }
+
+        if (!isRunning) {
+            log.error("Sink is stopped. Cannot send the record {}", 
sourceRecord);
+            sourceRecord.fail();
+            return;
+        }
+
+        try {
+            SinkRecord record = toSinkRecord(sourceRecord);
+            task.put(Lists.newArrayList(record));
+        } catch (Exception ex) {
+            log.error("Error sending the record {}", sourceRecord, ex);
+            sourceRecord.fail();
+            return;
+        }
+        pendingFlushQueue.add(sourceRecord);
+        flushIfNeeded(false);
+    }
+
+    @Override
+    public void close() throws Exception {
+        isRunning = false;
+        flushIfNeeded(true);
+        scheduledExecutor.shutdown();
+        if (!scheduledExecutor.awaitTermination(10 * lingerMs, 
TimeUnit.MILLISECONDS)) {
+            log.error("scheduledExecutor did not terminate in {} ms", 10 * 
lingerMs);
+        }
+
+        task.stop();
+        connector.stop();
+        taskContext.close();
+
+        log.info("Kafka sink stopped.");
+    }
+
+    @Override
+    public void open(Map<String, Object> config, SinkContext ctx) throws 
Exception {
+        kafkaSinkConfig = PulsarKafkaConnectSinkConfig.load(config);
+        Objects.requireNonNull(kafkaSinkConfig.getTopic(), "Kafka topic is not 
set");
+        topicName = kafkaSinkConfig.getTopic();
+        unwrapKeyValueIfAvailable = 
kafkaSinkConfig.isUnwrapKeyValueIfAvailable();
+
+        String kafkaConnectorFQClassName = 
kafkaSinkConfig.getKafkaConnectorSinkClass();
+        
kafkaSinkConfig.getKafkaConnectorConfigProperties().forEach(props::put);
+
+        Class<?> clazz = Class.forName(kafkaConnectorFQClassName);
+        connector = (SinkConnector) clazz.getConstructor().newInstance();
+
+        Class<? extends Task> taskClass = connector.taskClass();
+        sinkContext = new PulsarKafkaSinkContext();
+        connector.initialize(sinkContext);
+        connector.start(Maps.fromProperties(props));
+
+        List<Map<String, String>> configs = connector.taskConfigs(1);
+        Preconditions.checkNotNull(configs);
+        Preconditions.checkArgument(configs.size() == 1);
+
+        configs.forEach(x -> {
+            x.put(OFFSET_STORAGE_TOPIC_CONFIG, 
kafkaSinkConfig.getOffsetStorageTopic());
+            x.put(PULSAR_SERVICE_URL_CONFIG, 
kafkaSinkConfig.getPulsarServiceUrl());
+        });
+        task = (SinkTask) taskClass.getConstructor().newInstance();
+        taskContext =
+                new PulsarKafkaSinkTaskContext(configs.get(0), ctx, 
task::open);
+        task.initialize(taskContext);
+        task.start(configs.get(0));
+
+        batchSize = kafkaSinkConfig.getBatchSize();
+        lingerMs = kafkaSinkConfig.getLingerTimeMs();
+        scheduledExecutor.scheduleAtFixedRate(() ->
+                this.flushIfNeeded(true), lingerMs, lingerMs, 
TimeUnit.MILLISECONDS);
+
+
+        isRunning = true;
+        log.info("Kafka sink started : {}.", props);
+    }
+
+    private void flushIfNeeded(boolean force) {
+        if (force || pendingFlushQueue.stream().limit(batchSize).count() >= 
batchSize) {
+            scheduledExecutor.submit(this::flush);
+        }
+    }
+
+    // flush always happens on the same thread
+    public void flush() {
+        if (log.isDebugEnabled()) {
+            log.debug("flush requested, pending: {}, batchSize: {}",
+                    pendingFlushQueue.size(), batchSize);
+        }
+
+        if (pendingFlushQueue.isEmpty()) {
+            return;
+        }
+
+        final Record<GenericObject> lastNotFlushed = 
pendingFlushQueue.getLast();
+        Map<TopicPartition, OffsetAndMetadata> currentOffsets = 
taskContext.currentOffsets();
+
+        try {
+            task.flush(currentOffsets);
+            taskContext.flushOffsets(currentOffsets);
+            ackUntil(lastNotFlushed, Record::ack);
+        } catch (Throwable t) {
+            log.error("error flushing pending records", t);
+            ackUntil(lastNotFlushed, Record::fail);
+        }
+    }
+
+    private void ackUntil(Record<GenericObject> lastNotFlushed, 
java.util.function.Consumer<Record<GenericObject>> cb) {
+        while (!pendingFlushQueue.isEmpty()) {
+            Record<GenericObject> r = pendingFlushQueue.pollFirst();
+            cb.accept(r);
+            if (r == lastNotFlushed) {
+                break;
+            }
+        }
+    }
+
+    /**
+     * org.apache.kafka.connect.data.Schema for the object
+     * @param obj - Object to get schema of.
+     * @return org.apache.kafka.connect.data.Schema
+     */
+    private static Schema getKafkaConnectSchemaForObject(Object obj) {
+        if (obj != null && primitiveTypeToSchema.containsKey(obj.getClass())) {
+            return primitiveTypeToSchema.get(obj.getClass());
+        }
+        return null;
+    }
+
+    public static Schema 
getKafkaConnectSchema(org.apache.pulsar.client.api.Schema pulsarSchema, Object 
obj) {
+        if (pulsarSchema != null && pulsarSchema.getSchemaInfo() != null
+                && 
pulsarSchemaTypeTypeToKafkaSchema.containsKey(pulsarSchema.getSchemaInfo().getType()))
 {
+            return 
pulsarSchemaTypeTypeToKafkaSchema.get(pulsarSchema.getSchemaInfo().getType());
+        }
+
+        Schema result = getKafkaConnectSchemaForObject(obj);
+        if (result == null) {
+            throw new IllegalStateException("Unsupported kafka schema for 
Pulsar Schema "
+                    + (pulsarSchema == null || pulsarSchema.getSchemaInfo() == 
null
+                        ? "null"
+                        : pulsarSchema.getSchemaInfo().toString())
+                    + " object class "
+                    + (obj == null ? "null" : 
obj.getClass().getCanonicalName()));
+        }
+        return result;
+    }
+
+
+    @SuppressWarnings("rawtypes")
+    private SinkRecord toSinkRecord(Record<GenericObject> sourceRecord) {
+        final int partition = sourceRecord.getPartitionIndex().orElse(0);
+        final String topic = sourceRecord.getTopicName().orElse(topicName);
+        final Object key;
+        final Object value;
+        final Schema keySchema;
+        final Schema valueSchema;
+
+        // sourceRecord is never instanceof KVRecord
+        // https://github.com/apache/pulsar/pull/10113
+        if (unwrapKeyValueIfAvailable && sourceRecord.getSchema() != null
+                && sourceRecord.getSchema().getSchemaInfo() != null
+                && sourceRecord.getSchema().getSchemaInfo().getType() == 
SchemaType.KEY_VALUE) {
+            KeyValueSchema kvSchema = (KeyValueSchema) 
sourceRecord.getSchema();
+            KeyValue kv = (KeyValue) sourceRecord.getValue().getNativeObject();
+            key = kv.getKey();
+            value = kv.getValue();
+            keySchema = getKafkaConnectSchema(kvSchema.getKeySchema(), key);
+            valueSchema = getKafkaConnectSchema(kvSchema.getValueSchema(), 
value);
+        } else {
+            key = sourceRecord.getKey().orElse(null);
+            value = sourceRecord.getValue().getNativeObject();
+            keySchema = Schema.STRING_SCHEMA;

Review comment:
       This should be `BYTES`. Pulsar uses `string` for storing keys doesn't 
mean the schema type is `String`. Because bytes are encoded as strings there as 
well.

##########
File path: 
pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/KafkaConnectSink.java
##########
@@ -0,0 +1,325 @@
+/**
+ * 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.io.kafka.connect;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.TimestampType;
+import org.apache.kafka.connect.connector.Task;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.sink.SinkConnector;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.pulsar.client.api.schema.GenericObject;
+import org.apache.pulsar.client.impl.schema.KeyValueSchema;
+import org.apache.pulsar.common.schema.SchemaType;
+import org.apache.pulsar.functions.api.Record;
+import org.apache.pulsar.io.core.KeyValue;
+import org.apache.pulsar.io.core.Sink;
+import org.apache.pulsar.io.core.SinkContext;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig.OFFSET_STORAGE_TOPIC_CONFIG;
+import static 
org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig.PULSAR_SERVICE_URL_CONFIG;
+
+@Slf4j
+public class KafkaConnectSink implements Sink<GenericObject> {
+
+    private boolean unwrapKeyValueIfAvailable;
+
+    private final static ImmutableMap<Class<?>, Schema> primitiveTypeToSchema;
+    private final static ImmutableMap<SchemaType, Schema> 
pulsarSchemaTypeTypeToKafkaSchema;
+
+    static {
+        primitiveTypeToSchema = ImmutableMap.<Class<?>, Schema>builder()
+                .put(Boolean.class, Schema.BOOLEAN_SCHEMA)
+                .put(Byte.class, Schema.INT8_SCHEMA)
+                .put(Short.class, Schema.INT16_SCHEMA)
+                .put(Integer.class, Schema.INT32_SCHEMA)
+                .put(Long.class, Schema.INT64_SCHEMA)
+                .put(Float.class, Schema.FLOAT32_SCHEMA)
+                .put(Double.class, Schema.FLOAT64_SCHEMA)
+                .put(String.class, Schema.STRING_SCHEMA)
+                .put(byte[].class, Schema.BYTES_SCHEMA)
+                .build();
+        pulsarSchemaTypeTypeToKafkaSchema = ImmutableMap.<SchemaType, 
Schema>builder()
+                .put(SchemaType.BOOLEAN, Schema.BOOLEAN_SCHEMA)
+                .put(SchemaType.INT8, Schema.INT8_SCHEMA)
+                .put(SchemaType.INT16, Schema.INT16_SCHEMA)
+                .put(SchemaType.INT32, Schema.INT32_SCHEMA)
+                .put(SchemaType.INT64, Schema.INT64_SCHEMA)
+                .put(SchemaType.FLOAT, Schema.FLOAT32_SCHEMA)
+                .put(SchemaType.DOUBLE, Schema.FLOAT64_SCHEMA)
+                .put(SchemaType.STRING, Schema.STRING_SCHEMA)
+                .put(SchemaType.BYTES, Schema.BYTES_SCHEMA)
+                .build();
+    }
+
+    private PulsarKafkaSinkContext sinkContext;
+    private PulsarKafkaSinkTaskContext taskContext;
+    private SinkConnector connector;
+    private SinkTask task;
+
+    private int batchSize;
+    private long lingerMs;
+    private final ScheduledExecutorService scheduledExecutor =
+            Executors.newSingleThreadScheduledExecutor(new 
ThreadFactoryBuilder()
+                    .setNameFormat("pulsar-io-kafka-adaptor-sink-flush-%d")
+                    .build());
+    private final ConcurrentLinkedDeque<Record<GenericObject>> 
pendingFlushQueue = new ConcurrentLinkedDeque<>();
+    private volatile boolean isRunning = false;
+
+    private final Properties props = new Properties();
+    private PulsarKafkaConnectSinkConfig kafkaSinkConfig;
+
+    protected String topicName;
+
+    @Override
+    public void write(Record<GenericObject> sourceRecord) {
+        if (log.isDebugEnabled()) {
+            log.debug("Record sending to kafka, record={}.", sourceRecord);
+        }
+
+        if (!isRunning) {
+            log.error("Sink is stopped. Cannot send the record {}", 
sourceRecord);
+            sourceRecord.fail();
+            return;
+        }
+
+        try {
+            SinkRecord record = toSinkRecord(sourceRecord);
+            task.put(Lists.newArrayList(record));
+        } catch (Exception ex) {
+            log.error("Error sending the record {}", sourceRecord, ex);
+            sourceRecord.fail();
+            return;
+        }
+        pendingFlushQueue.add(sourceRecord);
+        flushIfNeeded(false);
+    }
+
+    @Override
+    public void close() throws Exception {
+        isRunning = false;
+        flushIfNeeded(true);
+        scheduledExecutor.shutdown();
+        if (!scheduledExecutor.awaitTermination(10 * lingerMs, 
TimeUnit.MILLISECONDS)) {
+            log.error("scheduledExecutor did not terminate in {} ms", 10 * 
lingerMs);
+        }
+
+        task.stop();
+        connector.stop();
+        taskContext.close();
+
+        log.info("Kafka sink stopped.");
+    }
+
+    @Override
+    public void open(Map<String, Object> config, SinkContext ctx) throws 
Exception {
+        kafkaSinkConfig = PulsarKafkaConnectSinkConfig.load(config);
+        Objects.requireNonNull(kafkaSinkConfig.getTopic(), "Kafka topic is not 
set");
+        topicName = kafkaSinkConfig.getTopic();
+        unwrapKeyValueIfAvailable = 
kafkaSinkConfig.isUnwrapKeyValueIfAvailable();
+
+        String kafkaConnectorFQClassName = 
kafkaSinkConfig.getKafkaConnectorSinkClass();
+        
kafkaSinkConfig.getKafkaConnectorConfigProperties().forEach(props::put);
+
+        Class<?> clazz = Class.forName(kafkaConnectorFQClassName);
+        connector = (SinkConnector) clazz.getConstructor().newInstance();
+
+        Class<? extends Task> taskClass = connector.taskClass();
+        sinkContext = new PulsarKafkaSinkContext();
+        connector.initialize(sinkContext);
+        connector.start(Maps.fromProperties(props));
+
+        List<Map<String, String>> configs = connector.taskConfigs(1);
+        Preconditions.checkNotNull(configs);
+        Preconditions.checkArgument(configs.size() == 1);
+
+        configs.forEach(x -> {
+            x.put(OFFSET_STORAGE_TOPIC_CONFIG, 
kafkaSinkConfig.getOffsetStorageTopic());
+            x.put(PULSAR_SERVICE_URL_CONFIG, 
kafkaSinkConfig.getPulsarServiceUrl());
+        });
+        task = (SinkTask) taskClass.getConstructor().newInstance();
+        taskContext =
+                new PulsarKafkaSinkTaskContext(configs.get(0), ctx, 
task::open);
+        task.initialize(taskContext);
+        task.start(configs.get(0));
+
+        batchSize = kafkaSinkConfig.getBatchSize();
+        lingerMs = kafkaSinkConfig.getLingerTimeMs();
+        scheduledExecutor.scheduleAtFixedRate(() ->
+                this.flushIfNeeded(true), lingerMs, lingerMs, 
TimeUnit.MILLISECONDS);
+
+
+        isRunning = true;
+        log.info("Kafka sink started : {}.", props);
+    }
+
+    private void flushIfNeeded(boolean force) {
+        if (force || pendingFlushQueue.stream().limit(batchSize).count() >= 
batchSize) {
+            scheduledExecutor.submit(this::flush);
+        }
+    }
+
+    // flush always happens on the same thread
+    public void flush() {
+        if (log.isDebugEnabled()) {
+            log.debug("flush requested, pending: {}, batchSize: {}",
+                    pendingFlushQueue.size(), batchSize);
+        }
+
+        if (pendingFlushQueue.isEmpty()) {
+            return;
+        }
+
+        final Record<GenericObject> lastNotFlushed = 
pendingFlushQueue.getLast();
+        Map<TopicPartition, OffsetAndMetadata> currentOffsets = 
taskContext.currentOffsets();
+
+        try {
+            task.flush(currentOffsets);
+            taskContext.flushOffsets(currentOffsets);
+            ackUntil(lastNotFlushed, Record::ack);
+        } catch (Throwable t) {
+            log.error("error flushing pending records", t);
+            ackUntil(lastNotFlushed, Record::fail);
+        }
+    }
+
+    private void ackUntil(Record<GenericObject> lastNotFlushed, 
java.util.function.Consumer<Record<GenericObject>> cb) {
+        while (!pendingFlushQueue.isEmpty()) {
+            Record<GenericObject> r = pendingFlushQueue.pollFirst();
+            cb.accept(r);
+            if (r == lastNotFlushed) {
+                break;
+            }
+        }
+    }
+
+    /**
+     * org.apache.kafka.connect.data.Schema for the object
+     * @param obj - Object to get schema of.
+     * @return org.apache.kafka.connect.data.Schema
+     */
+    private static Schema getKafkaConnectSchemaForObject(Object obj) {
+        if (obj != null && primitiveTypeToSchema.containsKey(obj.getClass())) {
+            return primitiveTypeToSchema.get(obj.getClass());
+        }
+        return null;
+    }
+
+    public static Schema 
getKafkaConnectSchema(org.apache.pulsar.client.api.Schema pulsarSchema, Object 
obj) {
+        if (pulsarSchema != null && pulsarSchema.getSchemaInfo() != null
+                && 
pulsarSchemaTypeTypeToKafkaSchema.containsKey(pulsarSchema.getSchemaInfo().getType()))
 {
+            return 
pulsarSchemaTypeTypeToKafkaSchema.get(pulsarSchema.getSchemaInfo().getType());
+        }
+
+        Schema result = getKafkaConnectSchemaForObject(obj);
+        if (result == null) {
+            throw new IllegalStateException("Unsupported kafka schema for 
Pulsar Schema "
+                    + (pulsarSchema == null || pulsarSchema.getSchemaInfo() == 
null
+                        ? "null"
+                        : pulsarSchema.getSchemaInfo().toString())
+                    + " object class "
+                    + (obj == null ? "null" : 
obj.getClass().getCanonicalName()));
+        }
+        return result;
+    }
+
+
+    @SuppressWarnings("rawtypes")
+    private SinkRecord toSinkRecord(Record<GenericObject> sourceRecord) {
+        final int partition = sourceRecord.getPartitionIndex().orElse(0);
+        final String topic = sourceRecord.getTopicName().orElse(topicName);
+        final Object key;
+        final Object value;
+        final Schema keySchema;
+        final Schema valueSchema;
+
+        // sourceRecord is never instanceof KVRecord
+        // https://github.com/apache/pulsar/pull/10113
+        if (unwrapKeyValueIfAvailable && sourceRecord.getSchema() != null
+                && sourceRecord.getSchema().getSchemaInfo() != null
+                && sourceRecord.getSchema().getSchemaInfo().getType() == 
SchemaType.KEY_VALUE) {
+            KeyValueSchema kvSchema = (KeyValueSchema) 
sourceRecord.getSchema();
+            KeyValue kv = (KeyValue) sourceRecord.getValue().getNativeObject();
+            key = kv.getKey();
+            value = kv.getValue();
+            keySchema = getKafkaConnectSchema(kvSchema.getKeySchema(), key);
+            valueSchema = getKafkaConnectSchema(kvSchema.getValueSchema(), 
value);
+        } else {
+            key = sourceRecord.getKey().orElse(null);
+            value = sourceRecord.getValue().getNativeObject();
+            keySchema = Schema.STRING_SCHEMA;
+            valueSchema = getKafkaConnectSchema(sourceRecord.getSchema(), 
value);
+        }
+
+        long offset = sourceRecord.getRecordSequence()
+                .orElse(-1L);
+        if (offset < 0) {
+            offset = taskContext.currentOffset(topic, partition)

Review comment:
       I don't think this is a good idea. We should fail fast on this. Because 
the source record here is from a Pulsar topic, it should always have the 
sequence id.

##########
File path: 
pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/PulsarKafkaConnectSinkConfig.java
##########
@@ -0,0 +1,100 @@
+/**
+ * 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.io.kafka.connect;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import lombok.Data;
+import lombok.experimental.Accessors;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Map;
+import org.apache.pulsar.io.core.annotations.FieldDoc;
+
+@Data
+@Accessors(chain = true)
+public class PulsarKafkaConnectSinkConfig implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    @FieldDoc(
+            defaultValue = "1",
+            help = "The batch size that Kafka producer will attempt to batch 
records together.")
+    private int batchSize = 1;

Review comment:
       1) if `batchSize` is used for representing number of messages, it should 
be renamed to `batchedMessages` or something else. `size` should be used for 
representing the total bytes. 
   
   2) You can and probably should implement the flush logic based on bytes not 
messages. As you said, `messages` is not good for calculating the total amount 
of memory, but `bytes` can.

##########
File path: 
pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/KafkaConnectSink.java
##########
@@ -0,0 +1,325 @@
+/**
+ * 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.io.kafka.connect;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.TimestampType;
+import org.apache.kafka.connect.connector.Task;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.sink.SinkConnector;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.pulsar.client.api.schema.GenericObject;
+import org.apache.pulsar.client.impl.schema.KeyValueSchema;
+import org.apache.pulsar.common.schema.SchemaType;
+import org.apache.pulsar.functions.api.Record;
+import org.apache.pulsar.io.core.KeyValue;
+import org.apache.pulsar.io.core.Sink;
+import org.apache.pulsar.io.core.SinkContext;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig.OFFSET_STORAGE_TOPIC_CONFIG;
+import static 
org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig.PULSAR_SERVICE_URL_CONFIG;
+
+@Slf4j
+public class KafkaConnectSink implements Sink<GenericObject> {
+
+    private boolean unwrapKeyValueIfAvailable;
+
+    private final static ImmutableMap<Class<?>, Schema> primitiveTypeToSchema;
+    private final static ImmutableMap<SchemaType, Schema> 
pulsarSchemaTypeTypeToKafkaSchema;
+
+    static {
+        primitiveTypeToSchema = ImmutableMap.<Class<?>, Schema>builder()
+                .put(Boolean.class, Schema.BOOLEAN_SCHEMA)
+                .put(Byte.class, Schema.INT8_SCHEMA)
+                .put(Short.class, Schema.INT16_SCHEMA)
+                .put(Integer.class, Schema.INT32_SCHEMA)
+                .put(Long.class, Schema.INT64_SCHEMA)
+                .put(Float.class, Schema.FLOAT32_SCHEMA)
+                .put(Double.class, Schema.FLOAT64_SCHEMA)
+                .put(String.class, Schema.STRING_SCHEMA)
+                .put(byte[].class, Schema.BYTES_SCHEMA)
+                .build();
+        pulsarSchemaTypeTypeToKafkaSchema = ImmutableMap.<SchemaType, 
Schema>builder()
+                .put(SchemaType.BOOLEAN, Schema.BOOLEAN_SCHEMA)
+                .put(SchemaType.INT8, Schema.INT8_SCHEMA)
+                .put(SchemaType.INT16, Schema.INT16_SCHEMA)
+                .put(SchemaType.INT32, Schema.INT32_SCHEMA)
+                .put(SchemaType.INT64, Schema.INT64_SCHEMA)
+                .put(SchemaType.FLOAT, Schema.FLOAT32_SCHEMA)
+                .put(SchemaType.DOUBLE, Schema.FLOAT64_SCHEMA)
+                .put(SchemaType.STRING, Schema.STRING_SCHEMA)
+                .put(SchemaType.BYTES, Schema.BYTES_SCHEMA)
+                .build();
+    }
+
+    private PulsarKafkaSinkContext sinkContext;
+    private PulsarKafkaSinkTaskContext taskContext;
+    private SinkConnector connector;
+    private SinkTask task;
+
+    private int batchSize;
+    private long lingerMs;
+    private final ScheduledExecutorService scheduledExecutor =
+            Executors.newSingleThreadScheduledExecutor(new 
ThreadFactoryBuilder()
+                    .setNameFormat("pulsar-io-kafka-adaptor-sink-flush-%d")
+                    .build());
+    private final ConcurrentLinkedDeque<Record<GenericObject>> 
pendingFlushQueue = new ConcurrentLinkedDeque<>();
+    private volatile boolean isRunning = false;
+
+    private final Properties props = new Properties();
+    private PulsarKafkaConnectSinkConfig kafkaSinkConfig;
+
+    protected String topicName;
+
+    @Override
+    public void write(Record<GenericObject> sourceRecord) {
+        if (log.isDebugEnabled()) {
+            log.debug("Record sending to kafka, record={}.", sourceRecord);
+        }
+
+        if (!isRunning) {
+            log.error("Sink is stopped. Cannot send the record {}", 
sourceRecord);

Review comment:
       Do we need this error message? I fail it can be`info` or `warning`.

##########
File path: 
pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/PulsarKafkaSinkTaskContext.java
##########
@@ -0,0 +1,212 @@
+/**
+ * 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.io.kafka.connect;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.connect.sink.SinkTaskContext;
+import org.apache.kafka.connect.storage.OffsetBackingStore;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static 
org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig.TOPIC_NAMESPACE_CONFIG;
+
+@Slf4j
+public class PulsarKafkaSinkTaskContext implements SinkTaskContext {
+
+    private final Map<String, String> config;
+
+    private final OffsetBackingStore offsetStore;
+    private final String topicNamespace;
+    private final Consumer<Collection<TopicPartition>> onPartitionChange;
+    private final AtomicBoolean runRepartition = new AtomicBoolean(false);
+
+    private final ConcurrentHashMap<TopicPartition, AtomicLong> currentOffsets 
= new ConcurrentHashMap<>();

Review comment:
       We shouldn't increment the sequence id here. Because the input here is a 
Pulsar topic which should *always* have a sequence id. See comment in 
KafkaConnectSink class.

##########
File path: 
pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/KafkaConnectSink.java
##########
@@ -0,0 +1,277 @@
+/**
+ * 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.io.kafka.connect;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.TimestampType;
+import org.apache.kafka.connect.connector.Task;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.sink.SinkConnector;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.pulsar.functions.api.Record;
+import org.apache.pulsar.io.core.KeyValue;
+import org.apache.pulsar.io.core.Sink;
+import org.apache.pulsar.io.core.SinkContext;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static 
org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig.OFFSET_STORAGE_TOPIC_CONFIG;
+import static 
org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig.PULSAR_SERVICE_URL_CONFIG;
+
+@Slf4j
+public class KafkaConnectSink implements Sink<Object> {
+
+    private boolean unwrapKeyValueIfAvailable;
+
+    private final static ImmutableMap<Class<?>, Schema> primitiveTypeToSchema;
+    static {
+        primitiveTypeToSchema = ImmutableMap.<Class<?>, Schema>builder()
+                .put(Boolean.class, Schema.BOOLEAN_SCHEMA)
+                .put(Byte.class, Schema.INT8_SCHEMA)
+                .put(Short.class, Schema.INT16_SCHEMA)
+                .put(Integer.class, Schema.INT32_SCHEMA)
+                .put(Long.class, Schema.INT64_SCHEMA)
+                .put(Float.class, Schema.FLOAT32_SCHEMA)
+                .put(Double.class, Schema.FLOAT64_SCHEMA)
+                .put(String.class, Schema.STRING_SCHEMA)
+                .put(byte[].class, Schema.BYTES_SCHEMA)
+                .build();
+    }
+
+    private PulsarKafkaSinkContext sinkContext;
+    private PulsarKafkaSinkTaskContext taskContext;
+    private SinkConnector connector;
+    private SinkTask task;
+
+    private Schema defaultKeySchema;
+    private Schema defaultValueSchema;
+
+
+    private int batchSize;
+    private long lingerMs;
+    private final ScheduledExecutorService scheduledExecutor =
+            Executors.newSingleThreadScheduledExecutor(new 
ThreadFactoryBuilder()
+                    .setNameFormat("pulsar-io-kafka-adaptor-sink-flush-%d")
+                    .build());
+    private final AtomicInteger numPendingRecords = new AtomicInteger(0);
+
+    private volatile CompletableFuture<Void> pendingFlush = new 
CompletableFuture<>();
+    private volatile boolean isRunning = false;
+
+    private Properties props = new Properties();
+    private PulsarKafkaConnectSinkConfig kafkaSinkConfig;
+
+    protected String topicName;
+
+    @Override
+    public void write(Record<Object> sourceRecord) {
+        if (log.isDebugEnabled()) {
+            log.debug("Record sending to kafka, record={}.", sourceRecord);
+        }
+
+        if (!isRunning) {
+            log.error("Sink is stopped. Cannot send the record {}", 
sourceRecord);
+            sourceRecord.fail();
+            return;
+        }
+
+        try {
+            SinkRecord record = toSinkRecord(sourceRecord);
+            task.put(Lists.newArrayList(record));
+        } catch (Exception ex) {
+            log.error("Error sending the record {}", sourceRecord, ex);
+            sourceRecord.fail();
+            return;
+        }
+        pendingFlush.whenComplete((ignore, ex) -> {
+            if (ex == null) {
+                sourceRecord.ack();
+            } else {
+                log.error("Error sending the record {}", sourceRecord, ex);
+                sourceRecord.fail();
+            }
+            throw new IllegalArgumentException();
+        });
+        numPendingRecords.incrementAndGet();
+        flushIfNeeded(false);
+    }
+
+    @Override
+    public void close() throws Exception {
+        isRunning = false;
+        flushIfNeeded(true);
+        scheduledExecutor.shutdown();
+        if (!scheduledExecutor.awaitTermination(10 * lingerMs, 
TimeUnit.MILLISECONDS)) {
+            log.error("scheduledExecutor did not terminate in {} ms", 10 * 
lingerMs);
+        }
+
+        task.stop();
+        connector.stop();
+        taskContext.close();
+
+        log.info("Kafka sink stopped.");
+    }
+
+    @Override
+    public void open(Map<String, Object> config, SinkContext ctx) throws 
Exception {

Review comment:
       KafkaBytesSink is different because it doesn't wrap any existing Kafka 
Connect Sink. Kafka Connect Sink implementation assumes data coming from Kafka 
topics which data are consumed in partition order. Hence we need to make sure 
we are receiving messages using a Failover subscription which is consistent 
with Kafka's consumption model. This is important to make things correct. 




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

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to