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



##########
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);

Review comment:
       I am not sure we want to return -1 here.
   If we see a temporary error we should fail and let the system recover

##########
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 {
+        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()));
+
+        defaultKeySchema = (Schema)Schema.class

Review comment:
       IMHO we should have some explicit mapping of supported types.
   Using reflection may lead to unexpected results in the future
   




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