AmatyaAvadhanula commented on code in PR #14137: URL: https://github.com/apache/druid/pull/14137#discussion_r1447017652
########## extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java: ########## @@ -43,15 +43,8 @@ public class KinesisIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningC static final int ASSUMED_RECORD_SIZE = 10_000; static final int ASSUMED_RECORD_SIZE_AGGREGATE = 1_000_000; - /** Review Comment: Could you please restore these javadocs? ########## extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamRecordSupplier.java: ########## @@ -0,0 +1,481 @@ +/* + * 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.druid.indexing.rabbitstream; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Queues; +import com.rabbitmq.stream.Consumer; +import com.rabbitmq.stream.ConsumerBuilder; +import com.rabbitmq.stream.Environment; +import com.rabbitmq.stream.EnvironmentBuilder; +import com.rabbitmq.stream.Message; +import com.rabbitmq.stream.MessageHandler; +import com.rabbitmq.stream.OffsetSpecification; +import com.rabbitmq.stream.impl.Client; +import com.rabbitmq.stream.impl.Client.ClientParameters; +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.indexing.rabbitstream.supervisor.RabbitStreamSupervisorIOConfig; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.metadata.DynamicConfigProvider; + +import javax.annotation.Nonnull; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class RabbitStreamRecordSupplier implements RecordSupplier<String, Long, ByteEntity>, MessageHandler +{ + + private static final EmittingLogger log = new EmittingLogger(RabbitStreamRecordSupplier.class); + private Environment env; + private Map<String, ConsumerBuilder> streamBuilders; + private boolean closed; + private BlockingQueue<OrderedPartitionableRecord<String, Long, ByteEntity>> queue; + private String superStream; + private String uri; + private ObjectMapper mapper; + + private final int recordBufferOfferTimeout; + private final int maxRecordsPerPoll; + private final int recordBufferSize; + + public final Map<String, OffsetSpecification> offsetMap; + + List<Consumer> consumers; Review Comment: Why is this not private? ########## extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamRecordSupplier.java: ########## @@ -0,0 +1,481 @@ +/* + * 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.druid.indexing.rabbitstream; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Queues; +import com.rabbitmq.stream.Consumer; +import com.rabbitmq.stream.ConsumerBuilder; +import com.rabbitmq.stream.Environment; +import com.rabbitmq.stream.EnvironmentBuilder; +import com.rabbitmq.stream.Message; +import com.rabbitmq.stream.MessageHandler; +import com.rabbitmq.stream.OffsetSpecification; +import com.rabbitmq.stream.impl.Client; +import com.rabbitmq.stream.impl.Client.ClientParameters; +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.indexing.rabbitstream.supervisor.RabbitStreamSupervisorIOConfig; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.metadata.DynamicConfigProvider; + +import javax.annotation.Nonnull; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class RabbitStreamRecordSupplier implements RecordSupplier<String, Long, ByteEntity>, MessageHandler +{ + + private static final EmittingLogger log = new EmittingLogger(RabbitStreamRecordSupplier.class); + private Environment env; + private Map<String, ConsumerBuilder> streamBuilders; + private boolean closed; + private BlockingQueue<OrderedPartitionableRecord<String, Long, ByteEntity>> queue; + private String superStream; + private String uri; + private ObjectMapper mapper; + + private final int recordBufferOfferTimeout; + private final int maxRecordsPerPoll; + private final int recordBufferSize; + + public final Map<String, OffsetSpecification> offsetMap; Review Comment: Need not be public. ########## extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java: ########## @@ -43,15 +43,8 @@ public class KinesisIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningC static final int ASSUMED_RECORD_SIZE = 10_000; static final int ASSUMED_RECORD_SIZE_AGGREGATE = 1_000_000; - /** - * Together with {@link KinesisIndexTaskIOConfig#MAX_RECORD_FETCH_MEMORY}, don't take up more than 200MB per task. - */ private static final int MAX_RECORD_BUFFER_MEMORY = 100_000_000; - /** Review Comment: These javadocs may have been deleted accidentally as well. ########## extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamRecordSupplier.java: ########## @@ -0,0 +1,481 @@ +/* + * 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.druid.indexing.rabbitstream; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Queues; +import com.rabbitmq.stream.Consumer; +import com.rabbitmq.stream.ConsumerBuilder; +import com.rabbitmq.stream.Environment; +import com.rabbitmq.stream.EnvironmentBuilder; +import com.rabbitmq.stream.Message; +import com.rabbitmq.stream.MessageHandler; +import com.rabbitmq.stream.OffsetSpecification; +import com.rabbitmq.stream.impl.Client; +import com.rabbitmq.stream.impl.Client.ClientParameters; +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.indexing.rabbitstream.supervisor.RabbitStreamSupervisorIOConfig; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.metadata.DynamicConfigProvider; + +import javax.annotation.Nonnull; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class RabbitStreamRecordSupplier implements RecordSupplier<String, Long, ByteEntity>, MessageHandler +{ + + private static final EmittingLogger log = new EmittingLogger(RabbitStreamRecordSupplier.class); + private Environment env; + private Map<String, ConsumerBuilder> streamBuilders; + private boolean closed; + private BlockingQueue<OrderedPartitionableRecord<String, Long, ByteEntity>> queue; + private String superStream; + private String uri; + private ObjectMapper mapper; + + private final int recordBufferOfferTimeout; + private final int maxRecordsPerPoll; + private final int recordBufferSize; + + public final Map<String, OffsetSpecification> offsetMap; + + List<Consumer> consumers; + private boolean isRunning; + private Semaphore stateSemaphore; + + private String password; + private String username; + + public RabbitStreamRecordSupplier( + Map<String, Object> consumerProperties, + ObjectMapper mapper, + String uri, + int recordBufferSize, + int recordBufferOfferTimeout, + int maxRecordsPerPoll + ) + { + this.uri = uri; + this.mapper = mapper; + + this.recordBufferSize = recordBufferSize; + this.maxRecordsPerPoll = maxRecordsPerPoll; + + this.recordBufferOfferTimeout = recordBufferOfferTimeout; + + // Messages will be added to this queue from multiple threads + queue = new LinkedBlockingQueue<>(recordBufferSize); + + offsetMap = new ConcurrentHashMap<>(); + streamBuilders = new ConcurrentHashMap<>(); + + // stateSemaphore protects isRunning and consumers + stateSemaphore = new Semaphore(1, true); + isRunning = false; + consumers = new ArrayList<Consumer>(); + + this.password = null; + this.username = null; + this.env = null; + + if (consumerProperties != null) { + + // Additional DynamicConfigProvider based extensible support for all consumer + // properties + Object dynamicConfigProviderJson = consumerProperties + .get(RabbitStreamSupervisorIOConfig.DRUID_DYNAMIC_CONFIG_PROVIDER_KEY); + if (dynamicConfigProviderJson != null) { + DynamicConfigProvider dynamicConfigProvider = this.mapper.convertValue(dynamicConfigProviderJson, + DynamicConfigProvider.class); + Map<String, String> dynamicConfig = dynamicConfigProvider.getConfig(); + for (Map.Entry<String, String> e : dynamicConfig.entrySet()) { + if (e.getKey().equals("password")) { + this.password = e.getValue(); + } + if (e.getKey().equals("username")) { + this.username = e.getValue(); + } + } + } + } + } + + public void startBackgroundFetch() + { + try { + // aquire uninteruptibly to prevent state corruption issues + // on consumers and isRunning + stateSemaphore.acquireUninterruptibly(); + if (this.isRunning != false) { + return; + } + for (Map.Entry<String, ConsumerBuilder> entry : streamBuilders.entrySet()) { + consumers.add( + entry.getValue().offset( + offsetMap.get(entry.getKey())).build()); + } + this.isRunning = true; + } + finally { + stateSemaphore.release(); + } + } + + @VisibleForTesting + public int bufferSize() + { + return queue.size(); + } + + @VisibleForTesting + public boolean isRunning() + { + return this.isRunning; + } + + @VisibleForTesting + public OffsetSpecification getOffset(StreamPartition<String> partition) + { + return this.offsetMap.get(partition.getPartitionId()); + } + + public void stopBackgroundFetch() + { + try { + stateSemaphore.acquire(); + try { + if (this.isRunning != true) { + return; + } + for (Consumer consumer : consumers) { + consumer.close(); + } + this.consumers.clear(); + this.isRunning = false; + } + finally { + stateSemaphore.release(); + } + } + catch (InterruptedException exc) { + } + + } + + public EnvironmentBuilder getEnvBuilder() + { + return Environment.builder(); + } + + public Environment getRabbitEnvironment() + { + if (this.env != null) { + return this.env; + } + + EnvironmentBuilder envBuilder = this.getEnvBuilder().uri(this.uri); + + if (this.password != null) { + envBuilder = envBuilder.password(this.password); + } + if (this.username != null) { + envBuilder = envBuilder.username(this.username); + } + + this.env = envBuilder.build(); + return this.env; + } + + public static String getStreamFromSubstream(String partionID) + { + String[] res = partionID.split("-"); + res = Arrays.copyOf(res, res.length - 1); + return String.join("-", res); + } + + private void removeOldAssignments(Set<StreamPartition<String>> streamPartitionstoKeep) + { + Iterator<Map.Entry<String, ConsumerBuilder>> streamBuilderIterator = streamBuilders.entrySet().iterator(); + while (streamBuilderIterator.hasNext()) { + Map.Entry<String, ConsumerBuilder> entry = streamBuilderIterator.next(); + StreamPartition<String> comparitor = new StreamPartition<String>(getStreamFromSubstream(entry.getKey()), entry.getKey()); + if (!streamPartitionstoKeep.contains(comparitor)) { + streamBuilderIterator.remove(); + } + } + + Iterator<Map.Entry<String, OffsetSpecification>> offsetItterator = offsetMap.entrySet().iterator(); Review Comment: typo: offsetIterator ########## extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamRecordSupplier.java: ########## @@ -0,0 +1,481 @@ +/* + * 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.druid.indexing.rabbitstream; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Queues; +import com.rabbitmq.stream.Consumer; +import com.rabbitmq.stream.ConsumerBuilder; +import com.rabbitmq.stream.Environment; +import com.rabbitmq.stream.EnvironmentBuilder; +import com.rabbitmq.stream.Message; +import com.rabbitmq.stream.MessageHandler; +import com.rabbitmq.stream.OffsetSpecification; +import com.rabbitmq.stream.impl.Client; +import com.rabbitmq.stream.impl.Client.ClientParameters; +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.indexing.rabbitstream.supervisor.RabbitStreamSupervisorIOConfig; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.metadata.DynamicConfigProvider; + +import javax.annotation.Nonnull; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class RabbitStreamRecordSupplier implements RecordSupplier<String, Long, ByteEntity>, MessageHandler +{ + + private static final EmittingLogger log = new EmittingLogger(RabbitStreamRecordSupplier.class); + private Environment env; + private Map<String, ConsumerBuilder> streamBuilders; + private boolean closed; + private BlockingQueue<OrderedPartitionableRecord<String, Long, ByteEntity>> queue; + private String superStream; + private String uri; + private ObjectMapper mapper; + + private final int recordBufferOfferTimeout; + private final int maxRecordsPerPoll; + private final int recordBufferSize; + + public final Map<String, OffsetSpecification> offsetMap; + + List<Consumer> consumers; + private boolean isRunning; + private Semaphore stateSemaphore; + + private String password; + private String username; + + public RabbitStreamRecordSupplier( + Map<String, Object> consumerProperties, + ObjectMapper mapper, + String uri, + int recordBufferSize, + int recordBufferOfferTimeout, + int maxRecordsPerPoll + ) + { + this.uri = uri; + this.mapper = mapper; + + this.recordBufferSize = recordBufferSize; + this.maxRecordsPerPoll = maxRecordsPerPoll; + + this.recordBufferOfferTimeout = recordBufferOfferTimeout; + + // Messages will be added to this queue from multiple threads + queue = new LinkedBlockingQueue<>(recordBufferSize); + + offsetMap = new ConcurrentHashMap<>(); + streamBuilders = new ConcurrentHashMap<>(); + + // stateSemaphore protects isRunning and consumers + stateSemaphore = new Semaphore(1, true); + isRunning = false; + consumers = new ArrayList<Consumer>(); + + this.password = null; + this.username = null; + this.env = null; + + if (consumerProperties != null) { + + // Additional DynamicConfigProvider based extensible support for all consumer + // properties + Object dynamicConfigProviderJson = consumerProperties + .get(RabbitStreamSupervisorIOConfig.DRUID_DYNAMIC_CONFIG_PROVIDER_KEY); + if (dynamicConfigProviderJson != null) { + DynamicConfigProvider dynamicConfigProvider = this.mapper.convertValue(dynamicConfigProviderJson, + DynamicConfigProvider.class); + Map<String, String> dynamicConfig = dynamicConfigProvider.getConfig(); + for (Map.Entry<String, String> e : dynamicConfig.entrySet()) { + if (e.getKey().equals("password")) { + this.password = e.getValue(); + } + if (e.getKey().equals("username")) { + this.username = e.getValue(); + } + } + } + } + } + + public void startBackgroundFetch() Review Comment: Need not be public -- 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. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
