You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2021/06/18 21:22:02 UTC

[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #7026: Add Apache Pulsar low level and high level connector

mcvsubbu commented on a change in pull request #7026:
URL: https://github.com/apache/incubator-pinot/pull/7026#discussion_r653794441



##########
File path: pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConsumer.java
##########
@@ -0,0 +1,119 @@
+/**
+ * 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.pinot.plugin.stream.pulsar;
+
+import com.google.common.collect.Iterables;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.pinot.spi.stream.MessageBatch;
+import org.apache.pinot.spi.stream.PartitionGroupConsumer;
+import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus;
+import org.apache.pinot.spi.stream.StreamConfig;
+import org.apache.pinot.spi.stream.StreamPartitionMsgOffset;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link PartitionGroupConsumer} implementation for the Pulsar stream
+ */
+public class PulsarPartitionLevelConsumer extends PulsarPartitionLevelConnectionHandler implements PartitionGroupConsumer {
+  private static final Logger LOGGER = LoggerFactory.getLogger(PulsarPartitionLevelConsumer.class);
+  private final ExecutorService _executorService;
+
+  public PulsarPartitionLevelConsumer(String clientId, StreamConfig streamConfig,
+      PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) {
+    super(clientId, streamConfig, partitionGroupConsumptionStatus.getPartitionGroupId());
+    _executorService = Executors.newSingleThreadExecutor();
+  }
+
+  /**
+   * Fetch records from the Pulsar stream between the start and end KinesisCheckpoint
+   * Used {@link org.apache.pulsar.client.api.Reader} to read the messaged from pulsar partitioned topic
+   * The reader seeks to the startMsgOffset and starts reading records in a loop until endMsgOffset or timeout is reached.
+   */
+  @Override
+  public MessageBatch fetchMessages(StreamPartitionMsgOffset startMsgOffset, StreamPartitionMsgOffset endMsgOffset,
+      int timeoutMillis) {
+    final MessageId startMessageId = ((MessageIdStreamOffset) startMsgOffset).getMessageId();
+    final MessageId endMessageId =
+        endMsgOffset == null ? MessageId.latest : ((MessageIdStreamOffset) endMsgOffset).getMessageId();
+
+    List<Message<byte[]>> messagesList = new ArrayList<>();
+    Future<PulsarMessageBatch> pulsarResultFuture =
+        _executorService.submit(() -> fetchMessages(startMessageId, endMessageId, messagesList));
+
+    try {
+      return pulsarResultFuture.get(timeoutMillis, TimeUnit.MILLISECONDS);
+    } catch (TimeoutException e) {
+      // The fetchMessages has thrown an exception. Most common cause is the timeout.
+      // We return the records fetched till now along with the next start offset.
+      return new PulsarMessageBatch(buildOffsetFilteringIterable(messagesList, startMessageId, endMessageId));

Review comment:
       Should we not cancel the job submitted?

##########
File path: pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/MessageIdStreamOffset.java
##########
@@ -0,0 +1,72 @@
+/**
+ * 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.pinot.plugin.stream.pulsar;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import org.apache.pinot.spi.stream.StreamPartitionMsgOffset;
+import org.apache.pulsar.client.api.MessageId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * {@link StreamPartitionMsgOffset} implementation for Pulsar {@link MessageId}
+ */
+public class MessageIdStreamOffset implements StreamPartitionMsgOffset {
+  private Logger LOGGER = LoggerFactory.getLogger(MessageIdStreamOffset.class);
+  private MessageId _messageId;
+
+  public MessageIdStreamOffset(MessageId messageId) {
+    _messageId = messageId;
+  }
+
+  /**
+   * returns the class object from string message id in the format ledgerId:entryId:partitionId
+   * throws {@link IOException} if message if format is invalid.
+   * @param messageId
+   */
+  public MessageIdStreamOffset(String messageId) {
+    try {
+      _messageId = MessageId.fromByteArray(messageId.getBytes(StandardCharsets.UTF_8));
+    } catch (IOException e) {
+      LOGGER.warn("Cannot parse message id " + messageId, e);

Review comment:
       Why swallow the exception? This object will be unusable if `_messageId` is null, right?

##########
File path: pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfig.java
##########
@@ -0,0 +1,102 @@
+/**
+ * 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.pinot.plugin.stream.pulsar;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import org.apache.pinot.spi.stream.StreamConfig;
+import org.apache.pinot.spi.stream.StreamConfigProperties;
+import org.apache.pulsar.client.api.MessageId;
+
+public class PulsarConfig {
+  public static final String STREAM_TYPE = "pulsar";
+  public static final String PULSAR_PROP_PREFIX = "consumer.prop";
+  public static final String BOOTSTRAP_SERVERS = "bootstrap.servers";
+  public static final String START_POSITION = "start_position";
+  public static final String DEFAULT_BOOTSTRAP_BROKERS = "pulsar://localhost:6650";
+
+  private String _pulsarTopicName;
+  private String _subscriberId;
+  private String _bootstrapServers;
+  private MessageId _initialMessageId = MessageId.latest;
+  private Map<String, String> _pulsarConsumerProperties;
+
+  public PulsarConfig(StreamConfig streamConfig, String subscriberId){
+    Map<String, String> streamConfigMap = streamConfig.getStreamConfigsMap();
+    _pulsarTopicName = streamConfig.getTopicName();
+    _bootstrapServers = streamConfigMap.getOrDefault(BOOTSTRAP_SERVERS, DEFAULT_BOOTSTRAP_BROKERS);
+
+    Preconditions.checkNotNull(_bootstrapServers,
+        "Must specify bootstrap broker connect string " + BOOTSTRAP_SERVERS + " in high level pulsar consumer");
+    _subscriberId = subscriberId;
+
+    String startPositionProperty = StreamConfigProperties.constructStreamProperty(STREAM_TYPE, START_POSITION);
+    String startPosition = streamConfigMap.getOrDefault(startPositionProperty, "latest");

Review comment:
       Please use `OffsetCriteria`, already defined in open source.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org