snleee commented on a change in pull request #7026: URL: https://github.com/apache/incubator-pinot/pull/7026#discussion_r647769770
########## File path: pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfig.java ########## @@ -0,0 +1,98 @@ +/** + * 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.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); + _subscriberId = subscriberId; + + String startPositionProperty = StreamConfigProperties.constructStreamProperty(STREAM_TYPE, START_POSITION); + String startPosition = streamConfigMap.getOrDefault(startPositionProperty, "latest"); + if(startPosition.equals("earliest")){ Review comment: use constant variables `e.g. private static final String LATEST` for `latest` & `earliest` string. Also, let's make it case insensitive to make it more robust. ########## File path: pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml ########## @@ -0,0 +1,55 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + + 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. + +--> +<project xmlns="http://maven.apache.org/POM/4.0.0" + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <parent> + <artifactId>pinot-stream-ingestion</artifactId> + <groupId>org.apache.pinot</groupId> + <version>0.8.0-SNAPSHOT</version> + <relativePath>..</relativePath> + </parent> + <modelVersion>4.0.0</modelVersion> + + <name>Pinot Pulsar</name> + <url>https://pinot.apache.org/</url> + <artifactId>pinot-pulsar</artifactId> + + <properties> + <pinot.root>${basedir}/../../..</pinot.root> + <pulsar.version>2.7.2</pulsar.version> + </properties> + + <dependencies> + <dependency> + <groupId>org.apache.pulsar</groupId> + <artifactId>pulsar-client</artifactId> + <version>${pulsar.version}</version> + </dependency> + <dependency> + <groupId>org.apache.pulsar</groupId> + <artifactId>pulsar-client-admin</artifactId> + <version>${pulsar.version}</version> + </dependency> + </dependencies> + +</project> Review comment: add a line ########## File path: pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfig.java ########## @@ -0,0 +1,98 @@ +/** + * 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.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); + _subscriberId = subscriberId; + + String startPositionProperty = StreamConfigProperties.constructStreamProperty(STREAM_TYPE, START_POSITION); + String startPosition = streamConfigMap.getOrDefault(startPositionProperty, "latest"); + if(startPosition.equals("earliest")){ + _initialMessageId = MessageId.earliest; + } else if(startPosition.equals("latest")) { + _initialMessageId = MessageId.latest; + } else { + try { + _initialMessageId = MessageId.fromByteArray(startPosition.getBytes()); + } catch (IOException e){ + throw new RuntimeException("Invalid start position found: " + startPosition); + } + } + + _pulsarConsumerProperties = new HashMap<>(); + + String pulsarConsumerPropertyPrefix = + StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PULSAR_PROP_PREFIX); + for (String key : streamConfigMap.keySet()) { + if (key.startsWith(pulsarConsumerPropertyPrefix)) { + _pulsarConsumerProperties + .put(StreamConfigProperties.getPropertySuffix(key, pulsarConsumerPropertyPrefix), streamConfigMap.get(key)); + } + } + } + + public String getPulsarTopicName() { + return _pulsarTopicName; + } + + public String getSubscriberId() { + return _subscriberId; + } + + public String getBootstrapServers() { + return _bootstrapServers; + } + + public Properties getPulsarConsumerProperties() { + Properties props = new Properties(); + for (String key : _pulsarConsumerProperties.keySet()) { + props.put(key, _pulsarConsumerProperties.get(key)); + } + Review comment: remove line ########## File path: pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfig.java ########## @@ -0,0 +1,98 @@ +/** + * 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.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); Review comment: I think that we should enforce `BOOTSTRAP_SERVERS` to be a required config instead of giving a default value of `localhost:6650`. ########## File path: pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarMessageBatch.java ########## @@ -0,0 +1,73 @@ +/** + * 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.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import org.apache.pinot.spi.stream.MessageBatch; +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.impl.MessageIdImpl; +import org.apache.pulsar.client.internal.DefaultImplementation; + + +public class PulsarMessageBatch implements MessageBatch<byte[]> { + + private List<Message<byte[]>> messageList = new ArrayList<>(); Review comment: Let's add `final` ``` private final List<Message<byte[]>> messageList; ``` ########## File path: pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConnectionHandler.java ########## @@ -0,0 +1,77 @@ +/** + * 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.annotations.VisibleForTesting; +import java.io.IOException; +import java.util.List; +import org.apache.pinot.spi.stream.StreamConfig; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Reader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class PulsarPartitionLevelConnectionHandler { + private static final Logger LOGGER = LoggerFactory.getLogger(PulsarPartitionLevelConnectionHandler.class); + + public static final String SEPERATOR = "-"; Review comment: Where do we need this? ########## File path: pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConnectionHandler.java ########## @@ -0,0 +1,77 @@ +/** + * 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.annotations.VisibleForTesting; +import java.io.IOException; +import java.util.List; +import org.apache.pinot.spi.stream.StreamConfig; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Reader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class PulsarPartitionLevelConnectionHandler { + private static final Logger LOGGER = LoggerFactory.getLogger(PulsarPartitionLevelConnectionHandler.class); + + public static final String SEPERATOR = "-"; + public static final String TOPIC_PARTITION_NAME_SUFFIX = "partition"; Review comment: Same. no usage. ########## File path: pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConsumer.java ########## @@ -0,0 +1,115 @@ +/** + * 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.PartitionLevelConsumer; +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; + + +public class PulsarPartitionLevelConsumer extends PulsarPartitionLevelConnectionHandler implements PartitionGroupConsumer { + private static final Logger LOGGER = LoggerFactory.getLogger(PulsarPartitionLevelConsumer.class); + private final ExecutorService _executorService; + private final PartitionGroupConsumptionStatus _partitionGroupConsumptionStatus; + + public PulsarPartitionLevelConsumer(String clientId, StreamConfig streamConfig, + PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) { + super(clientId, streamConfig, partitionGroupConsumptionStatus.getPartitionGroupId()); + _partitionGroupConsumptionStatus = partitionGroupConsumptionStatus; + _executorService = Executors.newSingleThreadExecutor(); + } + + @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. Review comment: Let's add a space after `//` ########## File path: pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarStandaloneCluster.java ########## @@ -0,0 +1,115 @@ +/** + * 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.File; +import org.apache.commons.io.FileUtils; + + +public class PulsarStandaloneCluster { + public static final Integer DEFAULT_BROKER_PORT = 6650; + public static final Integer DEFAULT_ADMIN_PORT = 8080; + public static final String DEFAULT_DATA_MOUNT_DIRECTORY = "pulsar-data"; + public static final String DEFAULT_CONF_MOUNT_DIRECTORY = "pulsar-conf"; + public static final String DOCKER_CONTAINER_NAME = "pulsar_standalone_pinot"; + + private Integer _brokerPort; + private Integer _adminPort; + private String _dataMountDirectory; + private String _confMountDirectory; + + private Process _pulsarCluster; + + public static final String DOCKER_COMMAND = Review comment: +1 on not using docker for integration test. Based on some search, I see some relevant info at the following: https://stackoverflow.com/questions/56515083/test-apache-pulsar-functions-in-an-embedded-standalone-environment @KKcorps Can you take a look into the approach using `PulsarStandaloneBuilder` ? ########## File path: pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerFactory.java ########## @@ -0,0 +1,63 @@ +/** + * 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.util.Set; +import org.apache.pinot.spi.stream.PartitionGroupConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; +import org.apache.pinot.spi.stream.PartitionLevelConsumer; +import org.apache.pinot.spi.stream.StreamConsumerFactory; +import org.apache.pinot.spi.stream.StreamLevelConsumer; +import org.apache.pinot.spi.stream.StreamMetadataProvider; +import org.apache.pinot.spi.stream.StreamPartitionMsgOffsetFactory; + + +public class PulsarConsumerFactory extends StreamConsumerFactory { + @Override + public PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int partition) { Review comment: @npawar @KKcorps For abstraction functions that are no longer used, let's put `@deprecated` annotation along with some explanation on which function needs to be implemented and which do not. ########## File path: pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarStreamLevelConsumerManager.java ########## @@ -0,0 +1,171 @@ +/** + * 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.util.concurrent.Uninterruptibles; +import java.util.HashMap; +import java.util.IdentityHashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.TimeUnit; +import org.apache.commons.lang3.tuple.ImmutableTriple; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Reader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class PulsarStreamLevelConsumerManager { + private static final Logger LOGGER = LoggerFactory.getLogger(PulsarStreamLevelConsumerManager.class); + private static final Long IN_USE = -1L; + private static final long CONSUMER_SHUTDOWN_DELAY_MILLIS = TimeUnit.SECONDS.toMillis(60); // One minute + private static final Map<ImmutableTriple<String, String, String>, Reader<byte[]>> CONSUMER_FOR_CONFIG_KEY = + new HashMap<>(); + private static final IdentityHashMap<Reader<byte[]>, Long> CONSUMER_RELEASE_TIME = new IdentityHashMap<>(); + protected static PulsarClient _pulsarClient; + protected static Reader<byte[]> _reader; + + public static Reader<byte[]> acquirePulsarConsumerForConfig(PulsarConfig pulsarStreamLevelStreamConfig) { + final ImmutableTriple<String, String, String> configKey = + new ImmutableTriple<>(pulsarStreamLevelStreamConfig.getPulsarTopicName(), + pulsarStreamLevelStreamConfig.getSubscriberId(), pulsarStreamLevelStreamConfig.getBootstrapServers()); + + synchronized (PulsarStreamLevelConsumerManager.class) { + // If we have the consumer and it's not already acquired, return it, otherwise error out if it's already acquired + if (CONSUMER_FOR_CONFIG_KEY.containsKey(configKey)) { + Reader<byte[]> pulsarConsumer = CONSUMER_FOR_CONFIG_KEY.get(configKey); + if (CONSUMER_RELEASE_TIME.get(pulsarConsumer).equals(IN_USE)) { + throw new RuntimeException("Consumer " + pulsarConsumer + " already in use!"); + } else { + LOGGER.info("Reusing pulsar consumer with id {}", pulsarConsumer); + CONSUMER_RELEASE_TIME.put(pulsarConsumer, IN_USE); + return pulsarConsumer; + } + } + + LOGGER.info("Creating new pulsar consumer and iterator for topic {}", + pulsarStreamLevelStreamConfig.getPulsarTopicName()); + + // Create the consumer + + Properties consumerProp = new Properties(); Review comment: We don't use this local `consumerProp` anywhere. ########## File path: pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConsumer.java ########## @@ -0,0 +1,115 @@ +/** + * 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.PartitionLevelConsumer; +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; + + +public class PulsarPartitionLevelConsumer extends PulsarPartitionLevelConnectionHandler implements PartitionGroupConsumer { + private static final Logger LOGGER = LoggerFactory.getLogger(PulsarPartitionLevelConsumer.class); + private final ExecutorService _executorService; + private final PartitionGroupConsumptionStatus _partitionGroupConsumptionStatus; Review comment: Where do we need this? -- 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: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org