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 2021/04/20 16:24:43 UTC

[GitHub] [pulsar] eolivelli commented on a change in pull request #9927: Add ability to use Kafka's sinks as pulsar sinks

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



##########
File path: pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/KafkaConnectSink.java
##########
@@ -0,0 +1,321 @@
+/**
+ * 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.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.KVRecord;
+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 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().entrySet()
+                .forEach(kv -> props.put(kv.getKey(), kv.getValue()));
+
+        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);
+        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);

Review comment:
       can we validate that `configs` size is 1 ?

##########
File path: pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/KafkaConnectSink.java
##########
@@ -0,0 +1,321 @@
+/**
+ * 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.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.KVRecord;
+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 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().entrySet()
+                .forEach(kv -> props.put(kv.getKey(), kv.getValue()));
+
+        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);
+        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
+     * @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
+                && 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 ? "null" : pulsarSchema.getSchemaInfo().toString())

Review comment:
       getSchemaInfo() may return null

##########
File path: pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/PulsarKafkaSinkTaskContext.java
##########
@@ -0,0 +1,214 @@
+/**
+ * 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 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.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 (Exception e) {
+                log.error("error getting initial state of " + topicPartition.toString(), e);
+                return new AtomicLong(-1L);
+            }
+        });
+        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) {
+        // noop
+    }
+
+    @Override
+    public Set<TopicPartition> assignment() {
+        return currentOffsets.keySet();
+    }
+
+    @Override
+    public void pause(TopicPartition... topicPartitions) {
+        // noop
+    }
+
+    @Override
+    public void resume(TopicPartition... topicPartitions) {
+        // noop
+    }
+
+    @Override
+    public void requestCommit() {
+        // noop
+    }
+
+    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", ex);

Review comment:
       can we log more information, like the offsets we wanted to commit ?

##########
File path: pulsar-io/kafka-connect-adaptor/src/test/java/org/apache/pulsar/io/kafka/connect/SchemaedFileStreamSinkTask.java
##########
@@ -0,0 +1,78 @@
+/**
+ * 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.google.common.collect.Lists;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.file.FileStreamSinkTask;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.testng.collections.Maps;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+@SuppressWarnings({"unchecked", "rawtypes"})
+public class SchemaedFileStreamSinkTask extends FileStreamSinkTask {
+
+    @Override
+    public void put(Collection<SinkRecord> sinkRecords) {
+
+        List<SinkRecord> out = Lists.newLinkedList();
+
+        Iterator var2 = sinkRecords.iterator();
+
+        while (var2.hasNext()) {
+            SinkRecord record = (SinkRecord) var2.next();
+
+            Object val = record.valueSchema() == Schema.BYTES_SCHEMA
+                    ? new String((byte[])record.value(), StandardCharsets.US_ASCII)
+                    : record.value();
+
+            Map<String, Object> recOut = Maps.newHashMap();
+            recOut.put("keySchema", record.keySchema().type().toString());
+            recOut.put("valueSchema", record.valueSchema().type().toString());
+            recOut.put("key", record.key());
+            recOut.put("value", val);
+
+            ObjectMapper om = new ObjectMapper();
+            try {
+                SinkRecord toSink = new SinkRecord(record.topic(),

Review comment:
       why are you cloning the SinkRecord ?

##########
File path: pulsar-io/kafka-connect-adaptor/src/test/java/org/apache/pulsar/io/kafka/connect/SchemaedFileStreamSinkTask.java
##########
@@ -0,0 +1,78 @@
+/**
+ * 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.google.common.collect.Lists;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.file.FileStreamSinkTask;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.testng.collections.Maps;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+@SuppressWarnings({"unchecked", "rawtypes"})
+public class SchemaedFileStreamSinkTask extends FileStreamSinkTask {

Review comment:
       can we write a line of JavaDoc in order to explain what it is this 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:
       @sijie can we do this in a follow up work ?

##########
File path: pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/PulsarKafkaSinkContext.java
##########
@@ -0,0 +1,37 @@
+/**
+ * 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) {
+        throw new UnsupportedOperationException("not implemented");

Review comment:
       can we at least log the exception (with stacktrace) ?

##########
File path: pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/PulsarKafkaConnectSinkConfig.java
##########
@@ -0,0 +1,90 @@
+/**
+ * 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 = "Number of messages the sink processes before flush.")
+    private int batchSize = 1;
+
+    @FieldDoc(
+            defaultValue = "2147483647L",
+            help = "The batch size that Kafka producer will attempt to batch records together.")
+    private long lingerTimeMs = 2147483647L;
+
+    @FieldDoc(
+            defaultValue = "pulsar-io-adaptor-topic",
+            help = "The Kafka topic name that passed to kafka sink.")
+    private String topic = "pulsar-io-adaptor-topic";
+
+    @FieldDoc(
+            required = true,
+            defaultValue = "",
+            help = "A kafka-connector sink class to use.")
+    private String kafkaConnectorSinkClass;
+
+    @FieldDoc(
+            defaultValue = "",
+            help = "Config properties to pass to the kafka connector.")
+    private Map<String, String> kafkaConnectorConfigProperties;
+
+    @FieldDoc(
+            defaultValue = "kafka-adaptor-sink-offsets",
+            help = "Pulsar topic to store offsets at.")
+    private String offsetStorageTopic = "kafka-adaptor-sink-offsets";
+
+    @FieldDoc(
+            required = true,
+            defaultValue = "",
+            help = "Pulsar service URL.")
+    private String pulsarServiceUrl;

Review comment:
       why do we need this value ? we are running inside Pulsar

##########
File path: pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/PulsarKafkaConnectSinkConfig.java
##########
@@ -0,0 +1,90 @@
+/**
+ * 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 = "Number of messages the sink processes before flush.")
+    private int batchSize = 1;
+
+    @FieldDoc(
+            defaultValue = "2147483647L",
+            help = "The batch size that Kafka producer will attempt to batch records together.")
+    private long lingerTimeMs = 2147483647L;
+
+    @FieldDoc(
+            defaultValue = "pulsar-io-adaptor-topic",
+            help = "The Kafka topic name that passed to kafka sink.")
+    private String topic = "pulsar-io-adaptor-topic";

Review comment:
       can we make this default an empty string and make it required (fail if it is not configured) ?

##########
File path: pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/PulsarKafkaConnectSinkConfig.java
##########
@@ -0,0 +1,90 @@
+/**
+ * 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 = "Number of messages the sink processes before flush.")
+    private int batchSize = 1;
+
+    @FieldDoc(
+            defaultValue = "2147483647L",
+            help = "The batch size that Kafka producer will attempt to batch records together.")
+    private long lingerTimeMs = 2147483647L;
+
+    @FieldDoc(
+            defaultValue = "pulsar-io-adaptor-topic",
+            help = "The Kafka topic name that passed to kafka sink.")
+    private String topic = "pulsar-io-adaptor-topic";
+
+    @FieldDoc(
+            required = true,
+            defaultValue = "",
+            help = "A kafka-connector sink class to use.")
+    private String kafkaConnectorSinkClass;
+
+    @FieldDoc(
+            defaultValue = "",
+            help = "Config properties to pass to the kafka connector.")
+    private Map<String, String> kafkaConnectorConfigProperties;
+
+    @FieldDoc(
+            defaultValue = "kafka-adaptor-sink-offsets",
+            help = "Pulsar topic to store offsets at.")
+    private String offsetStorageTopic = "kafka-adaptor-sink-offsets";

Review comment:
       can we make this default an empty string and make it required (fail if it is not configured) ?
   
   or if we do not make this required, can we leave it blank and compute a name based on the `topic` configuration parameter ?

##########
File path: pulsar-io/kafka-connect-adaptor/src/test/java/org/apache/pulsar/io/kafka/connect/SchemaedFileStreamSinkTask.java
##########
@@ -0,0 +1,78 @@
+/**
+ * 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.google.common.collect.Lists;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.file.FileStreamSinkTask;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.testng.collections.Maps;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+@SuppressWarnings({"unchecked", "rawtypes"})
+public class SchemaedFileStreamSinkTask extends FileStreamSinkTask {
+
+    @Override
+    public void put(Collection<SinkRecord> sinkRecords) {
+
+        List<SinkRecord> out = Lists.newLinkedList();
+
+        Iterator var2 = sinkRecords.iterator();

Review comment:
       <SinkRecord>

##########
File path: pulsar-io/kafka-connect-adaptor/src/test/java/org/apache/pulsar/io/kafka/connect/GenericObjectWrapper.java
##########
@@ -0,0 +1,51 @@
+/**
+ * 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.Builder;
+import lombok.Data;
+import org.apache.pulsar.client.api.schema.Field;
+import org.apache.pulsar.client.api.schema.GenericRecord;
+import org.apache.pulsar.common.schema.SchemaType;
+
+import java.util.List;
+
+/**
+ * org.apache.pulsar.client.impl.schema.GenericObjectWrapper is not public
+ * (and is not intended to be, at least for now).
+ * this one implements similar functionality, for testing.
+ */
+@Data
+@Builder
+class GenericObjectWrapper implements GenericRecord {

Review comment:
       what about naming this MockGenericObjectWrapper ? (otherwise in the IDE everytime someone looks for GenericObjectWrapper he will find this class as well)

##########
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<>();
+
+    public PulsarKafkaSinkTaskContext(Map<String, String> config,
+                                      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 (Exception e) {
+                log.error("error getting initial state of " + topicPartition.toString(), e);
+                return new AtomicLong(-1L);
+            }
+        });
+        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) {

Review comment:
       I would create an issue for further work
   we could log a line at "error" level, this way if a Connector uses this feature it will be logged and the user will probably see it

##########
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<>();
+
+    public PulsarKafkaSinkTaskContext(Map<String, String> config,

Review comment:
        makes sense to me, although probably it is not needed, at least for this patch




-- 
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:
users@infra.apache.org