From 7f87cd3a9ffd7f73c14502d9457a037926aab75f Mon Sep 17 00:00:00 2001 From: "Xiaotian (Jackie) Jiang" <17555551+Jackie-Jiang@users.noreply.github.com> Date: Sun, 7 Apr 2024 14:19:44 -0700 Subject: [PATCH 01/72] Cleanup the consumer interfaces and legacy code (#12697) --- .../broker/FakeStreamConsumerFactory.java | 97 ------- .../broker/broker/HelixBrokerStarterTest.java | 2 +- .../utils/config/TableConfigUtilsTest.java | 11 +- .../realtime/RealtimeSegmentDataManager.java | 74 +++-- .../FakePartitionLevelConsumer.java | 43 +-- .../fakestream/FakeStreamConsumerFactory.java | 63 +---- .../fakestream/FakeStreamMessageBatch.java | 47 ++-- .../LLCRealtimeClusterIntegrationTest.java | 19 +- .../stream/kafka20/KafkaConsumerFactory.java | 17 +- .../stream/kafka20/KafkaMessageBatch.java | 87 ++---- .../kafka20/KafkaMetadataExtractor.java | 55 ---- .../KafkaPartitionLevelConnectionHandler.java | 2 - .../kafka20/KafkaPartitionLevelConsumer.java | 86 +++--- .../stream/kafka20/KafkaStreamMessage.java | 38 --- .../kafka20/KafkaStreamMessageMetadata.java | 3 +- .../KafkaPartitionLevelConsumerTest.java | 261 ++++++++---------- .../kinesis/KinesisConnectionHandler.java | 148 ++++------ .../stream/kinesis/KinesisConsumer.java | 159 +++++------ .../kinesis/KinesisConsumerFactory.java | 5 - .../stream/kinesis/KinesisMessageBatch.java | 61 ++++ .../kinesis/KinesisMetadataExtractor.java | 45 --- .../kinesis/KinesisMsgOffsetFactory.java | 13 +- .../kinesis/KinesisPartitionGroupOffset.java | 87 ++---- .../stream/kinesis/KinesisRecordsBatch.java | 96 ------- .../kinesis/KinesisStreamMessageMetadata.java | 6 +- .../KinesisStreamMetadataProvider.java | 34 +-- .../stream/kinesis/KinesisConsumerTest.java | 94 ++----- .../kinesis/KinesisMessageBatchTest.java | 84 ++++++ .../kinesis/KinesisRecordsBatchTest.java | 85 ------ .../KinesisStreamMetadataProviderTest.java | 16 +- .../pinot-pulsar/pom.xml | 5 + .../stream/pulsar/MessageIdStreamOffset.java | 21 +- .../pulsar/MessageIdStreamOffsetFactory.java | 8 - .../stream/pulsar/PulsarConsumerFactory.java | 8 +- .../stream/pulsar/PulsarMessageBatch.java | 108 ++------ .../pulsar/PulsarMetadataExtractor.java | 182 ------------ ...PulsarPartitionLevelConnectionHandler.java | 88 ++---- .../pulsar/PulsarPartitionLevelConsumer.java | 90 +++--- .../stream/pulsar/PulsarStreamMessage.java | 47 ---- .../pulsar/PulsarStreamMessageMetadata.java | 71 ++--- .../plugin/stream/pulsar/PulsarUtils.java | 180 +++++++++++- .../stream/pulsar/PulsarConfigTest.java | 86 +++--- .../stream/pulsar/PulsarConsumerTest.java | 146 ++++------ .../stream/pulsar/PulsarMessageBatchTest.java | 79 +++--- ...xtractorTest.java => PulsarUtilsTest.java} | 62 +++-- .../pinot/spi/stream/BytesStreamMessage.java | 20 +- .../pinot/spi/stream/LongMsgOffset.java | 5 - .../spi/stream/LongMsgOffsetFactory.java | 6 +- .../apache/pinot/spi/stream/MessageBatch.java | 137 +++++---- .../spi/stream/PartitionGroupConsumer.java | 24 +- .../spi/stream/PartitionLevelConsumer.java | 8 +- .../apache/pinot/spi/stream/RowMetadata.java | 38 ++- .../spi/stream/StreamConsumerFactory.java | 34 +-- .../spi/stream/StreamDataDecoderImpl.java | 7 +- .../pinot/spi/stream/StreamMessage.java | 47 +++- .../spi/stream/StreamMessageMetadata.java | 86 +++++- .../spi/stream/StreamPartitionMsgOffset.java | 7 +- .../StreamPartitionMsgOffsetFactory.java | 14 +- .../spi/stream/StreamDataDecoderImplTest.java | 18 +- .../pinot/spi/stream/StreamMessageTest.java | 55 ---- 60 files changed, 1358 insertions(+), 2167 deletions(-) delete mode 100644 pinot-broker/src/test/java/org/apache/pinot/broker/broker/FakeStreamConsumerFactory.java delete mode 100644 pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaMetadataExtractor.java delete mode 100644 pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMessage.java create mode 100644 pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMessageBatch.java delete mode 100644 pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMetadataExtractor.java delete mode 100644 pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisRecordsBatch.java create mode 100644 pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisMessageBatchTest.java delete mode 100644 pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisRecordsBatchTest.java delete mode 100644 pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarMetadataExtractor.java delete mode 100644 pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarStreamMessage.java rename pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/{PulsarMetadataExtractorTest.java => PulsarUtilsTest.java} (57%) rename pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMessage.java => pinot-spi/src/main/java/org/apache/pinot/spi/stream/BytesStreamMessage.java (62%) delete mode 100644 pinot-spi/src/test/java/org/apache/pinot/spi/stream/StreamMessageTest.java diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/broker/FakeStreamConsumerFactory.java b/pinot-broker/src/test/java/org/apache/pinot/broker/broker/FakeStreamConsumerFactory.java deleted file mode 100644 index b0fee613222a..000000000000 --- a/pinot-broker/src/test/java/org/apache/pinot/broker/broker/FakeStreamConsumerFactory.java +++ /dev/null @@ -1,97 +0,0 @@ -/** - * 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.broker.broker; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Set; -import java.util.concurrent.TimeoutException; -import org.apache.pinot.spi.stream.LongMsgOffset; -import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.OffsetCriteria; -import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; -import org.apache.pinot.spi.stream.PartitionGroupMetadata; -import org.apache.pinot.spi.stream.PartitionLevelConsumer; -import org.apache.pinot.spi.stream.StreamConfig; -import org.apache.pinot.spi.stream.StreamConsumerFactory; -import org.apache.pinot.spi.stream.StreamMetadataProvider; -import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; - - -public class FakeStreamConsumerFactory extends StreamConsumerFactory { - @Override - public PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int partition) { - return new FakePartitionLevelConsumer(); - } - @Override - public StreamMetadataProvider createPartitionMetadataProvider(String clientId, int partition) { - return new FakesStreamMetadataProvider(); - } - - @Override - public StreamMetadataProvider createStreamMetadataProvider(String clientId) { - return new FakesStreamMetadataProvider(); - } - - public class FakePartitionLevelConsumer implements PartitionLevelConsumer { - - @Override - public MessageBatch fetchMessages(long startOffset, long endOffset, int timeoutMillis) - throws TimeoutException { - return null; - } - - @Override - public void close() - throws IOException { - } - } - - public class FakesStreamMetadataProvider implements StreamMetadataProvider { - - @Override - public List computePartitionGroupMetadata(String clientId, StreamConfig streamConfig, - List partitionGroupConsumptionStatuses, int timeoutMillis) - throws IOException, TimeoutException { - return Collections.singletonList(new PartitionGroupMetadata(0, new LongMsgOffset(0))); - } - - @Override - public int fetchPartitionCount(long timeoutMillis) { - return 1; - } - - @Override - public Set fetchPartitionIds(long timeoutMillis) { - return Collections.singleton(0); - } - - @Override - public StreamPartitionMsgOffset fetchStreamPartitionOffset(OffsetCriteria offsetCriteria, long timeoutMillis) - throws TimeoutException { - return new LongMsgOffset(0); - } - - @Override - public void close() - throws IOException { - } - } -} diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java index 039d7a42053e..b08feaf87822 100644 --- a/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java +++ b/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java @@ -124,7 +124,7 @@ private Map getStreamConfigs() { streamConfigs.put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaAvroMessageDecoder"); streamConfigs.put("stream.kafka.consumer.factory.class.name", - "org.apache.pinot.broker.broker.FakeStreamConsumerFactory"); + "org.apache.pinot.core.realtime.impl.fakestream.FakeStreamConsumerFactory"); return streamConfigs; } diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigUtilsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigUtilsTest.java index 2ff578b5ee35..1b98a0851cff 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigUtilsTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigUtilsTest.java @@ -30,7 +30,8 @@ import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig; import org.apache.pinot.spi.data.readers.GenericRow; -import org.apache.pinot.spi.stream.PartitionLevelConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; import org.apache.pinot.spi.stream.StreamConfigProperties; import org.apache.pinot.spi.stream.StreamConsumerFactory; import org.apache.pinot.spi.stream.StreamMessageDecoder; @@ -205,18 +206,20 @@ public GenericRow decode(byte[] payload, int offset, int length, GenericRow dest } private class TestStreamConsumerFactory extends StreamConsumerFactory { + @Override - public PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int partition) { + public StreamMetadataProvider createPartitionMetadataProvider(String clientId, int partition) { return null; } @Override - public StreamMetadataProvider createPartitionMetadataProvider(String clientId, int partition) { + public StreamMetadataProvider createStreamMetadataProvider(String clientId) { return null; } @Override - public StreamMetadataProvider createStreamMetadataProvider(String clientId) { + public PartitionGroupConsumer createPartitionGroupConsumer(String clientId, + PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) { return null; } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java index e226a13502a2..d15e79116052 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java @@ -92,7 +92,9 @@ import org.apache.pinot.spi.stream.StreamDataDecoderImpl; import org.apache.pinot.spi.stream.StreamDataDecoderResult; import org.apache.pinot.spi.stream.StreamDecoderProvider; +import org.apache.pinot.spi.stream.StreamMessage; import org.apache.pinot.spi.stream.StreamMessageDecoder; +import org.apache.pinot.spi.stream.StreamMessageMetadata; import org.apache.pinot.spi.stream.StreamMetadataProvider; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; import org.apache.pinot.spi.stream.StreamPartitionMsgOffsetFactory; @@ -242,7 +244,7 @@ public void deleteSegmentFile() { private final PartitionUpsertMetadataManager _partitionUpsertMetadataManager; private final BooleanSupplier _isReadyToConsumeData; private final MutableSegmentImpl _realtimeSegment; - private volatile StreamPartitionMsgOffset _currentOffset; + private volatile StreamPartitionMsgOffset _currentOffset; // Next offset to be consumed private volatile State _state; private volatile int _numRowsConsumed = 0; private volatile int _numRowsIndexed = 0; // Can be different from _numRowsConsumed when metrics update is enabled. @@ -262,7 +264,7 @@ public void deleteSegmentFile() { private volatile boolean _hasMessagesFetched = false; private volatile boolean _endOfPartitionGroup = false; private volatile boolean _forceCommitMessageReceived = false; - private StreamPartitionMsgOffset _finalOffset; // Used when we want to catch up to this one + private volatile StreamPartitionMsgOffset _finalOffset; // Exclusive, used when we want to catch up to this one private volatile boolean _shouldStop = false; // It takes 30s to locate controller leader, and more if there are multiple controller failures. @@ -426,8 +428,7 @@ protected boolean consumeLoop() // Update _currentOffset upon return from this method MessageBatch messageBatch; try { - messageBatch = - _partitionGroupConsumer.fetchMessages(_currentOffset, null, _streamConfig.getFetchTimeoutMillis()); + messageBatch = _partitionGroupConsumer.fetchMessages(_currentOffset, _streamConfig.getFetchTimeoutMillis()); //track realtime rows fetched on a table level. This included valid + invalid rows _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_ROWS_FETCHED, messageBatch.getUnfilteredMessageCount()); @@ -519,13 +520,13 @@ protected boolean consumeLoop() } /** - * @param messagesAndOffsets batch of messages to process + * @param messageBatch batch of messages to process * @param idlePipeSleepTimeMillis wait time in case no messages were read * @return returns true if the process loop ended before processing the batch, false * otherwise */ - private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePipeSleepTimeMillis) { - int messageCount = messagesAndOffsets.getMessageCount(); + private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSleepTimeMillis) { + int messageCount = messageBatch.getMessageCount(); _partitionRateLimiter.throttle(messageCount); _serverRateLimiter.throttle(messageCount); @@ -536,11 +537,9 @@ private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePi int indexedMessageCount = 0; int streamMessageCount = 0; boolean canTakeMore = true; - boolean hasTransformedRows = false; TransformPipeline.Result reusedResult = new TransformPipeline.Result(); boolean prematureExit = false; - RowMetadata msgMetadata = null; for (int index = 0; index < messageCount; index++) { prematureExit = _shouldStop || endCriteriaReached(); @@ -573,9 +572,19 @@ private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePi } // Decode message - StreamDataDecoderResult decodedRow = _streamDataDecoder.decode(messagesAndOffsets.getStreamMessage(index)); - msgMetadata = messagesAndOffsets.getStreamMessage(index).getMetadata(); - StreamPartitionMsgOffset messageOffset = messagesAndOffsets.getNextStreamPartitionMsgOffsetAtIndex(index); + StreamMessage streamMessage = messageBatch.getStreamMessage(index); + StreamDataDecoderResult decodedRow = _streamDataDecoder.decode(streamMessage); + StreamMessageMetadata metadata = streamMessage.getMetadata(); + StreamPartitionMsgOffset offset = null; + StreamPartitionMsgOffset nextOffset = null; + if (metadata != null) { + offset = metadata.getOffset(); + nextOffset = metadata.getNextOffset(); + } + // Backward compatible + if (nextOffset == null) { + nextOffset = messageBatch.getNextStreamPartitionMsgOffsetAtIndex(index); + } if (decodedRow.getException() != null) { // TODO: based on a config, decide whether the record should be silently dropped or stop further consumption on // decode error @@ -591,15 +600,15 @@ private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePi _numRowsErrored++; // when exception happens we prefer abandoning the whole batch and not partially indexing some rows reusedResult.getTransformedRows().clear(); - String errorMessage = String.format("Caught exception while transforming the record at offset: %s , row: %s", - messageOffset, decodedRow.getResult()); + String errorMessage = + String.format("Caught exception while transforming the record at offset: %s , row: %s", offset, + decodedRow.getResult()); _segmentLogger.error(errorMessage, e); _realtimeTableDataManager.addSegmentError(_segmentNameStr, new SegmentErrorInfo(now(), errorMessage, e)); } if (reusedResult.getSkippedRowCount() > 0) { - realtimeRowsDroppedMeter = - _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_ROWS_FILTERED, - reusedResult.getSkippedRowCount(), realtimeRowsDroppedMeter); + realtimeRowsDroppedMeter = _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_ROWS_FILTERED, + reusedResult.getSkippedRowCount(), realtimeRowsDroppedMeter); } if (reusedResult.getIncompleteRowCount() > 0) { realtimeIncompleteRowsConsumedMeter = @@ -607,14 +616,11 @@ private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePi reusedResult.getIncompleteRowCount(), realtimeIncompleteRowsConsumedMeter); } List transformedRows = reusedResult.getTransformedRows(); - if (transformedRows.size() > 0) { - hasTransformedRows = true; - } for (GenericRow transformedRow : transformedRows) { try { - canTakeMore = _realtimeSegment.index(transformedRow, msgMetadata); + canTakeMore = _realtimeSegment.index(transformedRow, metadata); indexedMessageCount++; - _lastRowMetadata = msgMetadata; + _lastRowMetadata = metadata; _lastConsumedTimestampMs = System.currentTimeMillis(); realtimeRowsConsumedMeter = _serverMetrics.addMeteredTableValue(_clientId, ServerMeter.REALTIME_ROWS_CONSUMED, 1, @@ -622,36 +628,24 @@ private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePi _serverMetrics.addMeteredGlobalValue(ServerMeter.REALTIME_ROWS_CONSUMED, 1L); } catch (Exception e) { _numRowsErrored++; - String errorMessage = String.format("Caught exception while indexing the record at offset: %s , row: %s", - messageOffset, transformedRow); + String errorMessage = + String.format("Caught exception while indexing the record at offset: %s , row: %s", offset, + transformedRow); _segmentLogger.error(errorMessage, e); _realtimeTableDataManager.addSegmentError(_segmentNameStr, new SegmentErrorInfo(now(), errorMessage, e)); } } } - _currentOffset = messageOffset; + _currentOffset = nextOffset; _numRowsIndexed = _realtimeSegment.getNumDocsIndexed(); _numRowsConsumed++; streamMessageCount++; } - if (indexedMessageCount > 0) { - // Record Ingestion delay for this partition with metadata for last message we processed - updateIngestionDelay(_lastRowMetadata); - } else if (!hasTransformedRows && (msgMetadata != null)) { - // If all messages were filtered by transformation, we still attempt to update ingestion delay using - // the metadata for the last message we processed if any. - updateIngestionDelay(msgMetadata); - } - updateCurrentDocumentCountMetrics(); - if (messagesAndOffsets.getUnfilteredMessageCount() > 0) { + if (messageBatch.getUnfilteredMessageCount() > 0) { + updateIngestionDelay(messageBatch.getLastMessageMetadata()); _hasMessagesFetched = true; - if (messageCount == 0) { - // If we received events from the stream but all were filtered, we attempt to estimate the ingestion - // delay from the metadata of the last filtered message received. - updateIngestionDelay(messagesAndOffsets.getLastMessageMetadata()); - } if (streamMessageCount > 0 && _segmentLogger.isDebugEnabled()) { _segmentLogger.debug("Indexed {} messages ({} messages read from stream) current offset {}", indexedMessageCount, streamMessageCount, _currentOffset); diff --git a/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakePartitionLevelConsumer.java b/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakePartitionLevelConsumer.java index cbb77066f2a8..3f52ca4740f2 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakePartitionLevelConsumer.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakePartitionLevelConsumer.java @@ -24,7 +24,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.concurrent.TimeoutException; import org.apache.avro.file.DataFileStream; import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; @@ -34,7 +33,7 @@ import org.apache.pinot.plugin.inputformat.avro.AvroUtils; import org.apache.pinot.spi.stream.LongMsgOffset; import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.PartitionLevelConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumer; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; import org.slf4j.Logger; @@ -44,15 +43,14 @@ /** - * Implementation of {@link PartitionLevelConsumer} for fake stream + * Implementation of {@link PartitionGroupConsumer} for fake stream * Unpacks tar files in /resources/data/On_Time_Performance_2014_partition_.tar.gz as source of messages */ -public class FakePartitionLevelConsumer implements PartitionLevelConsumer { +public class FakePartitionLevelConsumer implements PartitionGroupConsumer { + private static final Logger LOGGER = LoggerFactory.getLogger(FakePartitionLevelConsumer.class); - private static final Logger LOGGER = LoggerFactory.getLogger(FakePartitionLevelConsumer.class.getName()); - - private List _messageOffsets = new ArrayList<>(); - private List _messageBytes = new ArrayList<>(); + private final List _messageOffsets = new ArrayList<>(); + private final List _messageBytes = new ArrayList<>(); private final int _defaultBatchSize; FakePartitionLevelConsumer(int partition, StreamConfig streamConfig, int defaultBatchSize) { @@ -95,32 +93,15 @@ public class FakePartitionLevelConsumer implements PartitionLevelConsumer { } } - public MessageBatch fetchMessages(long startOffset, long endOffset, int timeoutMillis) - throws TimeoutException { - throw new UnsupportedOperationException("This method is deprecated"); - } - @Override - public MessageBatch fetchMessages(StreamPartitionMsgOffset startOffset, StreamPartitionMsgOffset endOffset, - int timeoutMillis) - throws TimeoutException { - if (startOffset.compareTo(FakeStreamConfigUtils.getLargestOffset()) >= 0) { - return new FakeStreamMessageBatch(Collections.emptyList(), Collections.emptyList()); - } - if (startOffset.compareTo(FakeStreamConfigUtils.getSmallestOffset()) < 0) { - startOffset = FakeStreamConfigUtils.getSmallestOffset(); - } - if (endOffset == null || endOffset.compareTo(FakeStreamConfigUtils.getLargestOffset()) > 0) { - endOffset = FakeStreamConfigUtils.getLargestOffset(); - } + public MessageBatch fetchMessages(StreamPartitionMsgOffset startOffset, int timeoutMs) { int startOffsetInt = (int) ((LongMsgOffset) startOffset).getOffset(); - int endOffsetInt = (int) ((LongMsgOffset) endOffset).getOffset(); - if (endOffsetInt > _messageOffsets.size() && _defaultBatchSize > 0) { - // Hack to get multiple batches - endOffsetInt = startOffsetInt + _defaultBatchSize; + if (startOffsetInt >= _messageOffsets.size()) { + return new FakeStreamMessageBatch(Collections.emptyList(), Collections.emptyList(), startOffsetInt); } - return new FakeStreamMessageBatch(_messageOffsets.subList(startOffsetInt, endOffsetInt), - _messageBytes.subList(startOffsetInt, endOffsetInt)); + int endOffsetInt = Math.min(startOffsetInt + _defaultBatchSize, _messageOffsets.size()); + return new FakeStreamMessageBatch(_messageBytes.subList(startOffsetInt, endOffsetInt), + _messageOffsets.subList(startOffsetInt, endOffsetInt), endOffsetInt); } @Override diff --git a/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakeStreamConsumerFactory.java b/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakeStreamConsumerFactory.java index 008d0dd9001d..3888a8cfe5c0 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakeStreamConsumerFactory.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakeStreamConsumerFactory.java @@ -18,21 +18,10 @@ */ package org.apache.pinot.core.realtime.impl.fakestream; -import org.apache.pinot.segment.local.utils.IngestionUtils; -import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.data.Schema; -import org.apache.pinot.spi.data.readers.GenericRow; -import org.apache.pinot.spi.stream.LongMsgOffset; -import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.OffsetCriteria; -import org.apache.pinot.spi.stream.PartitionLevelConsumer; -import org.apache.pinot.spi.stream.StreamConfig; +import org.apache.pinot.spi.stream.PartitionGroupConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; import org.apache.pinot.spi.stream.StreamConsumerFactory; -import org.apache.pinot.spi.stream.StreamConsumerFactoryProvider; -import org.apache.pinot.spi.stream.StreamDecoderProvider; -import org.apache.pinot.spi.stream.StreamMessageDecoder; import org.apache.pinot.spi.stream.StreamMetadataProvider; -import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; /** @@ -43,11 +32,6 @@ */ public class FakeStreamConsumerFactory extends StreamConsumerFactory { - @Override - public PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int partition) { - return new FakePartitionLevelConsumer(partition, _streamConfig, FakeStreamConfigUtils.MESSAGE_BATCH_SIZE); - } - @Override public StreamMetadataProvider createPartitionMetadataProvider(String clientId, int partition) { return new FakeStreamMetadataProvider(_streamConfig); @@ -58,43 +42,10 @@ public StreamMetadataProvider createStreamMetadataProvider(String clientId) { return new FakeStreamMetadataProvider(_streamConfig); } - public static void main(String[] args) - throws Exception { - String clientId = "client_id_localhost_tester"; - - // stream config - int numPartitions = 5; - StreamConfig streamConfig = FakeStreamConfigUtils.getDefaultLowLevelStreamConfigs(numPartitions); - - // stream consumer factory - StreamConsumerFactory streamConsumerFactory = StreamConsumerFactoryProvider.create(streamConfig); - - // stream metadata provider - StreamMetadataProvider streamMetadataProvider = streamConsumerFactory.createStreamMetadataProvider(clientId); - int partitionCount = streamMetadataProvider.fetchPartitionCount(10_000); - System.out.println(partitionCount); - - // Partition metadata provider - int partition = 3; - StreamMetadataProvider partitionMetadataProvider = - streamConsumerFactory.createPartitionMetadataProvider(clientId, partition); - StreamPartitionMsgOffset partitionOffset = - partitionMetadataProvider.fetchStreamPartitionOffset(OffsetCriteria.SMALLEST_OFFSET_CRITERIA, 10_000); - System.out.println(partitionOffset); - - // Partition level consumer - PartitionLevelConsumer partitionLevelConsumer = - streamConsumerFactory.createPartitionLevelConsumer(clientId, partition); - MessageBatch messageBatch = - partitionLevelConsumer.fetchMessages(new LongMsgOffset(10), new LongMsgOffset(40), 10_000); - - // Message decoder - Schema pinotSchema = FakeStreamConfigUtils.getPinotSchema(); - TableConfig tableConfig = FakeStreamConfigUtils.getTableConfig(); - StreamMessageDecoder streamMessageDecoder = StreamDecoderProvider.create(streamConfig, - IngestionUtils.getFieldsForRecordExtractor(tableConfig.getIngestionConfig(), pinotSchema)); - GenericRow decodedRow = new GenericRow(); - streamMessageDecoder.decode(messageBatch.getMessageAtIndex(0), decodedRow); - System.out.println(decodedRow); + @Override + public PartitionGroupConsumer createPartitionGroupConsumer(String clientId, + PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) { + return new FakePartitionLevelConsumer(partitionGroupConsumptionStatus.getPartitionGroupId(), _streamConfig, + FakeStreamConfigUtils.MESSAGE_BATCH_SIZE); } } diff --git a/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakeStreamMessageBatch.java b/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakeStreamMessageBatch.java index 134aac9fd84a..41cb158e63f6 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakeStreamMessageBatch.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/realtime/impl/fakestream/FakeStreamMessageBatch.java @@ -19,50 +19,43 @@ package org.apache.pinot.core.realtime.impl.fakestream; import java.util.List; +import org.apache.pinot.spi.stream.BytesStreamMessage; import org.apache.pinot.spi.stream.LongMsgOffset; import org.apache.pinot.spi.stream.MessageBatch; +import org.apache.pinot.spi.stream.StreamMessageMetadata; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; /** * MessageBatch implementation for the fake stream */ -public class FakeStreamMessageBatch implements MessageBatch { - private List _messageOffsets; - private List _messageBytes; - - FakeStreamMessageBatch(List messageOffsets, List messageBytes) { - _messageOffsets = messageOffsets; - _messageBytes = messageBytes; +class FakeStreamMessageBatch implements MessageBatch { + private final List _values; + private final List _offsets; + private final int _offsetOfNextBatch; + + FakeStreamMessageBatch(List values, List offsets, int offsetOfNextBatch) { + _values = values; + _offsets = offsets; + _offsetOfNextBatch = offsetOfNextBatch; } @Override public int getMessageCount() { - return _messageOffsets.size(); - } - - @Override - public byte[] getMessageAtIndex(int index) { - return _messageBytes.get(index); - } - - @Override - public int getMessageOffsetAtIndex(int index) { - return _messageOffsets.get(index); - } - - @Override - public int getMessageLengthAtIndex(int index) { - return _messageBytes.get(index).length; + return _values.size(); } @Override - public long getNextStreamMessageOffsetAtIndex(int index) { - throw new UnsupportedOperationException("This method is deprecated"); + public BytesStreamMessage getStreamMessage(int index) { + byte[] value = _values.get(index); + int offset = _offsets.get(index); + return new BytesStreamMessage(value, + new StreamMessageMetadata.Builder().setOffset(new LongMsgOffset(offset), new LongMsgOffset(offset + 1)) + .build()); } @Override - public StreamPartitionMsgOffset getNextStreamPartitionMsgOffsetAtIndex(int index) { - return new LongMsgOffset(_messageOffsets.get(index) + 1); + public StreamPartitionMsgOffset getOffsetOfNextBatch() { + return new LongMsgOffset(_offsetOfNextBatch); } } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java index 2389fe8ba68f..56d418a08013 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java @@ -47,6 +47,7 @@ import org.apache.pinot.common.utils.LLCSegmentName; import org.apache.pinot.controller.ControllerConf; import org.apache.pinot.plugin.stream.kafka20.KafkaConsumerFactory; +import org.apache.pinot.plugin.stream.kafka20.KafkaMessageBatch; import org.apache.pinot.plugin.stream.kafka20.KafkaPartitionLevelConsumer; import org.apache.pinot.spi.config.table.IndexingConfig; import org.apache.pinot.spi.config.table.TableConfig; @@ -55,11 +56,9 @@ import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.env.PinotConfiguration; -import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.PartitionLevelConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.stream.StreamConfigProperties; -import org.apache.pinot.spi.stream.StreamMessage; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.JsonUtils; @@ -463,8 +462,10 @@ public static void init(String helixClusterName, HelixAdmin helixAdmin, String t _helixClusterName = helixClusterName; _tableName = tableName; } + @Override - public PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int partition) { + public KafkaPartitionLevelConsumer createPartitionGroupConsumer(String clientId, + PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) { /* * The segment data manager is creating a consumer to consume rows into a segment. * Check the partition and sequence number of the segment and decide whether it @@ -473,6 +474,7 @@ public PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int * - Throwing exception during consumption. * Make sure that this still works if retries are added in RealtimeSegmentDataManager */ + int partition = partitionGroupConsumptionStatus.getPartitionGroupId(); boolean exceptionDuringConsume = false; int seqNum = getSegmentSeqNum(partition); if (partition == PARTITION_FOR_EXCEPTIONS) { @@ -504,20 +506,21 @@ private int getSegmentSeqNum(int partition) { return seqNum.get(); } - public class ExceptingKafkaConsumer extends KafkaPartitionLevelConsumer { + public static class ExceptingKafkaConsumer extends KafkaPartitionLevelConsumer { private final boolean _exceptionDuringConsume; + public ExceptingKafkaConsumer(String clientId, StreamConfig streamConfig, int partition, boolean exceptionDuringConsume) { super(clientId, streamConfig, partition); _exceptionDuringConsume = exceptionDuringConsume; } + @Override - public MessageBatch> fetchMessages(StreamPartitionMsgOffset startMsgOffset, - StreamPartitionMsgOffset endMsgOffset, int timeoutMillis) { + public KafkaMessageBatch fetchMessages(StreamPartitionMsgOffset startOffset, int timeoutMs) { if (_exceptionDuringConsume) { throw new RuntimeException("TestException during consumption"); } - return super.fetchMessages(startMsgOffset, endMsgOffset, timeoutMillis); + return super.fetchMessages(startOffset, timeoutMs); } } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaConsumerFactory.java b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaConsumerFactory.java index 9679a175dda0..aeae590bdeb6 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaConsumerFactory.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaConsumerFactory.java @@ -18,25 +18,26 @@ */ package org.apache.pinot.plugin.stream.kafka20; -import org.apache.pinot.spi.stream.PartitionLevelConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; import org.apache.pinot.spi.stream.StreamConsumerFactory; -import org.apache.pinot.spi.stream.StreamMetadataProvider; public class KafkaConsumerFactory extends StreamConsumerFactory { @Override - public PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int partition) { - return new KafkaPartitionLevelConsumer(clientId, _streamConfig, partition); + public KafkaStreamMetadataProvider createPartitionMetadataProvider(String clientId, int partition) { + return new KafkaStreamMetadataProvider(clientId, _streamConfig, partition); } @Override - public StreamMetadataProvider createPartitionMetadataProvider(String clientId, int partition) { - return new KafkaStreamMetadataProvider(clientId, _streamConfig, partition); + public KafkaStreamMetadataProvider createStreamMetadataProvider(String clientId) { + return new KafkaStreamMetadataProvider(clientId, _streamConfig); } @Override - public StreamMetadataProvider createStreamMetadataProvider(String clientId) { - return new KafkaStreamMetadataProvider(clientId, _streamConfig); + public KafkaPartitionLevelConsumer createPartitionGroupConsumer(String clientId, + PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) { + return new KafkaPartitionLevelConsumer(clientId, _streamConfig, + partitionGroupConsumptionStatus.getPartitionGroupId()); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaMessageBatch.java b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaMessageBatch.java index 005b4c27b3ad..3f137b54af3c 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaMessageBatch.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaMessageBatch.java @@ -18,51 +18,42 @@ */ package org.apache.pinot.plugin.stream.kafka20; -import java.nio.ByteBuffer; import java.util.List; +import javax.annotation.Nullable; +import org.apache.pinot.spi.stream.BytesStreamMessage; import org.apache.pinot.spi.stream.LongMsgOffset; import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.RowMetadata; -import org.apache.pinot.spi.stream.StreamMessage; import org.apache.pinot.spi.stream.StreamMessageMetadata; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; -public class KafkaMessageBatch implements MessageBatch> { - private final List> _messageList; +public class KafkaMessageBatch implements MessageBatch { + private final List _messages; private final int _unfilteredMessageCount; + private final long _offsetOfNextBatch; private final long _firstOffset; - private final long _lastOffset; private final StreamMessageMetadata _lastMessageMetadata; /** + * @param messages the messages, which may be smaller than {@see unfilteredMessageCount} * @param unfilteredMessageCount how many messages were received from the topic before being filtered - * @param firstOffset the offset of the first message in the batch - * @param lastOffset the offset of the last message in the batch - * @param batch the messages, which may be smaller than {@see unfilteredMessageCount} - * @param lastMessageMetadata metadata for last filtered message in the batch, useful for estimating ingestion delay - * when a batch has all messages filtered. + * @param offsetOfNextBatch the offset of the next batch + * @param firstOffset the offset of the first unfiltered message, -1 if no unfiltered messages + * @param lastMessageMetadata metadata for the last unfiltered message in the batch, useful for estimating ingestion + * delay when a batch has all messages filtered. */ - public KafkaMessageBatch(int unfilteredMessageCount, long firstOffset, long lastOffset, - List> batch, StreamMessageMetadata lastMessageMetadata) { - _messageList = batch; - _firstOffset = firstOffset; - _lastOffset = lastOffset; + public KafkaMessageBatch(List messages, int unfilteredMessageCount, long offsetOfNextBatch, + long firstOffset, @Nullable StreamMessageMetadata lastMessageMetadata) { + _messages = messages; _unfilteredMessageCount = unfilteredMessageCount; + _offsetOfNextBatch = offsetOfNextBatch; + _firstOffset = firstOffset; _lastMessageMetadata = lastMessageMetadata; } - @Override - /** - * Returns the metadata for the last filtered message if any, null otherwise. - */ - public StreamMessageMetadata getLastMessageMetadata() { - return _lastMessageMetadata; - } - @Override public int getMessageCount() { - return _messageList.size(); + return _messages.size(); } @Override @@ -71,52 +62,24 @@ public int getUnfilteredMessageCount() { } @Override - public StreamMessage getMessageAtIndex(int index) { - return _messageList.get(index); - } - - @Override - public int getMessageOffsetAtIndex(int index) { - return ByteBuffer.wrap(_messageList.get(index).getValue()).arrayOffset(); - } - - @Override - public int getMessageLengthAtIndex(int index) { - return _messageList.get(index).getValue().length; - } - - @Override - public long getNextStreamMessageOffsetAtIndex(int index) { - throw new UnsupportedOperationException("This method is deprecated"); - } - - @Override - public StreamPartitionMsgOffset getNextStreamPartitionMsgOffsetAtIndex(int index) { - return new LongMsgOffset(((KafkaStreamMessage) _messageList.get(index)).getNextOffset()); + public BytesStreamMessage getStreamMessage(int index) { + return _messages.get(index); } @Override public StreamPartitionMsgOffset getOffsetOfNextBatch() { - return new LongMsgOffset(_lastOffset + 1); - } - - @Override - public RowMetadata getMetadataAtIndex(int index) { - return _messageList.get(index).getMetadata(); + return new LongMsgOffset(_offsetOfNextBatch); } + @Nullable @Override - public byte[] getMessageBytesAtIndex(int index) { - return _messageList.get(index).getValue(); - } - - @Override - public StreamMessage getStreamMessage(int index) { - return _messageList.get(index); + public StreamPartitionMsgOffset getFirstMessageOffset() { + return _firstOffset >= 0 ? new LongMsgOffset(_firstOffset) : null; } + @Nullable @Override - public StreamPartitionMsgOffset getFirstMessageOffset() { - return new LongMsgOffset(_firstOffset); + public StreamMessageMetadata getLastMessageMetadata() { + return _lastMessageMetadata; } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaMetadataExtractor.java b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaMetadataExtractor.java deleted file mode 100644 index 86708ae715ae..000000000000 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaMetadataExtractor.java +++ /dev/null @@ -1,55 +0,0 @@ -/** - * 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.kafka20; - -import java.util.HashMap; -import java.util.Map; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.common.header.Header; -import org.apache.kafka.common.header.Headers; -import org.apache.pinot.spi.data.readers.GenericRow; -import org.apache.pinot.spi.stream.RowMetadata; - - -@FunctionalInterface -public interface KafkaMetadataExtractor { - static KafkaMetadataExtractor build(boolean populateMetadata) { - return record -> { - long recordTimestamp = record.timestamp(); - Map metadataMap = new HashMap<>(); - metadataMap.put(KafkaStreamMessageMetadata.METADATA_OFFSET_KEY, String.valueOf(record.offset())); - metadataMap.put(KafkaStreamMessageMetadata.RECORD_TIMESTAMP_KEY, String.valueOf(recordTimestamp)); - - if (!populateMetadata) { - return new KafkaStreamMessageMetadata(recordTimestamp, RowMetadata.EMPTY_ROW, metadataMap); - } - GenericRow headerGenericRow = new GenericRow(); - Headers headers = record.headers(); - if (headers != null) { - Header[] headersArray = headers.toArray(); - for (Header header : headersArray) { - headerGenericRow.putValue(header.key(), header.value()); - } - } - return new KafkaStreamMessageMetadata(record.timestamp(), headerGenericRow, metadataMap); - }; - } - - RowMetadata extract(ConsumerRecord consumerRecord); -} diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConnectionHandler.java b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConnectionHandler.java index f0512cc8b339..15b6c08496fa 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConnectionHandler.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConnectionHandler.java @@ -51,7 +51,6 @@ public abstract class KafkaPartitionLevelConnectionHandler { protected final String _topic; protected final Consumer _consumer; protected final TopicPartition _topicPartition; - protected final KafkaMetadataExtractor _kafkaMetadataExtractor; public KafkaPartitionLevelConnectionHandler(String clientId, StreamConfig streamConfig, int partition) { _config = new KafkaPartitionLevelStreamConfig(streamConfig); @@ -71,7 +70,6 @@ public KafkaPartitionLevelConnectionHandler(String clientId, StreamConfig stream _consumer = createConsumer(consumerProp); _topicPartition = new TopicPartition(_topic, _partition); _consumer.assign(Collections.singletonList(_topicPartition)); - _kafkaMetadataExtractor = KafkaMetadataExtractor.build(_config.isPopulateMetadata()); } private Consumer createConsumer(Properties consumerProp) { diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConsumer.java b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConsumer.java index df51d2fda9a3..36a74c1e6530 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConsumer.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConsumer.java @@ -22,14 +22,17 @@ import java.time.Duration; import java.util.ArrayList; import java.util.List; +import java.util.Map; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.utils.Bytes; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.stream.BytesStreamMessage; import org.apache.pinot.spi.stream.LongMsgOffset; -import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.PartitionLevelConsumer; +import org.apache.pinot.spi.stream.PartitionGroupConsumer; import org.apache.pinot.spi.stream.StreamConfig; -import org.apache.pinot.spi.stream.StreamMessage; import org.apache.pinot.spi.stream.StreamMessageMetadata; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; import org.slf4j.Logger; @@ -37,7 +40,7 @@ public class KafkaPartitionLevelConsumer extends KafkaPartitionLevelConnectionHandler - implements PartitionLevelConsumer { + implements PartitionGroupConsumer { private static final Logger LOGGER = LoggerFactory.getLogger(KafkaPartitionLevelConsumer.class); private long _lastFetchedOffset = -1; @@ -47,18 +50,10 @@ public KafkaPartitionLevelConsumer(String clientId, StreamConfig streamConfig, i } @Override - public MessageBatch> fetchMessages(StreamPartitionMsgOffset startMsgOffset, - StreamPartitionMsgOffset endMsgOffset, int timeoutMillis) { - final long startOffset = ((LongMsgOffset) startMsgOffset).getOffset(); - final long endOffset = endMsgOffset == null ? Long.MAX_VALUE : ((LongMsgOffset) endMsgOffset).getOffset(); - return fetchMessages(startOffset, endOffset, timeoutMillis); - } - - public synchronized MessageBatch> fetchMessages(long startOffset, long endOffset, - int timeoutMillis) { + public synchronized KafkaMessageBatch fetchMessages(StreamPartitionMsgOffset startMsgOffset, int timeoutMs) { + long startOffset = ((LongMsgOffset) startMsgOffset).getOffset(); if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Polling partition: {}, startOffset: {}, endOffset: {} timeout: {}ms", _topicPartition, startOffset, - endOffset, timeoutMillis); + LOGGER.debug("Polling partition: {}, startOffset: {}, timeout: {}ms", _topicPartition, startOffset, timeoutMs); } if (_lastFetchedOffset < 0 || _lastFetchedOffset != startOffset - 1) { if (LOGGER.isDebugEnabled()) { @@ -66,34 +61,49 @@ public synchronized MessageBatch> fetchMessages(long start } _consumer.seek(_topicPartition, startOffset); } - ConsumerRecords consumerRecords = _consumer.poll(Duration.ofMillis(timeoutMillis)); - List> messageAndOffsets = consumerRecords.records(_topicPartition); - List> filtered = new ArrayList<>(messageAndOffsets.size()); - long firstOffset = startOffset; - long lastOffset = startOffset; - StreamMessageMetadata rowMetadata = null; - if (!messageAndOffsets.isEmpty()) { - firstOffset = messageAndOffsets.get(0).offset(); - } - for (ConsumerRecord messageAndOffset : messageAndOffsets) { - long offset = messageAndOffset.offset(); - _lastFetchedOffset = offset; - if (offset >= startOffset && (endOffset > offset || endOffset < 0)) { - Bytes message = messageAndOffset.value(); - rowMetadata = (StreamMessageMetadata) _kafkaMetadataExtractor.extract(messageAndOffset); + ConsumerRecords consumerRecords = _consumer.poll(Duration.ofMillis(timeoutMs)); + List> records = consumerRecords.records(_topicPartition); + List filteredRecords = new ArrayList<>(records.size()); + long firstOffset = -1; + long offsetOfNextBatch = startOffset; + StreamMessageMetadata lastMessageMetadata = null; + if (!records.isEmpty()) { + firstOffset = records.get(0).offset(); + _lastFetchedOffset = records.get(records.size() - 1).offset(); + offsetOfNextBatch = _lastFetchedOffset + 1; + for (ConsumerRecord record : records) { + StreamMessageMetadata messageMetadata = extractMessageMetadata(record); + Bytes message = record.value(); if (message != null) { - String key = messageAndOffset.key(); + String key = record.key(); byte[] keyBytes = key != null ? key.getBytes(StandardCharsets.UTF_8) : null; - filtered.add(new KafkaStreamMessage(keyBytes, message.get(), rowMetadata)); + filteredRecords.add(new BytesStreamMessage(keyBytes, message.get(), messageMetadata)); } else if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Tombstone message at offset: {}", offset); + LOGGER.debug("Tombstone message at offset: {}", record.offset()); + } + lastMessageMetadata = messageMetadata; + } + } + return new KafkaMessageBatch(filteredRecords, records.size(), offsetOfNextBatch, firstOffset, lastMessageMetadata); + } + + private StreamMessageMetadata extractMessageMetadata(ConsumerRecord record) { + long timestamp = record.timestamp(); + long offset = record.offset(); + StreamMessageMetadata.Builder builder = new StreamMessageMetadata.Builder().setRecordIngestionTimeMs(timestamp) + .setOffset(new LongMsgOffset(offset), new LongMsgOffset(offset + 1)); + if (_config.isPopulateMetadata()) { + Headers headers = record.headers(); + if (headers != null) { + GenericRow headerGenericRow = new GenericRow(); + for (Header header : headers.toArray()) { + headerGenericRow.putValue(header.key(), header.value()); } - lastOffset = offset; - } else if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Ignoring message at offset: {} (outside of offset range [{}, {}))", offset, startOffset, - endOffset); + builder.setHeaders(headerGenericRow); } + builder.setMetadata(Map.of(KafkaStreamMessageMetadata.RECORD_TIMESTAMP_KEY, String.valueOf(timestamp), + KafkaStreamMessageMetadata.METADATA_OFFSET_KEY, String.valueOf(offset))); } - return new KafkaMessageBatch(messageAndOffsets.size(), firstOffset, lastOffset, filtered, rowMetadata); + return builder.build(); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMessage.java b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMessage.java deleted file mode 100644 index a2bc3122d259..000000000000 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMessage.java +++ /dev/null @@ -1,38 +0,0 @@ -/** - * 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.kafka20; - -import javax.annotation.Nullable; -import org.apache.pinot.spi.stream.StreamMessage; -import org.apache.pinot.spi.stream.StreamMessageMetadata; - - -public class KafkaStreamMessage extends StreamMessage { - public KafkaStreamMessage(@Nullable byte[] key, byte[] value, @Nullable StreamMessageMetadata metadata) { - super(key, value, metadata, value.length); - } - - public long getNextOffset() { - if (_metadata != null) { - long offset = Long.parseLong(_metadata.getRecordMetadata().get(KafkaStreamMessageMetadata.METADATA_OFFSET_KEY)); - return offset < 0 ? -1 : offset + 1; - } - return -1; - } -} diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMessageMetadata.java b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMessageMetadata.java index ada4e22bbb52..7cecab28ef03 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMessageMetadata.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMessageMetadata.java @@ -23,11 +23,12 @@ import org.apache.pinot.spi.data.readers.GenericRow; import org.apache.pinot.spi.stream.StreamMessageMetadata; - +// TODO: Make it a util class public class KafkaStreamMessageMetadata extends StreamMessageMetadata { public static final String METADATA_OFFSET_KEY = "offset"; public static final String RECORD_TIMESTAMP_KEY = "recordTimestamp"; + @Deprecated public KafkaStreamMessageMetadata(long recordIngestionTimeMs, @Nullable GenericRow headers, Map metadata) { super(recordIngestionTimeMs, headers, metadata); diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/test/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConsumerTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/test/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConsumerTest.java index e783e091ae90..6719a722c761 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/test/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConsumerTest.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/test/java/org/apache/pinot/plugin/stream/kafka20/KafkaPartitionLevelConsumerTest.java @@ -32,17 +32,22 @@ import org.apache.pinot.spi.stream.LongMsgOffset; import org.apache.pinot.spi.stream.MessageBatch; import org.apache.pinot.spi.stream.OffsetCriteria; -import org.apache.pinot.spi.stream.PartitionLevelConsumer; -import org.apache.pinot.spi.stream.RowMetadata; +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.StreamConsumerFactory; import org.apache.pinot.spi.stream.StreamConsumerFactoryProvider; import org.apache.pinot.spi.stream.StreamMessage; -import org.testng.Assert; +import org.apache.pinot.spi.stream.StreamMessageMetadata; +import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; + /** * Tests for the KafkaPartitionLevelConsumer. @@ -104,8 +109,7 @@ public void tearDown() } @Test - public void testBuildConsumer() - throws Exception { + public void testBuildConsumer() { String streamType = "kafka"; String streamKafkaTopicName = "theTopic"; String streamKafkaBrokerList = _kafkaBrokerAddress; @@ -126,27 +130,25 @@ public void testBuildConsumer() // test default value KafkaPartitionLevelConsumer kafkaSimpleStreamConsumer = new KafkaPartitionLevelConsumer(clientId, streamConfig, 0); - kafkaSimpleStreamConsumer.fetchMessages(new LongMsgOffset(12345L), new LongMsgOffset(23456L), 10000); + kafkaSimpleStreamConsumer.fetchMessages(new LongMsgOffset(12345L), 10000); - Assert.assertEquals(KafkaStreamConfigProperties.LowLevelConsumer.KAFKA_BUFFER_SIZE_DEFAULT, + assertEquals(KafkaStreamConfigProperties.LowLevelConsumer.KAFKA_BUFFER_SIZE_DEFAULT, kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaBufferSize()); - Assert.assertEquals(KafkaStreamConfigProperties.LowLevelConsumer.KAFKA_SOCKET_TIMEOUT_DEFAULT, + assertEquals(KafkaStreamConfigProperties.LowLevelConsumer.KAFKA_SOCKET_TIMEOUT_DEFAULT, kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaSocketTimeout()); // test parsing values - Assert.assertEquals(10000, - kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaFetcherSizeBytes()); - Assert.assertEquals(20000, - kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaFetcherMinBytes()); + assertEquals(10000, kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaFetcherSizeBytes()); + assertEquals(20000, kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaFetcherMinBytes()); // test user defined values streamConfigMap.put("stream.kafka.buffer.size", "100"); streamConfigMap.put("stream.kafka.socket.timeout", "1000"); streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); kafkaSimpleStreamConsumer = new KafkaPartitionLevelConsumer(clientId, streamConfig, 0); - kafkaSimpleStreamConsumer.fetchMessages(new LongMsgOffset(12345L), new LongMsgOffset(23456L), 10000); - Assert.assertEquals(100, kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaBufferSize()); - Assert.assertEquals(1000, kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaSocketTimeout()); + kafkaSimpleStreamConsumer.fetchMessages(new LongMsgOffset(12345L), 10000); + assertEquals(100, kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaBufferSize()); + assertEquals(1000, kafkaSimpleStreamConsumer.getKafkaPartitionLevelStreamConfig().getKafkaSocketTimeout()); } @Test @@ -167,7 +169,7 @@ public void testGetPartitionCount() { StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); KafkaStreamMetadataProvider streamMetadataProvider = new KafkaStreamMetadataProvider(clientId, streamConfig); - Assert.assertEquals(streamMetadataProvider.fetchPartitionCount(10000L), 1); + assertEquals(streamMetadataProvider.fetchPartitionCount(10000L), 1); streamConfigMap = new HashMap<>(); streamConfigMap.put("streamType", streamType); @@ -179,12 +181,11 @@ public void testGetPartitionCount() { streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); streamMetadataProvider = new KafkaStreamMetadataProvider(clientId, streamConfig); - Assert.assertEquals(streamMetadataProvider.fetchPartitionCount(10000L), 2); + assertEquals(streamMetadataProvider.fetchPartitionCount(10000L), 2); } @Test - public void testFetchMessages() - throws Exception { + public void testFetchMessages() { String streamType = "kafka"; String streamKafkaTopicName = "theTopic"; String streamKafkaBrokerList = _kafkaBrokerAddress; @@ -204,18 +205,16 @@ public void testFetchMessages() int partition = 0; KafkaPartitionLevelConsumer kafkaSimpleStreamConsumer = new KafkaPartitionLevelConsumer(clientId, streamConfig, partition); - kafkaSimpleStreamConsumer.fetchMessages(new LongMsgOffset(12345L), new LongMsgOffset(23456L), 10000); + kafkaSimpleStreamConsumer.fetchMessages(new LongMsgOffset(12345L), 10000); } @Test - public void testFetchOffsets() - throws Exception { + public void testFetchOffsets() { testFetchOffsets(TEST_TOPIC_1); testFetchOffsets(TEST_TOPIC_2); } - private void testFetchOffsets(String topic) - throws Exception { + private void testFetchOffsets(String topic) { String streamType = "kafka"; String streamKafkaBrokerList = _kafkaBrokerAddress; String streamKafkaConsumerType = "simple"; @@ -235,14 +234,14 @@ private void testFetchOffsets(String topic) for (int partition = 0; partition < numPartitions; partition++) { KafkaStreamMetadataProvider kafkaStreamMetadataProvider = new KafkaStreamMetadataProvider(clientId, streamConfig, partition); - Assert.assertEquals(new LongMsgOffset(0).compareTo(kafkaStreamMetadataProvider.fetchStreamPartitionOffset( + assertEquals(new LongMsgOffset(0).compareTo(kafkaStreamMetadataProvider.fetchStreamPartitionOffset( new OffsetCriteria.OffsetCriteriaBuilder().withOffsetSmallest(), 10000)), 0); - Assert.assertEquals(new LongMsgOffset(0).compareTo(kafkaStreamMetadataProvider.fetchStreamPartitionOffset( + assertEquals(new LongMsgOffset(0).compareTo(kafkaStreamMetadataProvider.fetchStreamPartitionOffset( new OffsetCriteria.OffsetCriteriaBuilder().withOffsetAsPeriod("2d"), 10000)), 0); - Assert.assertEquals(new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION) - .compareTo(kafkaStreamMetadataProvider.fetchStreamPartitionOffset(new OffsetCriteria.OffsetCriteriaBuilder() - .withOffsetAsTimestamp(Instant.now().toString()), 10000)), 0); - Assert.assertEquals(new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION).compareTo( + assertEquals(new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION).compareTo( + kafkaStreamMetadataProvider.fetchStreamPartitionOffset( + new OffsetCriteria.OffsetCriteriaBuilder().withOffsetAsTimestamp(Instant.now().toString()), 10000)), 0); + assertEquals(new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION).compareTo( kafkaStreamMetadataProvider.fetchStreamPartitionOffset( new OffsetCriteria.OffsetCriteriaBuilder().withOffsetLargest(), 10000)), 0); } @@ -272,102 +271,100 @@ private void testConsumer(String topic) streamConfigMap.put("stream.kafka.decoder.class.name", "decoderClass"); StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); - final StreamConsumerFactory streamConsumerFactory = StreamConsumerFactoryProvider.create(streamConfig); + StreamConsumerFactory streamConsumerFactory = StreamConsumerFactoryProvider.create(streamConfig); int numPartitions = new KafkaStreamMetadataProvider(clientId, streamConfig).fetchPartitionCount(10000); for (int partition = 0; partition < numPartitions; partition++) { - final PartitionLevelConsumer consumer = streamConsumerFactory.createPartitionLevelConsumer(clientId, partition); + PartitionGroupConsumer consumer = streamConsumerFactory.createPartitionGroupConsumer(clientId, + new PartitionGroupConsumptionStatus(partition, 0, new LongMsgOffset(0), + new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION), "CONSUMING")); // Test consume a large batch, only 500 records will be returned. - final MessageBatch batch1 = - consumer.fetchMessages(new LongMsgOffset(0), new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION), 10000); - Assert.assertEquals(batch1.getMessageCount(), 500); - for (int i = 0; i < batch1.getMessageCount(); i++) { - StreamMessage streamMessage = batch1.getStreamMessage(i); - Assert.assertNotNull(streamMessage.getMetadata()); - final byte[] msg = (byte[]) streamMessage.getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + i); - Assert.assertNotNull(batch1.getMetadataAtIndex(i)); + MessageBatch messageBatch = consumer.fetchMessages(new LongMsgOffset(0), 10000); + assertEquals(messageBatch.getMessageCount(), 500); + assertEquals(messageBatch.getUnfilteredMessageCount(), 500); + for (int i = 0; i < 500; i++) { + StreamMessage streamMessage = messageBatch.getStreamMessage(i); + assertEquals(new String((byte[]) streamMessage.getValue()), "sample_msg_" + i); + StreamMessageMetadata metadata = streamMessage.getMetadata(); + assertNotNull(metadata); + assertEquals(metadata.getRecordIngestionTimeMs(), TIMESTAMP + i); + StreamPartitionMsgOffset offset = metadata.getOffset(); + assertTrue(offset instanceof LongMsgOffset); + assertEquals(((LongMsgOffset) offset).getOffset(), i); + StreamPartitionMsgOffset nextOffset = metadata.getNextOffset(); + assertTrue(nextOffset instanceof LongMsgOffset); + assertEquals(((LongMsgOffset) nextOffset).getOffset(), i + 1); } + assertEquals(messageBatch.getOffsetOfNextBatch().toString(), "500"); + assertEquals(messageBatch.getFirstMessageOffset().toString(), "0"); + assertEquals(messageBatch.getLastMessageMetadata().getOffset().toString(), "499"); + assertEquals(messageBatch.getLastMessageMetadata().getNextOffset().toString(), "500"); + // Test second half batch - final MessageBatch batch2 = - consumer.fetchMessages(new LongMsgOffset(500), new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION), 10000); - Assert.assertEquals(batch2.getMessageCount(), 500); - for (int i = 0; i < batch2.getMessageCount(); i++) { - StreamMessage streamMessage = batch2.getStreamMessage(i); - Assert.assertNotNull(streamMessage.getMetadata()); - final byte[] msg = (byte[]) streamMessage.getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + (500 + i)); - Assert.assertNotNull(batch1.getMetadataAtIndex(i)); + messageBatch = consumer.fetchMessages(new LongMsgOffset(500), 10000); + assertEquals(messageBatch.getMessageCount(), 500); + assertEquals(messageBatch.getUnfilteredMessageCount(), 500); + for (int i = 0; i < 500; i++) { + StreamMessage streamMessage = messageBatch.getStreamMessage(i); + assertEquals(new String((byte[]) streamMessage.getValue()), "sample_msg_" + (500 + i)); + StreamMessageMetadata metadata = streamMessage.getMetadata(); + assertNotNull(metadata); + assertEquals(metadata.getRecordIngestionTimeMs(), TIMESTAMP + 500 + i); + StreamPartitionMsgOffset offset = metadata.getOffset(); + assertTrue(offset instanceof LongMsgOffset); + assertEquals(((LongMsgOffset) offset).getOffset(), 500 + i); + StreamPartitionMsgOffset nextOffset = metadata.getNextOffset(); + assertTrue(nextOffset instanceof LongMsgOffset); + assertEquals(((LongMsgOffset) nextOffset).getOffset(), 501 + i); } + assertEquals(messageBatch.getOffsetOfNextBatch().toString(), "1000"); + assertEquals(messageBatch.getFirstMessageOffset().toString(), "500"); + assertEquals(messageBatch.getLastMessageMetadata().getOffset().toString(), "999"); + assertEquals(messageBatch.getLastMessageMetadata().getNextOffset().toString(), "1000"); + // Some random range - final MessageBatch batch3 = consumer.fetchMessages(new LongMsgOffset(10), new LongMsgOffset(35), 10000); - Assert.assertEquals(batch3.getMessageCount(), 25); - for (int i = 0; i < batch3.getMessageCount(); i++) { - StreamMessage streamMessage = batch3.getStreamMessage(i); - Assert.assertNotNull(streamMessage.getMetadata()); - final byte[] msg = (byte[]) streamMessage.getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + (10 + i)); - Assert.assertNotNull(batch1.getMetadataAtIndex(i)); + messageBatch = consumer.fetchMessages(new LongMsgOffset(10), 10000); + assertEquals(messageBatch.getMessageCount(), 500); + assertEquals(messageBatch.getUnfilteredMessageCount(), 500); + for (int i = 0; i < 500; i++) { + StreamMessage streamMessage = messageBatch.getStreamMessage(i); + assertEquals(new String((byte[]) streamMessage.getValue()), "sample_msg_" + (10 + i)); + StreamMessageMetadata metadata = streamMessage.getMetadata(); + assertNotNull(metadata); + assertEquals(metadata.getRecordIngestionTimeMs(), TIMESTAMP + 10 + i); + StreamPartitionMsgOffset offset = metadata.getOffset(); + assertTrue(offset instanceof LongMsgOffset); + assertEquals(((LongMsgOffset) offset).getOffset(), 10 + i); + StreamPartitionMsgOffset nextOffset = metadata.getNextOffset(); + assertTrue(nextOffset instanceof LongMsgOffset); + assertEquals(((LongMsgOffset) nextOffset).getOffset(), 11 + i); } - } - } - - @Test - public void testMessageMetadata() - throws Exception { - testMessageMetadata(TEST_TOPIC_1); - testMessageMetadata(TEST_TOPIC_2); - } - - private void testMessageMetadata(String topic) - throws TimeoutException { - String streamType = "kafka"; - String streamKafkaBrokerList = _kafkaBrokerAddress; - String streamKafkaConsumerType = "simple"; - String clientId = "clientId"; - String tableNameWithType = "tableName_REALTIME"; + assertEquals(messageBatch.getOffsetOfNextBatch().toString(), "510"); + assertEquals(messageBatch.getFirstMessageOffset().toString(), "10"); + assertEquals(messageBatch.getLastMessageMetadata().getOffset().toString(), "509"); + assertEquals(messageBatch.getLastMessageMetadata().getNextOffset().toString(), "510"); - Map streamConfigMap = new HashMap<>(); - streamConfigMap.put("streamType", streamType); - streamConfigMap.put("stream.kafka.topic.name", topic); - streamConfigMap.put("stream.kafka.broker.list", streamKafkaBrokerList); - streamConfigMap.put("stream.kafka.consumer.type", streamKafkaConsumerType); - streamConfigMap.put("stream.kafka.consumer.factory.class.name", getKafkaConsumerFactoryName()); - streamConfigMap.put("stream.kafka.decoder.class.name", "decoderClass"); - streamConfigMap.put("stream.kafka.metadata.populate", "true"); - StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); - - final StreamConsumerFactory streamConsumerFactory = StreamConsumerFactoryProvider.create(streamConfig); - int numPartitions = new KafkaStreamMetadataProvider(clientId, streamConfig).fetchPartitionCount(10000); - for (int partition = 0; partition < numPartitions; partition++) { - final PartitionLevelConsumer consumer = streamConsumerFactory.createPartitionLevelConsumer(clientId, partition); - - // Test consume a large batch, only 500 records will be returned. - final MessageBatch batch1 = - consumer.fetchMessages(new LongMsgOffset(0), new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION), 10000); - Assert.assertEquals(batch1.getMessageCount(), 500); - for (int i = 0; i < batch1.getMessageCount(); i++) { - final RowMetadata metadata = batch1.getMetadataAtIndex(i); - Assert.assertNotNull(metadata); - Assert.assertEquals(metadata.getRecordIngestionTimeMs(), TIMESTAMP + i); - } - // Test second half batch - final MessageBatch batch2 = - consumer.fetchMessages(new LongMsgOffset(500), new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION), 10000); - Assert.assertEquals(batch2.getMessageCount(), 500); - for (int i = 0; i < batch2.getMessageCount(); i++) { - final RowMetadata metadata = batch2.getMetadataAtIndex(i); - Assert.assertNotNull(metadata); - Assert.assertEquals(metadata.getRecordIngestionTimeMs(), TIMESTAMP + (500 + i)); - } // Some random range - final MessageBatch batch3 = consumer.fetchMessages(new LongMsgOffset(10), new LongMsgOffset(35), 10000); - Assert.assertEquals(batch3.getMessageCount(), 25); - for (int i = 0; i < batch3.getMessageCount(); i++) { - final RowMetadata metadata = batch3.getMetadataAtIndex(i); - Assert.assertNotNull(metadata); - Assert.assertEquals(metadata.getRecordIngestionTimeMs(), TIMESTAMP + (10 + i)); + messageBatch = consumer.fetchMessages(new LongMsgOffset(610), 10000); + assertEquals(messageBatch.getMessageCount(), 390); + assertEquals(messageBatch.getUnfilteredMessageCount(), 390); + for (int i = 0; i < 390; i++) { + StreamMessage streamMessage = messageBatch.getStreamMessage(i); + assertEquals(new String((byte[]) streamMessage.getValue()), "sample_msg_" + (610 + i)); + StreamMessageMetadata metadata = streamMessage.getMetadata(); + assertNotNull(metadata); + assertEquals(metadata.getRecordIngestionTimeMs(), TIMESTAMP + 610 + i); + StreamPartitionMsgOffset offset = metadata.getOffset(); + assertTrue(offset instanceof LongMsgOffset); + assertEquals(((LongMsgOffset) offset).getOffset(), 610 + i); + StreamPartitionMsgOffset nextOffset = metadata.getNextOffset(); + assertTrue(nextOffset instanceof LongMsgOffset); + assertEquals(((LongMsgOffset) nextOffset).getOffset(), 611 + i); } + assertEquals(messageBatch.getOffsetOfNextBatch().toString(), "1000"); + assertEquals(messageBatch.getFirstMessageOffset().toString(), "610"); + assertEquals(messageBatch.getLastMessageMetadata().getOffset().toString(), "999"); + assertEquals(messageBatch.getLastMessageMetadata().getNextOffset().toString(), "1000"); } } @@ -389,35 +386,17 @@ public void testOffsetsExpired() StreamConfig streamConfig = new StreamConfig("tableName_REALTIME", streamConfigMap); StreamConsumerFactory streamConsumerFactory = StreamConsumerFactoryProvider.create(streamConfig); - PartitionLevelConsumer consumer = streamConsumerFactory.createPartitionLevelConsumer("clientId", 0); + PartitionGroupConsumer consumer = streamConsumerFactory.createPartitionGroupConsumer("clientId", + new PartitionGroupConsumptionStatus(0, 0, new LongMsgOffset(0), + new LongMsgOffset(NUM_MSG_PRODUCED_PER_PARTITION), "CONSUMING")); // Start offset has expired. Automatically reset to earliest available and fetch whatever available - MessageBatch batch1 = consumer.fetchMessages(new LongMsgOffset(0), new LongMsgOffset(400), 10000); - Assert.assertEquals(batch1.getMessageCount(), 200); - for (int i = 0; i < batch1.getMessageCount(); i++) { - byte[] msg = (byte[]) batch1.getStreamMessage(i).getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + (i + 200)); - } - Assert.assertEquals(batch1.getOffsetOfNextBatch().toString(), "400"); - - // Start and end offset has expired - MessageBatch batch2 = consumer.fetchMessages(new LongMsgOffset(0), new LongMsgOffset(100), 10000); - Assert.assertEquals(batch2.getMessageCount(), 0); - - MessageBatch batch3 = consumer.fetchMessages(new LongMsgOffset(201), new LongMsgOffset(401), 10000); - Assert.assertEquals(batch3.getMessageCount(), 200); - for (int i = 0; i < batch3.getMessageCount(); i++) { - byte[] msg = (byte[]) batch3.getStreamMessage(i).getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + (i + 201)); - } - Assert.assertEquals(batch3.getOffsetOfNextBatch().toString(), "401"); - - MessageBatch batch4 = consumer.fetchMessages(new LongMsgOffset(0), null, 10000); - Assert.assertEquals(batch4.getMessageCount(), 500); - for (int i = 0; i < batch4.getMessageCount(); i++) { - byte[] msg = (byte[]) batch4.getStreamMessage(i).getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + (i + 200)); + MessageBatch messageBatch = consumer.fetchMessages(new LongMsgOffset(0), 10000); + assertEquals(messageBatch.getMessageCount(), 500); + assertEquals(messageBatch.getUnfilteredMessageCount(), 500); + for (int i = 0; i < 500; i++) { + assertEquals(new String((byte[]) messageBatch.getStreamMessage(i).getValue()), "sample_msg_" + (200 + i)); } - Assert.assertEquals(batch4.getOffsetOfNextBatch().toString(), "700"); + assertEquals(messageBatch.getOffsetOfNextBatch().toString(), "700"); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConnectionHandler.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConnectionHandler.java index 5c021e4b9396..7ae08cdbb5b2 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConnectionHandler.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConnectionHandler.java @@ -19,6 +19,7 @@ package org.apache.pinot.plugin.stream.kinesis; import com.google.common.annotations.VisibleForTesting; +import java.io.Closeable; import java.net.URI; import java.net.URISyntaxException; import java.util.List; @@ -42,115 +43,80 @@ /** * Manages the Kinesis stream connection, given the stream name and aws region */ -public class KinesisConnectionHandler { - protected KinesisClient _kinesisClient; - private final String _stream; - private final String _region; - private final String _accessKey; - private final String _secretKey; - private final String _endpoint; - private final KinesisConfig _kinesisConfig; - protected final KinesisMetadataExtractor _kinesisMetadataExtractor; +public class KinesisConnectionHandler implements Closeable { + protected final KinesisConfig _config; + protected final KinesisClient _kinesisClient; - public KinesisConnectionHandler(KinesisConfig kinesisConfig) { - _stream = kinesisConfig.getStreamTopicName(); - _region = kinesisConfig.getAwsRegion(); - _accessKey = kinesisConfig.getAccessKey(); - _secretKey = kinesisConfig.getSecretKey(); - _endpoint = kinesisConfig.getEndpoint(); - _kinesisConfig = kinesisConfig; - _kinesisMetadataExtractor = KinesisMetadataExtractor.build(kinesisConfig.isPopulateMetadata()); - createConnection(); + public KinesisConnectionHandler(KinesisConfig config) { + _config = config; + _kinesisClient = createClient(); } @VisibleForTesting - public KinesisConnectionHandler(KinesisConfig kinesisConfig, KinesisClient kinesisClient) { - _stream = kinesisConfig.getStreamTopicName(); - _region = kinesisConfig.getAwsRegion(); - _accessKey = kinesisConfig.getAccessKey(); - _secretKey = kinesisConfig.getSecretKey(); - _endpoint = kinesisConfig.getEndpoint(); - _kinesisConfig = kinesisConfig; - _kinesisMetadataExtractor = KinesisMetadataExtractor.build(kinesisConfig.isPopulateMetadata()); + public KinesisConnectionHandler(KinesisConfig config, KinesisClient kinesisClient) { + _config = config; _kinesisClient = kinesisClient; } - /** - * Lists all shards of the stream - */ - public List getShards() { - ListShardsResponse listShardsResponse = - _kinesisClient.listShards(ListShardsRequest.builder().streamName(_stream).build()); - return listShardsResponse.shards(); - } - - /** - * Creates a Kinesis client for the stream - */ - public void createConnection() { - if (_kinesisClient == null) { - KinesisClientBuilder kinesisClientBuilder; + private KinesisClient createClient() { + KinesisClientBuilder kinesisClientBuilder; + + AwsCredentialsProvider awsCredentialsProvider; + String accessKey = _config.getAccessKey(); + String secretKey = _config.getSecretKey(); + if (StringUtils.isNotBlank(accessKey) && StringUtils.isNotBlank(secretKey)) { + AwsBasicCredentials awsBasicCredentials = AwsBasicCredentials.create(accessKey, secretKey); + awsCredentialsProvider = StaticCredentialsProvider.create(awsBasicCredentials); + } else { + awsCredentialsProvider = DefaultCredentialsProvider.builder().build(); + } - AwsCredentialsProvider awsCredentialsProvider; - if (StringUtils.isNotBlank(_accessKey) && StringUtils.isNotBlank(_secretKey)) { - AwsBasicCredentials awsBasicCredentials = AwsBasicCredentials.create(_accessKey, _secretKey); - awsCredentialsProvider = StaticCredentialsProvider.create(awsBasicCredentials); + if (_config.isIamRoleBasedAccess()) { + AssumeRoleRequest.Builder assumeRoleRequestBuilder = + AssumeRoleRequest.builder().roleArn(_config.getRoleArn()).roleSessionName(_config.getRoleSessionName()) + .durationSeconds(_config.getSessionDurationSeconds()); + AssumeRoleRequest assumeRoleRequest; + String externalId = _config.getExternalId(); + if (StringUtils.isNotBlank(externalId)) { + assumeRoleRequest = assumeRoleRequestBuilder.externalId(externalId).build(); } else { - awsCredentialsProvider = DefaultCredentialsProvider.builder().build(); + assumeRoleRequest = assumeRoleRequestBuilder.build(); } - - if (_kinesisConfig.isIamRoleBasedAccess()) { - AssumeRoleRequest.Builder assumeRoleRequestBuilder = - AssumeRoleRequest.builder() - .roleArn(_kinesisConfig.getRoleArn()) - .roleSessionName(_kinesisConfig.getRoleSessionName()) - .durationSeconds(_kinesisConfig.getSessionDurationSeconds()); - - AssumeRoleRequest assumeRoleRequest; - if (StringUtils.isNotEmpty(_kinesisConfig.getExternalId())) { - assumeRoleRequest = assumeRoleRequestBuilder - .externalId(_kinesisConfig.getExternalId()) + StsClient stsClient = + StsClient.builder().region(Region.of(_config.getAwsRegion())).credentialsProvider(awsCredentialsProvider) .build(); - } else { - assumeRoleRequest = assumeRoleRequestBuilder.build(); - } + awsCredentialsProvider = + StsAssumeRoleCredentialsProvider.builder().stsClient(stsClient).refreshRequest(assumeRoleRequest) + .asyncCredentialUpdateEnabled(_config.isAsyncSessionUpdateEnabled()).build(); + } - StsClient stsClient = - StsClient.builder() - .region(Region.of(_region)) - .credentialsProvider(awsCredentialsProvider) - .build(); + kinesisClientBuilder = + KinesisClient.builder().region(Region.of(_config.getAwsRegion())).credentialsProvider(awsCredentialsProvider) + .httpClientBuilder(new ApacheSdkHttpService().createHttpClientBuilder()); - awsCredentialsProvider = - StsAssumeRoleCredentialsProvider.builder() - .stsClient(stsClient) - .refreshRequest(assumeRoleRequest) - .asyncCredentialUpdateEnabled(_kinesisConfig.isAsyncSessionUpdateEnabled()) - .build(); + String endpoint = _config.getEndpoint(); + if (StringUtils.isNotBlank(endpoint)) { + try { + kinesisClientBuilder = kinesisClientBuilder.endpointOverride(new URI(endpoint)); + } catch (URISyntaxException e) { + throw new IllegalArgumentException("URI syntax is not correctly specified for endpoint: " + endpoint, e); } + } - kinesisClientBuilder = - KinesisClient.builder() - .region(Region.of(_region)) - .credentialsProvider(awsCredentialsProvider) - .httpClientBuilder(new ApacheSdkHttpService().createHttpClientBuilder()); - - if (StringUtils.isNotBlank(_endpoint)) { - try { - kinesisClientBuilder = kinesisClientBuilder.endpointOverride(new URI(_endpoint)); - } catch (URISyntaxException e) { - throw new IllegalArgumentException("URI syntax is not correctly specified for endpoint: " + _endpoint, e); - } - } + return kinesisClientBuilder.build(); + } - _kinesisClient = kinesisClientBuilder.build(); - } + /** + * Lists all shards of the stream + */ + public List getShards() { + ListShardsResponse listShardsResponse = + _kinesisClient.listShards(ListShardsRequest.builder().streamName(_config.getStreamTopicName()).build()); + return listShardsResponse.shards(); } + @Override public void close() { - if (_kinesisClient != null) { - _kinesisClient.close(); - _kinesisClient = null; - } + _kinesisClient.close(); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumer.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumer.java index 53d0fdde2c88..3d307bfbb25e 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumer.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumer.java @@ -19,10 +19,8 @@ package org.apache.pinot.plugin.stream.kinesis; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; import java.nio.charset.StandardCharsets; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; @@ -30,7 +28,9 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import org.apache.pinot.spi.stream.BytesStreamMessage; import org.apache.pinot.spi.stream.PartitionGroupConsumer; +import org.apache.pinot.spi.stream.StreamMessageMetadata; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,6 +43,7 @@ import software.amazon.awssdk.services.kinesis.model.InvalidArgumentException; import software.amazon.awssdk.services.kinesis.model.KinesisException; import software.amazon.awssdk.services.kinesis.model.ProvisionedThroughputExceededException; +import software.amazon.awssdk.services.kinesis.model.Record; import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; @@ -52,106 +53,58 @@ */ public class KinesisConsumer extends KinesisConnectionHandler implements PartitionGroupConsumer { private static final Logger LOGGER = LoggerFactory.getLogger(KinesisConsumer.class); - public static final long SLEEP_TIME_BETWEEN_REQUESTS = 1000L; - private final String _streamTopicName; - private final int _numMaxRecordsToFetch; - private final ExecutorService _executorService; - private final ShardIteratorType _shardIteratorType; - private final int _rpsLimit; + private static final long SLEEP_TIME_BETWEEN_REQUESTS = 1000L; - public KinesisConsumer(KinesisConfig kinesisConfig) { - super(kinesisConfig); - _streamTopicName = kinesisConfig.getStreamTopicName(); - _numMaxRecordsToFetch = kinesisConfig.getNumMaxRecordsToFetch(); - _shardIteratorType = kinesisConfig.getShardIteratorType(); - _rpsLimit = kinesisConfig.getRpsLimit(); - _executorService = Executors.newSingleThreadExecutor(); + // TODO: Revisit the logic of using a separate executor to manage the request timeout. Currently it is not thread safe + private final ExecutorService _executorService = Executors.newSingleThreadExecutor(); + + public KinesisConsumer(KinesisConfig config) { + super(config); + LOGGER.info("Created Kinesis consumer with topic: {}, RPS limit: {}, max records per fetch: {}", + config.getStreamTopicName(), config.getRpsLimit(), config.getNumMaxRecordsToFetch()); } @VisibleForTesting - public KinesisConsumer(KinesisConfig kinesisConfig, KinesisClient kinesisClient) { - super(kinesisConfig, kinesisClient); - _kinesisClient = kinesisClient; - _streamTopicName = kinesisConfig.getStreamTopicName(); - _numMaxRecordsToFetch = kinesisConfig.getNumMaxRecordsToFetch(); - _shardIteratorType = kinesisConfig.getShardIteratorType(); - _rpsLimit = kinesisConfig.getRpsLimit(); - _executorService = Executors.newSingleThreadExecutor(); + public KinesisConsumer(KinesisConfig config, KinesisClient kinesisClient) { + super(config, kinesisClient); } /** * Fetch records from the Kinesis stream between the start and end KinesisCheckpoint */ @Override - public KinesisRecordsBatch fetchMessages(StreamPartitionMsgOffset startCheckpoint, - StreamPartitionMsgOffset endCheckpoint, int timeoutMs) { - List recordList = new ArrayList<>(); - Future kinesisFetchResultFuture = - _executorService.submit(() -> getResult(startCheckpoint, endCheckpoint, recordList)); - + public KinesisMessageBatch fetchMessages(StreamPartitionMsgOffset startMsgOffset, int timeoutMs) { + KinesisPartitionGroupOffset startOffset = (KinesisPartitionGroupOffset) startMsgOffset; + List messages = new ArrayList<>(); + Future kinesisFetchResultFuture = + _executorService.submit(() -> getResult(startOffset, messages)); try { return kinesisFetchResultFuture.get(timeoutMs, TimeUnit.MILLISECONDS); } catch (TimeoutException e) { kinesisFetchResultFuture.cancel(true); - return handleException((KinesisPartitionGroupOffset) startCheckpoint, recordList); } catch (Exception e) { - return handleException((KinesisPartitionGroupOffset) startCheckpoint, recordList); + // Ignored } + return buildKinesisMessageBatch(startOffset, messages, false); } - private KinesisRecordsBatch getResult(StreamPartitionMsgOffset startOffset, StreamPartitionMsgOffset endOffset, - List recordList) { - KinesisPartitionGroupOffset kinesisStartCheckpoint = (KinesisPartitionGroupOffset) startOffset; - + private KinesisMessageBatch getResult(KinesisPartitionGroupOffset startOffset, List messages) { try { - if (_kinesisClient == null) { - createConnection(); - } - - // TODO: iterate upon all the shardIds in the map - // Okay for now, since we have assumed that every partition group contains a single shard - Map startShardToSequenceMap = kinesisStartCheckpoint.getShardToStartSequenceMap(); - Preconditions.checkState(startShardToSequenceMap.size() == 1, - "Only 1 shard per consumer supported. Found: %s, in startShardToSequenceMap", - startShardToSequenceMap.keySet()); - Map.Entry startShardToSequenceNum = startShardToSequenceMap.entrySet().iterator().next(); - String shardIterator = getShardIterator(startShardToSequenceNum.getKey(), startShardToSequenceNum.getValue()); - - String kinesisEndSequenceNumber = null; - - if (endOffset != null) { - KinesisPartitionGroupOffset kinesisEndCheckpoint = (KinesisPartitionGroupOffset) endOffset; - Map endShardToSequenceMap = kinesisEndCheckpoint.getShardToStartSequenceMap(); - Preconditions.checkState(endShardToSequenceMap.size() == 1, - "Only 1 shard per consumer supported. Found: %s, in endShardToSequenceMap", endShardToSequenceMap.keySet()); - kinesisEndSequenceNumber = endShardToSequenceMap.values().iterator().next(); - } - - String nextStartSequenceNumber; - boolean isEndOfShard = false; + String shardId = startOffset.getShardId(); + String shardIterator = getShardIterator(shardId, startOffset.getSequenceNumber()); + boolean endOfShard = false; long currentWindow = System.currentTimeMillis() / SLEEP_TIME_BETWEEN_REQUESTS; int currentWindowRequests = 0; while (shardIterator != null) { GetRecordsRequest getRecordsRequest = GetRecordsRequest.builder().shardIterator(shardIterator).build(); - long requestSentTime = System.currentTimeMillis() / 1000; GetRecordsResponse getRecordsResponse = _kinesisClient.getRecords(getRecordsRequest); - - if (!getRecordsResponse.records().isEmpty()) { - getRecordsResponse.records().forEach(record -> { - recordList.add( - new KinesisStreamMessage(record.partitionKey().getBytes(StandardCharsets.UTF_8), - record.data().asByteArray(), record.sequenceNumber(), - (KinesisStreamMessageMetadata) _kinesisMetadataExtractor.extract(record), - record.data().asByteArray().length)); - }); - nextStartSequenceNumber = recordList.get(recordList.size() - 1).sequenceNumber(); - - if (kinesisEndSequenceNumber != null && kinesisEndSequenceNumber.compareTo(nextStartSequenceNumber) <= 0) { - break; + List records = getRecordsResponse.records(); + if (!records.isEmpty()) { + for (Record record : records) { + messages.add(extractStreamMessage(record, shardId)); } - - if (recordList.size() >= _numMaxRecordsToFetch) { + if (messages.size() >= _config.getNumMaxRecordsToFetch()) { break; } } @@ -159,7 +112,7 @@ private KinesisRecordsBatch getResult(StreamPartitionMsgOffset startOffset, Stre if (getRecordsResponse.hasChildShards() && !getRecordsResponse.childShards().isEmpty()) { //This statement returns true only when end of current shard has reached. // hasChildShards only checks if the childShard is null and is a valid instance. - isEndOfShard = true; + endOfShard = true; break; } @@ -178,7 +131,7 @@ private KinesisRecordsBatch getResult(StreamPartitionMsgOffset startOffset, Stre currentWindowRequests = 0; } - if (currentWindowRequests >= _rpsLimit) { + if (currentWindowRequests >= _config.getNumMaxRecordsToFetch()) { try { Thread.sleep(SLEEP_TIME_BETWEEN_REQUESTS); } catch (InterruptedException e) { @@ -188,20 +141,16 @@ private KinesisRecordsBatch getResult(StreamPartitionMsgOffset startOffset, Stre } } - return new KinesisRecordsBatch(recordList, startShardToSequenceNum.getKey(), isEndOfShard); + return buildKinesisMessageBatch(startOffset, messages, endOfShard); } catch (IllegalStateException e) { debugOrLogWarning("Illegal state exception, connection is broken", e); - return handleException(kinesisStartCheckpoint, recordList); } catch (ProvisionedThroughputExceededException e) { debugOrLogWarning("The request rate for the stream is too high", e); - return handleException(kinesisStartCheckpoint, recordList); } catch (ExpiredIteratorException e) { debugOrLogWarning("ShardIterator expired while trying to fetch records", e); - return handleException(kinesisStartCheckpoint, recordList); } catch (ResourceNotFoundException | InvalidArgumentException e) { // aws errors LOGGER.error("Encountered AWS error while attempting to fetch records", e); - return handleException(kinesisStartCheckpoint, recordList); } catch (KinesisException e) { debugOrLogWarning("Encountered unknown unrecoverable AWS exception", e); throw new RuntimeException(e); @@ -209,12 +158,12 @@ private KinesisRecordsBatch getResult(StreamPartitionMsgOffset startOffset, Stre if (!(e.getCause() instanceof InterruptedException)) { debugOrLogWarning("Task aborted due to exception", e); } - return handleException(kinesisStartCheckpoint, recordList); } catch (Throwable e) { // non transient errors LOGGER.error("Unknown fetchRecords exception", e); throw new RuntimeException(e); } + return buildKinesisMessageBatch(startOffset, messages, false); } private void debugOrLogWarning(String message, Throwable throwable) { @@ -225,32 +174,48 @@ private void debugOrLogWarning(String message, Throwable throwable) { } } - private KinesisRecordsBatch handleException(KinesisPartitionGroupOffset start, - List recordList) { - String shardId = start.getShardToStartSequenceMap().entrySet().iterator().next().getKey(); - - if (!recordList.isEmpty()) { - String nextStartSequenceNumber = recordList.get(recordList.size() - 1).sequenceNumber(); - Map newCheckpoint = new HashMap<>(start.getShardToStartSequenceMap()); - newCheckpoint.put(newCheckpoint.keySet().iterator().next(), nextStartSequenceNumber); + private KinesisMessageBatch buildKinesisMessageBatch(KinesisPartitionGroupOffset startOffset, + List messages, boolean endOfShard) { + KinesisPartitionGroupOffset offsetOfNextBatch; + if (messages.isEmpty()) { + offsetOfNextBatch = startOffset; + } else { + StreamMessageMetadata lastMessageMetadata = messages.get(messages.size() - 1).getMetadata(); + assert lastMessageMetadata != null; + offsetOfNextBatch = (KinesisPartitionGroupOffset) lastMessageMetadata.getNextOffset(); } - return new KinesisRecordsBatch(recordList, shardId, false); + return new KinesisMessageBatch(messages, offsetOfNextBatch, endOfShard); } private String getShardIterator(String shardId, String sequenceNumber) { GetShardIteratorRequest.Builder requestBuilder = - GetShardIteratorRequest.builder().streamName(_streamTopicName).shardId(shardId); - + GetShardIteratorRequest.builder().streamName(_config.getStreamTopicName()).shardId(shardId); if (sequenceNumber != null) { requestBuilder = requestBuilder.startingSequenceNumber(sequenceNumber) .shardIteratorType(ShardIteratorType.AFTER_SEQUENCE_NUMBER); } else { - requestBuilder = requestBuilder.shardIteratorType(_shardIteratorType); + requestBuilder = requestBuilder.shardIteratorType(_config.getShardIteratorType()); } - return _kinesisClient.getShardIterator(requestBuilder.build()).shardIterator(); } + private BytesStreamMessage extractStreamMessage(Record record, String shardId) { + byte[] key = record.partitionKey().getBytes(StandardCharsets.UTF_8); + byte[] value = record.data().asByteArray(); + long timestamp = record.approximateArrivalTimestamp().toEpochMilli(); + String sequenceNumber = record.sequenceNumber(); + KinesisPartitionGroupOffset offset = new KinesisPartitionGroupOffset(shardId, sequenceNumber); + // NOTE: Use the same offset as next offset because the consumer starts consuming AFTER the start sequence number. + StreamMessageMetadata.Builder builder = + new StreamMessageMetadata.Builder().setRecordIngestionTimeMs(timestamp).setOffset(offset, offset); + if (_config.isPopulateMetadata()) { + builder.setMetadata(Map.of(KinesisStreamMessageMetadata.APPRX_ARRIVAL_TIMESTAMP_KEY, String.valueOf(timestamp), + KinesisStreamMessageMetadata.SEQUENCE_NUMBER_KEY, sequenceNumber)); + } + StreamMessageMetadata metadata = builder.build(); + return new BytesStreamMessage(key, value, metadata); + } + @Override public void close() { super.close(); diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerFactory.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerFactory.java index 6f81632c8c64..bd7d9ad8c09f 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerFactory.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerFactory.java @@ -20,7 +20,6 @@ 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.StreamMetadataProvider; import org.apache.pinot.spi.stream.StreamPartitionMsgOffsetFactory; @@ -31,10 +30,6 @@ */ public class KinesisConsumerFactory extends StreamConsumerFactory { - @Override - public PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int partition) { - throw new UnsupportedOperationException(); - } @Override public StreamMetadataProvider createPartitionMetadataProvider(String clientId, int partition) { return new KinesisStreamMetadataProvider(clientId, _streamConfig); diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMessageBatch.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMessageBatch.java new file mode 100644 index 000000000000..866e99594fe4 --- /dev/null +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMessageBatch.java @@ -0,0 +1,61 @@ +/** + * 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.kinesis; + +import java.util.List; +import org.apache.pinot.spi.stream.BytesStreamMessage; +import org.apache.pinot.spi.stream.MessageBatch; +import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; + + +/** + * A {@link MessageBatch} for collecting records from the Kinesis stream + */ +public class KinesisMessageBatch implements MessageBatch { + private final List _messages; + private final KinesisPartitionGroupOffset _offsetOfNextBatch; + private final boolean _endOfShard; + + public KinesisMessageBatch(List messages, KinesisPartitionGroupOffset offsetOfNextBatch, + boolean endOfShard) { + _messages = messages; + _offsetOfNextBatch = offsetOfNextBatch; + _endOfShard = endOfShard; + } + + @Override + public int getMessageCount() { + return _messages.size(); + } + + @Override + public BytesStreamMessage getStreamMessage(int index) { + return _messages.get(index); + } + + @Override + public StreamPartitionMsgOffset getOffsetOfNextBatch() { + return _offsetOfNextBatch; + } + + @Override + public boolean isEndOfPartitionGroup() { + return _endOfShard; + } +} diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMetadataExtractor.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMetadataExtractor.java deleted file mode 100644 index 963ff8f689ec..000000000000 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMetadataExtractor.java +++ /dev/null @@ -1,45 +0,0 @@ -/** - * 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.kinesis; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import org.apache.pinot.spi.stream.RowMetadata; -import software.amazon.awssdk.services.kinesis.model.Record; - -// TODO: Make this an interface/api in the stream spis st StreamMessageMetadata extract(T streamTypeRecord) - -public interface KinesisMetadataExtractor { - static KinesisMetadataExtractor build(boolean populateMetadata) { - return record -> { - long recordTimestamp = record.approximateArrivalTimestamp().toEpochMilli(); - - if (!populateMetadata) { - return new KinesisStreamMessageMetadata(recordTimestamp, null, Collections.emptyMap()); - } - String seqNumber = record.sequenceNumber(); - Map metadataMap = new HashMap<>(); - metadataMap.put(KinesisStreamMessageMetadata.APPRX_ARRIVAL_TIMESTAMP_KEY, String.valueOf(recordTimestamp)); - metadataMap.put(KinesisStreamMessageMetadata.SEQUENCE_NUMBER_KEY, seqNumber); - return new KinesisStreamMessageMetadata(recordTimestamp, null, metadataMap); - }; - } - RowMetadata extract(Record record); -} diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMsgOffsetFactory.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMsgOffsetFactory.java index f745b114ab46..5f17badfe622 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMsgOffsetFactory.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisMsgOffsetFactory.java @@ -18,7 +18,6 @@ */ package org.apache.pinot.plugin.stream.kinesis; -import java.io.IOException; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; import org.apache.pinot.spi.stream.StreamPartitionMsgOffsetFactory; @@ -35,16 +34,6 @@ public void init(StreamConfig streamConfig) { @Override public StreamPartitionMsgOffset create(String offsetStr) { - try { - return new KinesisPartitionGroupOffset(offsetStr); - } catch (IOException e) { - throw new IllegalStateException( - "Caught exception when creating KinesisPartitionGroupOffset from offsetStr: " + offsetStr); - } - } - - @Override - public StreamPartitionMsgOffset create(StreamPartitionMsgOffset other) { - return new KinesisPartitionGroupOffset(((KinesisPartitionGroupOffset) other).getShardToStartSequenceMap()); + return new KinesisPartitionGroupOffset(offsetStr); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisPartitionGroupOffset.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisPartitionGroupOffset.java index 12af4765fd03..e68c84e82e1d 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisPartitionGroupOffset.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisPartitionGroupOffset.java @@ -18,20 +18,17 @@ */ package org.apache.pinot.plugin.stream.kinesis; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.ObjectReader; -import com.fasterxml.jackson.databind.ObjectWriter; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.common.base.Preconditions; -import java.io.IOException; import java.util.Map; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; +import org.apache.pinot.spi.utils.JsonUtils; + /** * A {@link StreamPartitionMsgOffset} implementation for the Kinesis partition group consumption - * A partition group consists of 1 or more shards. The KinesisCheckpoint maintains a Map of shards to the - * sequenceNumber. + * A partition group consists of 1 shard. The KinesisCheckpoint maintains the shardId and sequenceNumber. * The sequenceNumber is the id (equivalent to offset in kafka) for the messages in the shard. * From the Kinesis documentation: * Each data record has a sequence number that is unique per partition-key within its shard. @@ -41,69 +38,41 @@ * The longer the time period between write requests, the larger the sequence numbers become. */ public class KinesisPartitionGroupOffset implements StreamPartitionMsgOffset { - private static final ObjectMapper DEFAULT_MAPPER = new ObjectMapper(); - public static final ObjectReader DEFAULT_READER = DEFAULT_MAPPER.reader(); - public static final ObjectWriter DEFAULT_WRITER = DEFAULT_MAPPER.writer(); - - private final Map _shardToStartSequenceMap; - - public KinesisPartitionGroupOffset(Map shardToStartSequenceMap) { - _shardToStartSequenceMap = shardToStartSequenceMap; - } + private final String _shardId; + private final String _sequenceNumber; - public KinesisPartitionGroupOffset(String offsetStr) - throws IOException { - _shardToStartSequenceMap = stringToObject(offsetStr, new TypeReference>() { - }); + public KinesisPartitionGroupOffset(String shardId, String sequenceNumber) { + _shardId = shardId; + _sequenceNumber = sequenceNumber; } - public Map getShardToStartSequenceMap() { - return _shardToStartSequenceMap; - } - - @Override - public String toString() { + public KinesisPartitionGroupOffset(String offsetStr) { try { - return objectToString(_shardToStartSequenceMap); - } catch (JsonProcessingException e) { - throw new IllegalStateException( - "Caught exception when converting KinesisCheckpoint to string: " + _shardToStartSequenceMap); + ObjectNode objectNode = (ObjectNode) JsonUtils.stringToJsonNode(offsetStr); + Preconditions.checkArgument(objectNode.size() == 1); + Map.Entry entry = objectNode.fields().next(); + _shardId = entry.getKey(); + _sequenceNumber = entry.getValue().asText(); + } catch (Exception e) { + throw new IllegalArgumentException("Invalid Kinesis offset: " + offsetStr); } } - @Override - public KinesisPartitionGroupOffset fromString(String kinesisCheckpointStr) { - try { - return new KinesisPartitionGroupOffset(kinesisCheckpointStr); - } catch (IOException e) { - throw new IllegalStateException( - "Caught exception when converting string to KinesisCheckpoint: " + kinesisCheckpointStr); - } + public String getShardId() { + return _shardId; } - @Override - public int compareTo(StreamPartitionMsgOffset o) { - Preconditions.checkNotNull(o); - KinesisPartitionGroupOffset other = (KinesisPartitionGroupOffset) o; - Preconditions.checkNotNull(other._shardToStartSequenceMap); - Preconditions.checkNotNull(_shardToStartSequenceMap); - Preconditions - .checkState(other._shardToStartSequenceMap.size() == 1, "Only 1 shard per consumer supported. Found: %s", - other._shardToStartSequenceMap); - Preconditions - .checkState(_shardToStartSequenceMap.size() == 1, "Only 1 shard per consumer supported. Found: %s", - _shardToStartSequenceMap); - return _shardToStartSequenceMap.values().iterator().next() - .compareTo(other._shardToStartSequenceMap.values().iterator().next()); + public String getSequenceNumber() { + return _sequenceNumber; } - public static T stringToObject(String jsonString, TypeReference valueTypeRef) - throws IOException { - return DEFAULT_READER.forType(valueTypeRef).readValue(jsonString); + @Override + public String toString() { + return JsonUtils.newObjectNode().put(_shardId, _sequenceNumber).toString(); } - public static String objectToString(Object object) - throws JsonProcessingException { - return DEFAULT_WRITER.writeValueAsString(object); + @Override + public int compareTo(StreamPartitionMsgOffset other) { + return _sequenceNumber.compareTo(((KinesisPartitionGroupOffset) other)._sequenceNumber); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisRecordsBatch.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisRecordsBatch.java deleted file mode 100644 index 3b6acb5542c6..000000000000 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisRecordsBatch.java +++ /dev/null @@ -1,96 +0,0 @@ -/** - * 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.kinesis; - -import java.nio.ByteBuffer; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.RowMetadata; -import org.apache.pinot.spi.stream.StreamMessage; -import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; - - -/** - * A {@link MessageBatch} for collecting records from the Kinesis stream - */ -public class KinesisRecordsBatch implements MessageBatch { - private final List _recordList; - private final String _shardId; - private final boolean _endOfShard; - - public KinesisRecordsBatch(List recordList, String shardId, boolean endOfShard) { - _recordList = recordList; - _shardId = shardId; - _endOfShard = endOfShard; - } - - @Override - public int getMessageCount() { - return _recordList.size(); - } - - @Override - public KinesisStreamMessage getMessageAtIndex(int index) { - return _recordList.get(index); - } - - @Override - public byte[] getMessageBytesAtIndex(int index) { - return _recordList.get(index).getValue(); - } - - @Override - public int getMessageOffsetAtIndex(int index) { - return ByteBuffer.wrap(_recordList.get(index).getValue()).arrayOffset(); - } - - @Override - public int getMessageLengthAtIndex(int index) { - return _recordList.get(index).getValue().length; - } - - @Override - public StreamPartitionMsgOffset getNextStreamPartitionMsgOffsetAtIndex(int index) { - Map shardToSequenceMap = new HashMap<>(); - shardToSequenceMap.put(_shardId, _recordList.get(index).sequenceNumber()); - return new KinesisPartitionGroupOffset(shardToSequenceMap); - } - - @Override - public long getNextStreamMessageOffsetAtIndex(int index) { - throw new UnsupportedOperationException(); - } - - @Override - public boolean isEndOfPartitionGroup() { - return _endOfShard; - } - - @Override - public RowMetadata getMetadataAtIndex(int index) { - return _recordList.get(index).getMetadata(); - } - - @Override - public StreamMessage getStreamMessage(int index) { - return _recordList.get(index); - } -} diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMessageMetadata.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMessageMetadata.java index 61c55efe6252..cbabaf608a09 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMessageMetadata.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMessageMetadata.java @@ -24,15 +24,17 @@ import org.apache.pinot.spi.stream.StreamMessageMetadata; +// TODO: Make it a util class public class KinesisStreamMessageMetadata extends StreamMessageMetadata { public static final String APPRX_ARRIVAL_TIMESTAMP_KEY = "apprxArrivalTimestamp"; public static final String SEQUENCE_NUMBER_KEY = "sequenceNumber"; - public KinesisStreamMessageMetadata(long recordIngestionTimeMs, - @Nullable GenericRow headers) { + @Deprecated + public KinesisStreamMessageMetadata(long recordIngestionTimeMs, @Nullable GenericRow headers) { super(recordIngestionTimeMs, headers); } + @Deprecated public KinesisStreamMessageMetadata(long recordIngestionTimeMs, @Nullable GenericRow headers, Map metadata) { super(recordIngestionTimeMs, headers, metadata); diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProvider.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProvider.java index 96bc0502c6bd..4a6617ad0923 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProvider.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProvider.java @@ -29,7 +29,6 @@ import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; import org.apache.pinot.spi.stream.ConsumerPartitionState; -import org.apache.pinot.spi.stream.MessageBatch; import org.apache.pinot.spi.stream.OffsetCriteria; import org.apache.pinot.spi.stream.PartitionGroupConsumer; import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; @@ -108,12 +107,12 @@ public List computePartitionGroupMetadata(String clientI for (PartitionGroupConsumptionStatus currentPartitionGroupConsumptionStatus : partitionGroupConsumptionStatuses) { KinesisPartitionGroupOffset kinesisStartCheckpoint = (KinesisPartitionGroupOffset) currentPartitionGroupConsumptionStatus.getStartOffset(); - String shardId = kinesisStartCheckpoint.getShardToStartSequenceMap().keySet().iterator().next(); + String shardId = kinesisStartCheckpoint.getShardId(); shardsInCurrent.add(shardId); Shard shard = shardIdToShardMap.get(shardId); if (shard == null) { // Shard has expired shardsEnded.add(shardId); - String lastConsumedSequenceID = kinesisStartCheckpoint.getShardToStartSequenceMap().get(shardId); + String lastConsumedSequenceID = kinesisStartCheckpoint.getSequenceNumber(); LOGGER.warn("Kinesis shard with id: " + shardId + " has expired. Data has been consumed from the shard till sequence number: " + lastConsumedSequenceID + ". There can be potential data loss."); @@ -145,7 +144,6 @@ public List computePartitionGroupMetadata(String clientI if (shardsInCurrent.contains(newShardId)) { continue; } - StreamPartitionMsgOffset newStartOffset; Shard newShard = entry.getValue(); String parentShardId = newShard.parentShardId(); @@ -155,9 +153,10 @@ public List computePartitionGroupMetadata(String clientI // 3. Parent reached EOL and completely consumed. if (parentShardId == null || !shardIdToShardMap.containsKey(parentShardId) || shardsEnded.contains( parentShardId)) { - Map shardToSequenceNumberMap = new HashMap<>(); - shardToSequenceNumberMap.put(newShardId, newShard.sequenceNumberRange().startingSequenceNumber()); - newStartOffset = new KinesisPartitionGroupOffset(shardToSequenceNumberMap); + // TODO: Revisit this. Kinesis starts consuming after the start sequence number, and we might miss the first + // message. + StreamPartitionMsgOffset newStartOffset = + new KinesisPartitionGroupOffset(newShardId, newShard.sequenceNumberRange().startingSequenceNumber()); int partitionGroupId = getPartitionGroupIdFromShardId(newShardId); newPartitionGroupMetadataList.add(new PartitionGroupMetadata(partitionGroupId, newStartOffset)); } @@ -179,30 +178,24 @@ private int getPartitionGroupIdFromShardId(String shardId) { private boolean consumedEndOfShard(StreamPartitionMsgOffset startCheckpoint, PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) throws IOException, TimeoutException { - PartitionGroupConsumer partitionGroupConsumer = - _kinesisStreamConsumerFactory.createPartitionGroupConsumer(_clientId, partitionGroupConsumptionStatus); - - MessageBatch messageBatch; - try { - messageBatch = partitionGroupConsumer.fetchMessages(startCheckpoint, null, _fetchTimeoutMs); - } finally { - partitionGroupConsumer.close(); + try (PartitionGroupConsumer partitionGroupConsumer = _kinesisStreamConsumerFactory.createPartitionGroupConsumer( + _clientId, partitionGroupConsumptionStatus)) { + return partitionGroupConsumer.fetchMessages(startCheckpoint, _fetchTimeoutMs).isEndOfPartitionGroup(); } - return messageBatch.isEndOfPartitionGroup(); } @Override public Map getCurrentPartitionLagState( Map currentPartitionStateMap) { Map perPartitionLag = new HashMap<>(); - for (Map.Entry entry: currentPartitionStateMap.entrySet()) { + for (Map.Entry entry : currentPartitionStateMap.entrySet()) { ConsumerPartitionState partitionState = entry.getValue(); // Compute record-availability String recordAvailabilityLag = "UNKNOWN"; RowMetadata lastProcessedMessageMetadata = partitionState.getLastProcessedRowMetadata(); if (lastProcessedMessageMetadata != null && partitionState.getLastProcessedTimeMs() > 0) { - long availabilityLag = partitionState.getLastProcessedTimeMs() - - lastProcessedMessageMetadata.getRecordIngestionTimeMs(); + long availabilityLag = + partitionState.getLastProcessedTimeMs() - lastProcessedMessageMetadata.getRecordIngestionTimeMs(); recordAvailabilityLag = String.valueOf(availabilityLag); } perPartitionLag.put(entry.getKey(), new KinesisConsumerPartitionLag(recordAvailabilityLag)); @@ -211,5 +204,6 @@ public Map getCurrentPartitionLagState( } @Override - public void close() { } + public void close() { + } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerTest.java index 5d620de604e1..95871cfa368b 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerTest.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerTest.java @@ -25,9 +25,7 @@ import java.util.Map; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.stream.StreamConfigProperties; -import org.apache.pinot.spi.stream.StreamConsumerFactory; import org.easymock.Capture; -import org.testng.Assert; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; import software.amazon.awssdk.core.SdkBytes; @@ -44,6 +42,9 @@ import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.replay; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; public class KinesisConsumerTest { @@ -58,19 +59,17 @@ public class KinesisConsumerTest { private static final String PLACEHOLDER = "DUMMY"; private static final int MAX_RECORDS_TO_FETCH = 20; - private KinesisConnectionHandler _kinesisConnectionHandler; - private StreamConsumerFactory _streamConsumerFactory; private KinesisClient _kinesisClient; - private List _recordList; + private KinesisConfig _kinesisConfig; + private List _records; private KinesisConfig getKinesisConfig() { Map props = new HashMap<>(); props.put(StreamConfigProperties.STREAM_TYPE, STREAM_TYPE); props.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, StreamConfigProperties.STREAM_TOPIC_NAME), STREAM_NAME); - props.put(StreamConfigProperties - .constructStreamProperty(STREAM_TYPE, StreamConfigProperties.STREAM_CONSUMER_FACTORY_CLASS), - KinesisConsumerFactory.class.getName()); + props.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, + StreamConfigProperties.STREAM_CONSUMER_FACTORY_CLASS), KinesisConsumerFactory.class.getName()); props.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, StreamConfigProperties.STREAM_DECODER_CLASS), "org.apache.pinot.plugin.inputformat.json.JSONMessageDecoder"); props.put(KinesisConfig.REGION, AWS_REGION); @@ -81,18 +80,14 @@ private KinesisConfig getKinesisConfig() { @BeforeMethod public void setupTest() { - _kinesisConnectionHandler = createMock(KinesisConnectionHandler.class); _kinesisClient = createMock(KinesisClient.class); - _streamConsumerFactory = createMock(StreamConsumerFactory.class); - - _recordList = new ArrayList<>(); - + _kinesisConfig = getKinesisConfig(); + _records = new ArrayList<>(NUM_RECORDS); for (int i = 0; i < NUM_RECORDS; i++) { Record record = Record.builder().data(SdkBytes.fromUtf8String(DUMMY_RECORD_PREFIX + i)).partitionKey(PARTITION_KEY_PREFIX + i) - .approximateArrivalTimestamp(Instant.now()) - .sequenceNumber(String.valueOf(i + 1)).build(); - _recordList.add(record); + .approximateArrivalTimestamp(Instant.now()).sequenceNumber(String.valueOf(i + 1)).build(); + _records.add(record); } } @@ -102,7 +97,7 @@ public void testBasicConsumer() { Capture getShardIteratorRequestCapture = Capture.newInstance(); GetRecordsResponse getRecordsResponse = - GetRecordsResponse.builder().nextShardIterator(null).records(_recordList).build(); + GetRecordsResponse.builder().nextShardIterator(null).records(_records).build(); GetShardIteratorResponse getShardIteratorResponse = GetShardIteratorResponse.builder().shardIterator(PLACEHOLDER).build(); @@ -112,56 +107,21 @@ public void testBasicConsumer() { replay(_kinesisClient); - KinesisConsumer kinesisConsumer = new KinesisConsumer(getKinesisConfig(), _kinesisClient); + KinesisConsumer kinesisConsumer = new KinesisConsumer(_kinesisConfig, _kinesisClient); + KinesisPartitionGroupOffset startOffset = new KinesisPartitionGroupOffset("0", "1"); + KinesisMessageBatch kinesisMessageBatch = kinesisConsumer.fetchMessages(startOffset, TIMEOUT); - Map shardToSequenceMap = new HashMap<>(); - shardToSequenceMap.put("0", "1"); - KinesisPartitionGroupOffset kinesisPartitionGroupOffset = new KinesisPartitionGroupOffset(shardToSequenceMap); - KinesisRecordsBatch kinesisRecordsBatch = kinesisConsumer.fetchMessages(kinesisPartitionGroupOffset, null, TIMEOUT); - - Assert.assertEquals(kinesisRecordsBatch.getMessageCount(), NUM_RECORDS); + assertEquals(kinesisMessageBatch.getMessageCount(), NUM_RECORDS); for (int i = 0; i < NUM_RECORDS; i++) { - Assert.assertEquals(baToString(kinesisRecordsBatch.getMessageAtIndex(i).getValue()), DUMMY_RECORD_PREFIX + i); + assertEquals(baToString(kinesisMessageBatch.getStreamMessage(i).getValue()), DUMMY_RECORD_PREFIX + i); } - Assert.assertFalse(kinesisRecordsBatch.isEndOfPartitionGroup()); - } - - @Test - public void testBasicConsumerWithMaxRecordsLimit() { - Capture getRecordsRequestCapture = Capture.newInstance(); - Capture getShardIteratorRequestCapture = Capture.newInstance(); - - GetRecordsResponse getRecordsResponse = - GetRecordsResponse.builder().nextShardIterator(PLACEHOLDER).records(_recordList).build(); - GetShardIteratorResponse getShardIteratorResponse = - GetShardIteratorResponse.builder().shardIterator(PLACEHOLDER).build(); - - expect(_kinesisClient.getRecords(capture(getRecordsRequestCapture))).andReturn(getRecordsResponse).anyTimes(); - expect(_kinesisClient.getShardIterator(capture(getShardIteratorRequestCapture))).andReturn(getShardIteratorResponse) - .anyTimes(); - - replay(_kinesisClient); - - KinesisConfig kinesisConfig = getKinesisConfig(); - KinesisConsumer kinesisConsumer = new KinesisConsumer(kinesisConfig, _kinesisClient); - - Map shardToSequenceMap = new HashMap<>(); - shardToSequenceMap.put("0", "1"); - KinesisPartitionGroupOffset kinesisPartitionGroupOffset = new KinesisPartitionGroupOffset(shardToSequenceMap); - KinesisRecordsBatch kinesisRecordsBatch = kinesisConsumer.fetchMessages(kinesisPartitionGroupOffset, null, TIMEOUT); - - Assert.assertEquals(kinesisRecordsBatch.getMessageCount(), MAX_RECORDS_TO_FETCH); - - for (int i = 0; i < NUM_RECORDS; i++) { - Assert.assertEquals(baToString(kinesisRecordsBatch.getMessageAtIndex(i).getValue()), DUMMY_RECORD_PREFIX + i); - } + assertFalse(kinesisMessageBatch.isEndOfPartitionGroup()); } @Test public void testBasicConsumerWithChildShard() { - List shardList = new ArrayList<>(); shardList.add(ChildShard.builder().shardId(PLACEHOLDER).parentShards("0").build()); @@ -169,7 +129,7 @@ public void testBasicConsumerWithChildShard() { Capture getShardIteratorRequestCapture = Capture.newInstance(); GetRecordsResponse getRecordsResponse = - GetRecordsResponse.builder().nextShardIterator(null).records(_recordList).childShards(shardList).build(); + GetRecordsResponse.builder().nextShardIterator(null).records(_records).childShards(shardList).build(); GetShardIteratorResponse getShardIteratorResponse = GetShardIteratorResponse.builder().shardIterator(PLACEHOLDER).build(); @@ -179,19 +139,15 @@ public void testBasicConsumerWithChildShard() { replay(_kinesisClient); - KinesisConfig kinesisConfig = getKinesisConfig(); - KinesisConsumer kinesisConsumer = new KinesisConsumer(kinesisConfig, _kinesisClient); - - Map shardToSequenceMap = new HashMap<>(); - shardToSequenceMap.put("0", "1"); - KinesisPartitionGroupOffset kinesisPartitionGroupOffset = new KinesisPartitionGroupOffset(shardToSequenceMap); - KinesisRecordsBatch kinesisRecordsBatch = kinesisConsumer.fetchMessages(kinesisPartitionGroupOffset, null, TIMEOUT); + KinesisConsumer kinesisConsumer = new KinesisConsumer(_kinesisConfig, _kinesisClient); + KinesisPartitionGroupOffset startOffset = new KinesisPartitionGroupOffset("0", "1"); + KinesisMessageBatch kinesisMessageBatch = kinesisConsumer.fetchMessages(startOffset, TIMEOUT); - Assert.assertTrue(kinesisRecordsBatch.isEndOfPartitionGroup()); - Assert.assertEquals(kinesisRecordsBatch.getMessageCount(), NUM_RECORDS); + assertTrue(kinesisMessageBatch.isEndOfPartitionGroup()); + assertEquals(kinesisMessageBatch.getMessageCount(), NUM_RECORDS); for (int i = 0; i < NUM_RECORDS; i++) { - Assert.assertEquals(baToString(kinesisRecordsBatch.getMessageAtIndex(i).getValue()), DUMMY_RECORD_PREFIX + i); + assertEquals(baToString(kinesisMessageBatch.getStreamMessage(i).getValue()), DUMMY_RECORD_PREFIX + i); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisMessageBatchTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisMessageBatchTest.java new file mode 100644 index 000000000000..4e64dae54961 --- /dev/null +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisMessageBatchTest.java @@ -0,0 +1,84 @@ +/** + * 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.kinesis; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import org.apache.pinot.spi.stream.BytesStreamMessage; +import org.apache.pinot.spi.stream.StreamMessageMetadata; +import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; +import org.testng.annotations.Test; + +import static org.testng.Assert.*; + + +public class KinesisMessageBatchTest { + private static final String SHARD_ID = "shard-0000000000"; + + @Test + public void testMessageBatch() { + int numMessages = 5; + long baseTimeMs = System.currentTimeMillis(); + List messages = new ArrayList<>(numMessages); + for (int i = 0; i < numMessages; i++) { + messages.add(createStreamMessage(i, "key-" + i, "value-" + i, baseTimeMs + i)); + } + KinesisMessageBatch batch = + new KinesisMessageBatch(messages, new KinesisPartitionGroupOffset(SHARD_ID, Integer.toString(numMessages - 1)), + false); + + for (int i = 0; i < numMessages; i++) { + BytesStreamMessage streamMessage = batch.getStreamMessage(i); + byte[] key = streamMessage.getKey(); + assertNotNull(key); + assertEquals(new String(key, StandardCharsets.UTF_8), "key-" + i); + byte[] value = streamMessage.getValue(); + assertEquals(new String(value, StandardCharsets.UTF_8), "value-" + i); + StreamMessageMetadata metadata = streamMessage.getMetadata(); + assertNotNull(metadata); + assertEquals(metadata.getRecordIngestionTimeMs(), baseTimeMs + i); + StreamPartitionMsgOffset offset = metadata.getOffset(); + assertTrue(offset instanceof KinesisPartitionGroupOffset); + assertEquals(((KinesisPartitionGroupOffset) offset).getShardId(), SHARD_ID); + assertEquals(((KinesisPartitionGroupOffset) offset).getSequenceNumber(), Integer.toString(i)); + assertSame(metadata.getNextOffset(), offset); + } + + // Batch level operations + assertEquals(batch.getMessageCount(), numMessages); + assertEquals(batch.getUnfilteredMessageCount(), numMessages); + assertFalse(batch.isEndOfPartitionGroup()); + StreamPartitionMsgOffset nextBatchOffset = batch.getOffsetOfNextBatch(); + assertTrue(nextBatchOffset instanceof KinesisPartitionGroupOffset); + assertEquals(((KinesisPartitionGroupOffset) nextBatchOffset).getShardId(), SHARD_ID); + assertEquals(((KinesisPartitionGroupOffset) nextBatchOffset).getSequenceNumber(), + Integer.toString(numMessages - 1)); + } + + private static BytesStreamMessage createStreamMessage(int sequenceNumber, String key, String value, + long recordIngestionTimeMs) { + KinesisPartitionGroupOffset offset = new KinesisPartitionGroupOffset(SHARD_ID, Integer.toString(sequenceNumber)); + StreamMessageMetadata metadata = + new StreamMessageMetadata.Builder().setRecordIngestionTimeMs(recordIngestionTimeMs).setOffset(offset, offset) + .build(); + return new BytesStreamMessage(key.getBytes(StandardCharsets.UTF_8), value.getBytes(StandardCharsets.UTF_8), + metadata); + } +} diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisRecordsBatchTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisRecordsBatchTest.java deleted file mode 100644 index 83d7ded36e14..000000000000 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisRecordsBatchTest.java +++ /dev/null @@ -1,85 +0,0 @@ -/** - * 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.kinesis; - -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.List; -import org.testng.Assert; -import org.testng.annotations.Test; - - -public class KinesisRecordsBatchTest { - private static final byte[] EMPTY_BYTE_ARRAY = new byte[0]; - - @Test - public void testMessageBatchAPIs() { - int msgCount = 5; - long baseTimeMs = System.currentTimeMillis(); - List msgList = new ArrayList<>(msgCount); - - - for (int i = 0; i < msgCount; i++) { - msgList.add(createStreamMessage(i, "key-" + i, "value-" + i, baseTimeMs + i)); - } - KinesisRecordsBatch batch = new KinesisRecordsBatch(msgList, "shard-0000000000", false); - - for (int i = 0; i < msgCount; i++) { - Assert.assertEquals(batch.getMessageLengthAtIndex(i), 7); // length of characters in "value-$i" - Assert.assertEquals(batch.getMessageOffsetAtIndex(i), 0); - - KinesisPartitionGroupOffset kinesisPartitionGroupOffset = - (KinesisPartitionGroupOffset) batch.getNextStreamPartitionMsgOffsetAtIndex(i); - Assert.assertNotNull(kinesisPartitionGroupOffset); - Assert.assertNotNull(kinesisPartitionGroupOffset.getShardToStartSequenceMap().get("shard-0000000000"), - String.valueOf(i)); // why is "next" stream partition msg offset return the exact offset at index i ? - - KinesisStreamMessageMetadata metadata = (KinesisStreamMessageMetadata) batch.getMetadataAtIndex(i); - Assert.assertNotNull(metadata); - Assert.assertEquals(metadata.getRecordIngestionTimeMs(), baseTimeMs + i); - Assert.assertEquals(batch.getMessageBytesAtIndex(i), ("value-" + i).getBytes(StandardCharsets.UTF_8)); - } - - // Batch level operations - Assert.assertEquals(batch.getMessageCount(), msgCount); - Assert.assertFalse(batch.isEndOfPartitionGroup()); - - KinesisPartitionGroupOffset nextBatchOffset = (KinesisPartitionGroupOffset) batch.getOffsetOfNextBatch(); - Assert.assertNotNull(nextBatchOffset); - Assert.assertNotNull(nextBatchOffset.getShardToStartSequenceMap().get("shard-0000000000"), - String.valueOf(msgCount - 1)); // default implementation doesn't make sense ?? - Assert.assertEquals(batch.getUnfilteredMessageCount(), msgCount); // always size of the batch for Kinesis - - // unsupported operations for kinesis batch - Assert.assertThrows(UnsupportedOperationException.class, () -> { - batch.getNextStreamMessageOffsetAtIndex(0); - }); - - // batch.getMessageAtIndex(i); // deprecated - } - - private static KinesisStreamMessage createStreamMessage(int sequenceNumber, String key, String value, - long recordIngestionTimeMs) { - KinesisStreamMessageMetadata metadata = new KinesisStreamMessageMetadata(recordIngestionTimeMs, null); - byte[] valueArray = value != null ? value.getBytes(StandardCharsets.UTF_8) : EMPTY_BYTE_ARRAY; - return new KinesisStreamMessage( - key != null ? key.getBytes(StandardCharsets.UTF_8) : EMPTY_BYTE_ARRAY, valueArray, - String.valueOf(sequenceNumber), metadata, valueArray.length); - } -} diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProviderTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProviderTest.java index 8d9288e2cf92..833f84d45b03 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProviderTest.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProviderTest.java @@ -103,9 +103,7 @@ public void getPartitionsGroupInfoEndOfShardTest() throws Exception { List currentPartitionGroupMeta = new ArrayList<>(); - Map shardToSequenceMap = new HashMap<>(); - shardToSequenceMap.put("0", "1"); - KinesisPartitionGroupOffset kinesisPartitionGroupOffset = new KinesisPartitionGroupOffset(shardToSequenceMap); + KinesisPartitionGroupOffset kinesisPartitionGroupOffset = new KinesisPartitionGroupOffset("0", "1"); currentPartitionGroupMeta.add(new PartitionGroupConsumptionStatus(0, 1, kinesisPartitionGroupOffset, kinesisPartitionGroupOffset, "CONSUMING")); @@ -123,9 +121,8 @@ public void getPartitionsGroupInfoEndOfShardTest() expect(_streamConsumerFactory .createPartitionGroupConsumer(capture(stringCapture), capture(partitionGroupMetadataCapture))) .andReturn(_partitionGroupConsumer).anyTimes(); - expect(_partitionGroupConsumer - .fetchMessages(capture(checkpointArgs), capture(checkpointArgs), captureInt(intArguments))) - .andReturn(new KinesisRecordsBatch(new ArrayList<>(), "0", true)).anyTimes(); + expect(_partitionGroupConsumer.fetchMessages(capture(checkpointArgs), captureInt(intArguments))).andReturn( + new KinesisMessageBatch(new ArrayList<>(), kinesisPartitionGroupOffset, true)).anyTimes(); replay(_kinesisConnectionHandler, _streamConsumerFactory, _partitionGroupConsumer); @@ -143,7 +140,7 @@ public void getPartitionsGroupInfoChildShardsest() Map shardToSequenceMap = new HashMap<>(); shardToSequenceMap.put("1", "1"); - KinesisPartitionGroupOffset kinesisPartitionGroupOffset = new KinesisPartitionGroupOffset(shardToSequenceMap); + KinesisPartitionGroupOffset kinesisPartitionGroupOffset = new KinesisPartitionGroupOffset("1", "1"); currentPartitionGroupMeta.add(new PartitionGroupConsumptionStatus(0, 1, kinesisPartitionGroupOffset, kinesisPartitionGroupOffset, "CONSUMING")); @@ -162,9 +159,8 @@ public void getPartitionsGroupInfoChildShardsest() expect(_streamConsumerFactory .createPartitionGroupConsumer(capture(stringCapture), capture(partitionGroupMetadataCapture))) .andReturn(_partitionGroupConsumer).anyTimes(); - expect(_partitionGroupConsumer - .fetchMessages(capture(checkpointArgs), capture(checkpointArgs), captureInt(intArguments))) - .andReturn(new KinesisRecordsBatch(new ArrayList<>(), "0", true)).anyTimes(); + expect(_partitionGroupConsumer.fetchMessages(capture(checkpointArgs), captureInt(intArguments))).andReturn( + new KinesisMessageBatch(new ArrayList<>(), kinesisPartitionGroupOffset, true)).anyTimes(); replay(_kinesisConnectionHandler, _streamConsumerFactory, _partitionGroupConsumer); diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml index c099839ec5e3..9ad9a4b62e64 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml @@ -51,6 +51,11 @@ 1.19.7 test + + org.mockito + mockito-core + test + org.apache.pulsar pulsar-client-original diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/MessageIdStreamOffset.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/MessageIdStreamOffset.java index d8e07658e1fa..b322d5c084a1 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/MessageIdStreamOffset.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/MessageIdStreamOffset.java @@ -18,35 +18,26 @@ */ package org.apache.pinot.plugin.stream.pulsar; -import java.io.IOException; import org.apache.commons.codec.binary.Hex; 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 static final Logger LOGGER = LoggerFactory.getLogger(MessageIdStreamOffset.class); - private MessageId _messageId; + private final 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(Hex.decodeHex(messageId)); } catch (Exception e) { - LOGGER.warn("Cannot parse message id " + messageId, e); + throw new IllegalArgumentException("Invalid Pulsar message id: " + messageId); } } @@ -54,15 +45,9 @@ public MessageId getMessageId() { return _messageId; } - @Override - public StreamPartitionMsgOffset fromString(String streamPartitionMsgOffsetStr) { - return new MessageIdStreamOffset(streamPartitionMsgOffsetStr); - } - @Override public int compareTo(StreamPartitionMsgOffset other) { - MessageIdStreamOffset messageIdStreamOffset = (MessageIdStreamOffset) other; - return _messageId.compareTo(messageIdStreamOffset.getMessageId()); + return _messageId.compareTo(((MessageIdStreamOffset) other).getMessageId()); } @Override diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/MessageIdStreamOffsetFactory.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/MessageIdStreamOffsetFactory.java index 69c1c5323858..74009048d596 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/MessageIdStreamOffsetFactory.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/MessageIdStreamOffsetFactory.java @@ -27,21 +27,13 @@ * {@link StreamPartitionMsgOffsetFactory} implementation for Pulsar streams. */ public class MessageIdStreamOffsetFactory implements StreamPartitionMsgOffsetFactory { - private StreamConfig _streamConfig; @Override public void init(StreamConfig streamConfig) { - _streamConfig = streamConfig; } @Override public StreamPartitionMsgOffset create(String offsetStr) { return new MessageIdStreamOffset(offsetStr); } - - @Override - public StreamPartitionMsgOffset create(StreamPartitionMsgOffset other) { - MessageIdStreamOffset messageIdStreamOffset = (MessageIdStreamOffset) other; - return new MessageIdStreamOffset(messageIdStreamOffset.getMessageId()); - } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerFactory.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerFactory.java index 6614bd321d10..65eb583df86e 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerFactory.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerFactory.java @@ -20,7 +20,6 @@ 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.StreamMetadataProvider; import org.apache.pinot.spi.stream.StreamPartitionMsgOffsetFactory; @@ -30,10 +29,6 @@ * A {@link StreamConsumerFactory} implementation for the Pulsar stream */ public class PulsarConsumerFactory extends StreamConsumerFactory { - @Override - public PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int partition) { - throw new UnsupportedOperationException("Partition Level consumer is deprecated!"); - } @Override public StreamMetadataProvider createPartitionMetadataProvider(String clientId, int partition) { @@ -53,6 +48,7 @@ public StreamPartitionMsgOffsetFactory createStreamMsgOffsetFactory() { @Override public PartitionGroupConsumer createPartitionGroupConsumer(String clientId, PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) { - return new PulsarPartitionLevelConsumer(clientId, _streamConfig, partitionGroupConsumptionStatus); + return new PulsarPartitionLevelConsumer(clientId, _streamConfig, + partitionGroupConsumptionStatus.getPartitionGroupId()); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarMessageBatch.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarMessageBatch.java index 15e7a4c1e2a9..10ea7ca9d6d1 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarMessageBatch.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarMessageBatch.java @@ -18,117 +18,43 @@ */ 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.BytesStreamMessage; import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.RowMetadata; -import org.apache.pinot.spi.stream.StreamMessage; -import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; -import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.impl.BatchMessageIdImpl; -import org.apache.pulsar.client.impl.MessageIdImpl; -import org.apache.pulsar.client.internal.DefaultImplementation; /** * A {@link MessageBatch} for collecting messages from pulsar topic - * When 'enableKeyValueStitch' flag is enabled, existing {@link org.apache.pinot.spi.stream.StreamMessageDecoder} - * plugins will not work. A custom decoder will be needed to unpack key and value byte arrays and decode - * them independently. */ -public class PulsarMessageBatch implements MessageBatch { - private final List _messageList = new ArrayList<>(); - private final boolean _enableKeyValueStitch; +public class PulsarMessageBatch implements MessageBatch { + private final List _messages; + private final MessageIdStreamOffset _offsetOfNextBatch; + private final boolean _endOfTopic; - public PulsarMessageBatch(Iterable iterable, boolean enableKeyValueStitch) { - iterable.forEach(_messageList::add); - _enableKeyValueStitch = enableKeyValueStitch; + public PulsarMessageBatch(List messages, MessageIdStreamOffset offsetOfNextBatch, + boolean endOfTopic) { + _messages = messages; + _offsetOfNextBatch = offsetOfNextBatch; + _endOfTopic = endOfTopic; } @Override public int getMessageCount() { - return _messageList.size(); + return _messages.size(); } - @Deprecated @Override - public PulsarStreamMessage getMessageAtIndex(int index) { - return getMessageList(index); + public BytesStreamMessage getStreamMessage(int index) { + return _messages.get(index); } @Override - public StreamMessage getStreamMessage(int index) { - return getMessageList(index); - } - - private PulsarStreamMessage getMessageList(int index) { - return _messageList.get(index); - } - @Override - public byte[] getMessageBytesAtIndex(int index) { - return getMessageList(index).getValue(); - } - - @Override - public RowMetadata getMetadataAtIndex(int index) { - return getMessageList(index).getMetadata(); - } - - @Override - public int getMessageOffsetAtIndex(int index) { - return ByteBuffer.wrap(_messageList.get(index).getValue()).arrayOffset(); - } - - @Override - public int getMessageLengthAtIndex(int index) { - return _messageList.get(index).getValue().length; //if _enableKeyValueStitch is true, - // then they are already stitched in the consumer. If false, then the value is the raw value - } - - /** - * Returns next message id supposed to be present in the pulsar topic partition. - * The message id is composed of 3 parts - ledgerId, entryId and partitionId. - * The ledger id are always increasing in number but may not be sequential. e.g. for first 10 records ledger id can - * be 12 but for next 10 it can be 18. - * each entry inside a ledger is always in a sequential and increases by 1 for next message. - * the partition id is fixed for a particular partition. - * We return entryId incremented by 1 while keeping ledgerId and partitionId as same. - * If ledgerId has incremented, the {@link org.apache.pulsar.client.api.Reader} takes care of that during seek - * operation - * and returns the first record in the new ledger. - */ - @Override - public StreamPartitionMsgOffset getNextStreamPartitionMsgOffsetAtIndex(int index) { - MessageIdImpl currentMessageId = MessageIdImpl.convertToMessageIdImpl(_messageList.get(index).getMessageId()); - MessageId nextMessageId; - - long currentLedgerId = currentMessageId.getLedgerId(); - long currentEntryId = currentMessageId.getEntryId(); - int currentPartitionIndex = currentMessageId.getPartitionIndex(); - - if (currentMessageId instanceof BatchMessageIdImpl) { - int currentBatchIndex = ((BatchMessageIdImpl) currentMessageId).getBatchIndex(); - int currentBatchSize = ((BatchMessageIdImpl) currentMessageId).getBatchSize(); - - if (currentBatchIndex < currentBatchSize - 1) { - nextMessageId = - new BatchMessageIdImpl(currentLedgerId, currentEntryId, currentPartitionIndex, currentBatchIndex + 1, - currentBatchSize, ((BatchMessageIdImpl) currentMessageId).getAcker()); - } else { - nextMessageId = - new BatchMessageIdImpl(currentLedgerId, currentEntryId + 1, currentPartitionIndex, 0, currentBatchSize, - ((BatchMessageIdImpl) currentMessageId).getAcker()); - } - } else { - nextMessageId = DefaultImplementation.getDefaultImplementation() - .newMessageId(currentLedgerId, currentEntryId + 1, currentPartitionIndex); - } - return new MessageIdStreamOffset(nextMessageId); + public MessageIdStreamOffset getOffsetOfNextBatch() { + return _offsetOfNextBatch; } @Override - public long getNextStreamMessageOffsetAtIndex(int index) { - throw new UnsupportedOperationException("Pulsar does not support long stream offsets"); + public boolean isEndOfPartitionGroup() { + return _endOfTopic; } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarMetadataExtractor.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarMetadataExtractor.java deleted file mode 100644 index 6b208af49504..000000000000 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarMetadataExtractor.java +++ /dev/null @@ -1,182 +0,0 @@ -/** - * 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.Base64; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; -import org.apache.commons.collections.MapUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.pinot.spi.data.readers.GenericRow; -import org.apache.pinot.spi.stream.RowMetadata; -import org.apache.pulsar.client.api.Message; - -public interface PulsarMetadataExtractor { - static PulsarMetadataExtractor build(boolean populateMetadata, - Set metadataValuesToExtract) { - return message -> { - long publishTime = message.getPublishTime(); - long brokerPublishTime = message.getBrokerPublishTime().orElse(0L); - long recordTimestamp = brokerPublishTime != 0 ? brokerPublishTime : publishTime; - - Map metadataMap = populateMetadataMap(populateMetadata, message, metadataValuesToExtract); - - GenericRow headerGenericRow = populateMetadata ? buildGenericRow(message) : null; - return new PulsarStreamMessageMetadata(recordTimestamp, headerGenericRow, metadataMap); - }; - } - - RowMetadata extract(Message record); - - static GenericRow buildGenericRow(Message message) { - if (MapUtils.isEmpty(message.getProperties())) { - return null; - } - GenericRow genericRow = new GenericRow(); - for (Map.Entry entry : message.getProperties().entrySet()) { - genericRow.putValue(entry.getKey(), entry.getValue()); - } - return genericRow; - } - - static Map populateMetadataMap(boolean populateAllFields, Message message, - Set metadataValuesToExtract) { - - Map metadataMap = new HashMap<>(); - populateMetadataField(PulsarStreamMessageMetadata.PulsarMessageMetadataValue.EVENT_TIME, message, metadataMap); - populateMetadataField(PulsarStreamMessageMetadata.PulsarMessageMetadataValue.PUBLISH_TIME, message, metadataMap); - populateMetadataField(PulsarStreamMessageMetadata.PulsarMessageMetadataValue.BROKER_PUBLISH_TIME, message, - metadataMap); - populateMetadataField(PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_KEY, message, metadataMap); - - // Populate some timestamps for lag calculation even if populateMetadata is false - - if (!populateAllFields) { - return metadataMap; - } - - for (PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataValue : metadataValuesToExtract) { - populateMetadataField(metadataValue, message, metadataMap); - } - - return metadataMap; - } - - static void populateMetadataField(PulsarStreamMessageMetadata.PulsarMessageMetadataValue value, - Message message, Map metadataMap) { - switch (value) { - case PUBLISH_TIME: - long publishTime = message.getPublishTime(); - if (publishTime > 0) { - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.PUBLISH_TIME, - publishTime); - } - break; - case EVENT_TIME: - long eventTime = message.getEventTime(); - if (eventTime > 0) { - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.EVENT_TIME, - eventTime); - } - break; - case BROKER_PUBLISH_TIME: - message.getBrokerPublishTime() - .ifPresent(brokerPublishTime -> setMetadataMapField(metadataMap, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue.BROKER_PUBLISH_TIME, brokerPublishTime)); - break; - case MESSAGE_KEY: - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_KEY, - message.getKey()); - break; - case MESSAGE_ID: - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID, - message.getMessageId().toString()); - break; - case MESSAGE_ID_BYTES_B64: - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID_BYTES_B64, - message.getMessageId().toByteArray()); - break; - case PRODUCER_NAME: - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.PRODUCER_NAME, - message.getProducerName()); - break; - case SCHEMA_VERSION: - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.SCHEMA_VERSION, - message.getSchemaVersion()); - break; - case SEQUENCE_ID: - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.SEQUENCE_ID, - message.getSequenceId()); - break; - case ORDERING_KEY: - if (message.hasOrderingKey()) { - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.ORDERING_KEY, - message.getOrderingKey()); - } - break; - case SIZE: - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.SIZE, - message.size()); - break; - case TOPIC_NAME: - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.TOPIC_NAME, - message.getTopicName()); - break; - case INDEX: - message.getIndex().ifPresent(index -> setMetadataMapField(metadataMap, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue.INDEX, index)); - break; - case REDELIVERY_COUNT: - setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.REDELIVERY_COUNT, - message.getRedeliveryCount()); - break; - default: - throw new IllegalArgumentException("Unsupported metadata value: " + value); - } - } - - static void setMetadataMapField(Map metadataMap, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataValue, - String value) { - if (StringUtils.isNotBlank(value)) { - metadataMap.put(metadataValue.getKey(), value); - } - } - - static void setMetadataMapField(Map metadataMap, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataValue, - int value) { - setMetadataMapField(metadataMap, metadataValue, String.valueOf(value)); - } - - static void setMetadataMapField(Map metadataMap, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataValue, - long value) { - setMetadataMapField(metadataMap, metadataValue, String.valueOf(value)); - } - - static void setMetadataMapField(Map metadataMap, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataValue, - byte[] value) { - if (value != null && value.length > 0) { - setMetadataMapField(metadataMap, metadataValue, Base64.getEncoder().encodeToString(value)); - } - } -} diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConnectionHandler.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConnectionHandler.java index 53b27eb963f0..444d05514534 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConnectionHandler.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConnectionHandler.java @@ -19,33 +19,24 @@ package org.apache.pinot.plugin.stream.pulsar; import java.io.IOException; -import java.net.MalformedURLException; import java.net.URL; import java.util.List; -import java.util.Optional; import org.apache.commons.lang3.StringUtils; import org.apache.pinot.spi.stream.StreamConfig; -import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ClientBuilder; -import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.impl.auth.oauth2.AuthenticationFactoryOAuth2; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Manages the Pulsar client connection, given the partition id and {@link PulsarConfig} */ public class PulsarPartitionLevelConnectionHandler { - private static final Logger LOGGER = LoggerFactory.getLogger(PulsarPartitionLevelConnectionHandler.class); - protected final PulsarConfig _config; protected final String _clientId; - protected PulsarClient _pulsarClient = null; - protected final PulsarMetadataExtractor _pulsarMetadataExtractor; + protected final PulsarClient _pulsarClient; /** * Creates a new instance of {@link PulsarClient} and {@link Reader} @@ -53,74 +44,41 @@ public class PulsarPartitionLevelConnectionHandler { public PulsarPartitionLevelConnectionHandler(String clientId, StreamConfig streamConfig) { _config = new PulsarConfig(streamConfig, clientId); _clientId = clientId; - _pulsarMetadataExtractor = PulsarMetadataExtractor.build(_config.isPopulateMetadata(), _config.getMetadataFields()); try { ClientBuilder pulsarClientBuilder = PulsarClient.builder().serviceUrl(_config.getBootstrapServers()); - if (_config.getTlsTrustCertsFilePath() != null) { - pulsarClientBuilder.tlsTrustCertsFilePath(_config.getTlsTrustCertsFilePath()); + String tlsTrustCertsFilePath = _config.getTlsTrustCertsFilePath(); + if (StringUtils.isNotBlank(tlsTrustCertsFilePath)) { + pulsarClientBuilder.tlsTrustCertsFilePath(tlsTrustCertsFilePath); } - - if (_config.getAuthenticationToken() != null) { - Authentication authentication = AuthenticationFactory.token(_config.getAuthenticationToken()); - pulsarClientBuilder.authentication(authentication); + String authenticationToken = _config.getAuthenticationToken(); + if (StringUtils.isNotBlank(authenticationToken)) { + pulsarClientBuilder.authentication(AuthenticationFactory.token(authenticationToken)); + } else { + String issuerUrl = _config.getIssuerUrl(); + String credentialsFilePath = _config.getCredentialsFilePath(); + String audience = _config.getAudience(); + if (StringUtils.isNotBlank(issuerUrl) && StringUtils.isNotBlank(credentialsFilePath) && StringUtils.isNotBlank( + audience)) { + pulsarClientBuilder.authentication( + AuthenticationFactoryOAuth2.clientCredentials(new URL(issuerUrl), new URL(credentialsFilePath), + audience)); + } } - - getAuthenticationFactory(_config).ifPresent(pulsarClientBuilder::authentication); _pulsarClient = pulsarClientBuilder.build(); - LOGGER.info("Created pulsar client {}", _pulsarClient); } catch (Exception e) { - LOGGER.error("Could not create pulsar consumer", e); + throw new RuntimeException("Caught exception while creating Pulsar client", e); } } - protected Optional getAuthenticationFactory(PulsarConfig pulsarConfig) { - if (StringUtils.isNotBlank(pulsarConfig.getIssuerUrl()) - && StringUtils.isNotBlank(pulsarConfig.getAudience()) - && StringUtils.isNotBlank(pulsarConfig.getCredentialsFilePath())) { - try { - return Optional.of(AuthenticationFactoryOAuth2.clientCredentials( - new URL(pulsarConfig.getIssuerUrl()), - new URL(pulsarConfig.getCredentialsFilePath()), - pulsarConfig.getAudience())); - } catch (MalformedURLException mue) { - LOGGER.error("Failed to create authentication factory for pulsar client with config: " - + "issuer: {}, credential file path: {}, audience: {}", - pulsarConfig.getIssuerUrl(), - pulsarConfig.getCredentialsFilePath(), - pulsarConfig.getAudience(), - mue); - } - } - return Optional.empty(); - } - protected Reader createReaderForPartition(String topic, int partition, MessageId initialMessageId) { - if (_pulsarClient == null) { - throw new RuntimeException("Failed to create reader as no pulsar client found for topic " + topic); - } - try { - return _pulsarClient.newReader().topic(getPartitionedTopicName(topic, partition)).startMessageId(initialMessageId) - .startMessageIdInclusive().create(); - } catch (Exception e) { - LOGGER.error("Failed to create pulsar consumer client for topic " + topic + " partition " + partition, e); - return null; - } - } - - /** - * A pulsar partitioned topic with N partitions is comprised of N topics with topicName as prefix and portitionId - * as suffix. - * The method fetches the names of N partitioned topic and returns the topic name of {@param partition} - */ - protected String getPartitionedTopicName(String topic, int partition) + protected Reader createReaderForPartition(int partitionId) throws Exception { - List partitionTopicList = _pulsarClient.getPartitionsForTopic(topic).get(); - return partitionTopicList.get(partition); + List partitions = _pulsarClient.getPartitionsForTopic(_config.getPulsarTopicName()).get(); + return _pulsarClient.newReader().topic(partitions.get(partitionId)).startMessageId(_config.getInitialMessageId()) + .startMessageIdInclusive().create(); } public void close() throws IOException { - if (_pulsarClient != null) { - _pulsarClient.close(); - } + _pulsarClient.close(); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConsumer.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConsumer.java index d1b80b0360c8..36a27d7e96ac 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConsumer.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConsumer.java @@ -18,7 +18,6 @@ */ 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; @@ -27,9 +26,10 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import org.apache.pinot.spi.stream.BytesStreamMessage; 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.StreamMessageMetadata; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; @@ -45,21 +45,21 @@ public class PulsarPartitionLevelConsumer extends PulsarPartitionLevelConnectionHandler implements PartitionGroupConsumer { private static final Logger LOGGER = LoggerFactory.getLogger(PulsarPartitionLevelConsumer.class); - private final ExecutorService _executorService; + private final Reader _reader; - private boolean _enableKeyValueStitch; - public PulsarPartitionLevelConsumer(String clientId, StreamConfig streamConfig, - PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) { + // TODO: Revisit the logic of using a separate executor to manage the request timeout. Currently it is not thread safe + private final ExecutorService _executorService = Executors.newSingleThreadExecutor(); + + public PulsarPartitionLevelConsumer(String clientId, StreamConfig streamConfig, int partitionId) { super(clientId, streamConfig); - PulsarConfig config = new PulsarConfig(streamConfig, clientId); - _reader = - createReaderForPartition(config.getPulsarTopicName(), partitionGroupConsumptionStatus.getPartitionGroupId(), - config.getInitialMessageId()); - LOGGER.info("Created pulsar reader with id {} for topic {} partition {}", _reader, _config.getPulsarTopicName(), - partitionGroupConsumptionStatus.getPartitionGroupId()); - _executorService = Executors.newSingleThreadExecutor(); - _enableKeyValueStitch = _config.getEnableKeyValueStitch(); + try { + _reader = createReaderForPartition(partitionId); + } catch (Exception e) { + throw new RuntimeException("Caught exception while creating Pulsar reader", e); + } + LOGGER.info("Created Pulsar reader with topic: {}, partition: {}, initial message id: {}", + _config.getPulsarTopicName(), partitionId, _config.getInitialMessageId()); } /** @@ -69,68 +69,50 @@ public PulsarPartitionLevelConsumer(String clientId, StreamConfig streamConfig, * reached. */ @Override - public PulsarMessageBatch fetchMessages(StreamPartitionMsgOffset startMsgOffset, - StreamPartitionMsgOffset endMsgOffset, int timeoutMillis) { - final MessageId startMessageId = ((MessageIdStreamOffset) startMsgOffset).getMessageId(); - final MessageId endMessageId = - endMsgOffset == null ? MessageId.latest : ((MessageIdStreamOffset) endMsgOffset).getMessageId(); - - List messagesList = new ArrayList<>(); - Future pulsarResultFuture = - _executorService.submit(() -> fetchMessages(startMessageId, endMessageId, messagesList)); - + public PulsarMessageBatch fetchMessages(StreamPartitionMsgOffset startMsgOffset, int timeoutMs) { + MessageIdStreamOffset startOffset = (MessageIdStreamOffset) startMsgOffset; + List messages = new ArrayList<>(); + Future pulsarResultFuture = _executorService.submit(() -> fetchMessages(startOffset, messages)); try { - return pulsarResultFuture.get(timeoutMillis, TimeUnit.MILLISECONDS); + return pulsarResultFuture.get(timeoutMs, 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. pulsarResultFuture.cancel(true); - return new PulsarMessageBatch(buildOffsetFilteringIterable(messagesList, startMessageId, endMessageId), - _enableKeyValueStitch); } catch (Exception e) { LOGGER.warn("Error while fetching records from Pulsar", e); - return new PulsarMessageBatch(buildOffsetFilteringIterable(messagesList, startMessageId, endMessageId), - _enableKeyValueStitch); } + return buildPulsarMessageBatch(startOffset, messages); } - public PulsarMessageBatch fetchMessages(MessageId startMessageId, MessageId endMessageId, - List messagesList) { + private PulsarMessageBatch fetchMessages(MessageIdStreamOffset startOffset, List messages) { try { + MessageId startMessageId = startOffset.getMessageId(); _reader.seek(startMessageId); - while (_reader.hasMessageAvailable()) { - Message nextMessage = _reader.readNext(); - - if (endMessageId != null) { - if (nextMessage.getMessageId().compareTo(endMessageId) > 0) { - break; - } - } - messagesList.add( - PulsarUtils.buildPulsarStreamMessage(nextMessage, _enableKeyValueStitch, _pulsarMetadataExtractor)); - + Message message = _reader.readNext(); + messages.add(PulsarUtils.buildPulsarStreamMessage(message, _config)); if (Thread.interrupted()) { break; } } - - return new PulsarMessageBatch(buildOffsetFilteringIterable(messagesList, startMessageId, endMessageId), - _enableKeyValueStitch); } catch (PulsarClientException e) { LOGGER.warn("Error consuming records from Pulsar topic", e); - return new PulsarMessageBatch(buildOffsetFilteringIterable(messagesList, startMessageId, endMessageId), - _enableKeyValueStitch); } + return buildPulsarMessageBatch(startOffset, messages); } - private Iterable buildOffsetFilteringIterable(final List messageAndOffsets, - final MessageId startOffset, final MessageId endOffset) { - return Iterables.filter(messageAndOffsets, input -> { - // Filter messages that are either null or have an offset ∉ [startOffset, endOffset] - return input != null && input.getValue() != null && (input.getMessageId().compareTo(startOffset) >= 0) && ( - (endOffset == null) || (input.getMessageId().compareTo(endOffset) < 0)); - }); + private PulsarMessageBatch buildPulsarMessageBatch(MessageIdStreamOffset startOffset, + List messages) { + MessageIdStreamOffset offsetOfNextBatch; + if (messages.isEmpty()) { + offsetOfNextBatch = startOffset; + } else { + StreamMessageMetadata lastMessageMetadata = messages.get(messages.size() - 1).getMetadata(); + assert lastMessageMetadata != null; + offsetOfNextBatch = (MessageIdStreamOffset) lastMessageMetadata.getNextOffset(); + } + return new PulsarMessageBatch(messages, offsetOfNextBatch, false); } @Override diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarStreamMessage.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarStreamMessage.java deleted file mode 100644 index 7e0919785794..000000000000 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarStreamMessage.java +++ /dev/null @@ -1,47 +0,0 @@ -/** - * 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 javax.annotation.Nullable; -import org.apache.pinot.spi.stream.StreamMessage; -import org.apache.pulsar.client.api.MessageId; - -public class PulsarStreamMessage extends StreamMessage { - - private final MessageId _messageId; - public PulsarStreamMessage(@Nullable byte[] key, byte[] value, MessageId messageId, - @Nullable PulsarStreamMessageMetadata metadata, int length) { - super(key, value, metadata, length); - _messageId = messageId; - } - - public MessageId getMessageId() { - return _messageId; - } - - int getKeyLength() { - byte[] key = getKey(); - return key == null ? 0 : key.length; - } - - int getValueLength() { - byte[] value = getValue(); - return value == null ? 0 : value.length; - } -} diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarStreamMessageMetadata.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarStreamMessageMetadata.java index 59220138d73b..fcf219e98df5 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarStreamMessageMetadata.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarStreamMessageMetadata.java @@ -25,52 +25,55 @@ import org.apache.pinot.spi.data.readers.GenericRow; import org.apache.pinot.spi.stream.StreamMessageMetadata; + /** * Pulsar specific implementation of {@link StreamMessageMetadata} * Pulsar makes many metadata values available for each message. Please see the pulsar documentation for more details. * @see Pulsar Message Properties */ +// TODO: Make it a util class public class PulsarStreamMessageMetadata extends StreamMessageMetadata { - public enum PulsarMessageMetadataValue { - PUBLISH_TIME("publishTime"), - EVENT_TIME("eventTime"), - BROKER_PUBLISH_TIME("brokerPublishTime"), - MESSAGE_KEY("key"), - MESSAGE_ID("messageId"), - MESSAGE_ID_BYTES_B64("messageIdBytes"), - PRODUCER_NAME("producerName"), - SCHEMA_VERSION("schemaVersion"), - SEQUENCE_ID("sequenceId"), - ORDERING_KEY("orderingKey"), - SIZE("size"), - TOPIC_NAME("topicName"), - INDEX("index"), - REDELIVERY_COUNT("redeliveryCount"); - - private final String _key; - - PulsarMessageMetadataValue(String key) { - _key = key; - } + public enum PulsarMessageMetadataValue { + PUBLISH_TIME("publishTime"), + EVENT_TIME("eventTime"), + BROKER_PUBLISH_TIME("brokerPublishTime"), + MESSAGE_KEY("key"), + MESSAGE_ID("messageId"), + MESSAGE_ID_BYTES_B64("messageIdBytes"), + PRODUCER_NAME("producerName"), + SCHEMA_VERSION("schemaVersion"), + SEQUENCE_ID("sequenceId"), + ORDERING_KEY("orderingKey"), + SIZE("size"), + TOPIC_NAME("topicName"), + INDEX("index"), + REDELIVERY_COUNT("redeliveryCount"); - public String getKey() { - return _key; - } + private final String _key; - public static PulsarMessageMetadataValue findByKey(final String key) { - EnumSet values = EnumSet.allOf(PulsarMessageMetadataValue.class); - return values.stream().filter(value -> value.getKey().equals(key)).findFirst().orElse(null); - } + PulsarMessageMetadataValue(String key) { + _key = key; } - public PulsarStreamMessageMetadata(long recordIngestionTimeMs, - @Nullable GenericRow headers) { - super(recordIngestionTimeMs, headers); + public String getKey() { + return _key; } - public PulsarStreamMessageMetadata(long recordIngestionTimeMs, @Nullable GenericRow headers, - Map metadata) { - super(recordIngestionTimeMs, headers, metadata); + public static PulsarMessageMetadataValue findByKey(final String key) { + EnumSet values = EnumSet.allOf(PulsarMessageMetadataValue.class); + return values.stream().filter(value -> value.getKey().equals(key)).findFirst().orElse(null); } + } + + @Deprecated + public PulsarStreamMessageMetadata(long recordIngestionTimeMs, @Nullable GenericRow headers) { + super(recordIngestionTimeMs, headers); + } + + @Deprecated + public PulsarStreamMessageMetadata(long recordIngestionTimeMs, @Nullable GenericRow headers, + Map metadata) { + super(recordIngestionTimeMs, headers, metadata); + } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtils.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtils.java index 772357aa6baf..301beb71fc2c 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtils.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtils.java @@ -18,11 +18,25 @@ */ package org.apache.pinot.plugin.stream.pulsar; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Maps; import java.nio.ByteBuffer; +import java.util.Base64; +import java.util.Map; +import java.util.Set; +import org.apache.commons.lang3.StringUtils; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.stream.BytesStreamMessage; import org.apache.pinot.spi.stream.OffsetCriteria; +import org.apache.pinot.spi.stream.StreamMessageMetadata; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.impl.BatchMessageAcker; +import org.apache.pulsar.client.impl.BatchMessageIdImpl; +import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.client.internal.DefaultImplementation; public class PulsarUtils { @@ -63,14 +77,170 @@ public static byte[] stitchKeyValue(byte[] keyBytes, byte[] valueBytes) { return stitchedBytes; } - public static PulsarStreamMessage buildPulsarStreamMessage(Message message, boolean enableKeyValueStitch, - PulsarMetadataExtractor pulsarMetadataExtractor) { + public static BytesStreamMessage buildPulsarStreamMessage(Message message, PulsarConfig config) { byte[] key = message.getKeyBytes(); byte[] value = message.getData(); - if (enableKeyValueStitch) { + if (config.getEnableKeyValueStitch()) { value = stitchKeyValue(key, value); } - return new PulsarStreamMessage(key, value, message.getMessageId(), - (PulsarStreamMessageMetadata) pulsarMetadataExtractor.extract(message), value.length); + return new BytesStreamMessage(key, value, extractMessageMetadata(message, config)); + } + + @VisibleForTesting + static StreamMessageMetadata extractMessageMetadata(Message message, PulsarConfig config) { + long recordIngestionTimeMs = message.getBrokerPublishTime().orElse(message.getPublishTime()); + MessageId messageId = message.getMessageId(); + MessageIdStreamOffset offset = new MessageIdStreamOffset(messageId); + MessageIdStreamOffset nextOffset = new MessageIdStreamOffset(getNextMessageId(messageId)); + StreamMessageMetadata.Builder builder = + new StreamMessageMetadata.Builder().setRecordIngestionTimeMs(recordIngestionTimeMs) + .setOffset(offset, nextOffset); + if (config.isPopulateMetadata()) { + Map properties = message.getProperties(); + if (!properties.isEmpty()) { + GenericRow header = new GenericRow(); + for (Map.Entry entry : properties.entrySet()) { + header.putValue(entry.getKey(), entry.getValue()); + } + builder.setHeaders(header); + } + Set metadataFields = config.getMetadataFields(); + if (!metadataFields.isEmpty()) { + Map metadataMap = Maps.newHashMapWithExpectedSize(metadataFields.size()); + for (PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataField : metadataFields) { + populateMetadataField(message, metadataField, metadataMap); + } + builder.setMetadata(metadataMap); + } + } + return builder.build(); + } + + /** + * Returns next message id supposed to be present in the pulsar topic partition. + * + * The message id is composed of 3 parts - ledgerId, entryId and partitionId. + * The ledger id are always increasing in number but may not be sequential. e.g. for first 10 records ledger id can + * be 12 but for next 10 it can be 18. Each entry inside a ledger is always in a sequential and increases by 1 for + * next message. + * The partition id is fixed for a particular partition. + * We return entryId incremented by 1 while keeping ledgerId and partitionId as same. + * If ledgerId has incremented, the {@link Reader} takes care of that during seek operation, and returns the first + * record in the new ledger. + */ + public static MessageId getNextMessageId(MessageId messageId) { + MessageIdImpl messageIdImpl = MessageIdImpl.convertToMessageIdImpl(messageId); + long ledgerId = messageIdImpl.getLedgerId(); + long entryId = messageIdImpl.getEntryId(); + int partitionIndex = messageIdImpl.getPartitionIndex(); + if (messageIdImpl instanceof BatchMessageIdImpl) { + BatchMessageIdImpl batchMessageIdImpl = (BatchMessageIdImpl) messageIdImpl; + int batchIndex = batchMessageIdImpl.getBatchIndex(); + int batchSize = batchMessageIdImpl.getBatchSize(); + BatchMessageAcker acker = batchMessageIdImpl.getAcker(); + if (batchIndex < batchSize - 1) { + return new BatchMessageIdImpl(ledgerId, entryId, partitionIndex, batchIndex + 1, batchSize, acker); + } else { + return new BatchMessageIdImpl(ledgerId, entryId + 1, partitionIndex, 0, batchSize, acker); + } + } else { + return DefaultImplementation.getDefaultImplementation().newMessageId(ledgerId, entryId + 1, partitionIndex); + } + } + + private static void populateMetadataField(Message message, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataField, Map metadataMap) { + switch (metadataField) { + case PUBLISH_TIME: + long publishTime = message.getPublishTime(); + if (publishTime > 0) { + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.PUBLISH_TIME, + publishTime); + } + break; + case EVENT_TIME: + long eventTime = message.getEventTime(); + if (eventTime > 0) { + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.EVENT_TIME, + eventTime); + } + break; + case BROKER_PUBLISH_TIME: + message.getBrokerPublishTime().ifPresent(brokerPublishTime -> setMetadataMapField(metadataMap, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue.BROKER_PUBLISH_TIME, brokerPublishTime)); + break; + case MESSAGE_KEY: + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_KEY, + message.getKey()); + break; + case MESSAGE_ID: + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID, + message.getMessageId().toString()); + break; + case MESSAGE_ID_BYTES_B64: + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID_BYTES_B64, + message.getMessageId().toByteArray()); + break; + case PRODUCER_NAME: + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.PRODUCER_NAME, + message.getProducerName()); + break; + case SCHEMA_VERSION: + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.SCHEMA_VERSION, + message.getSchemaVersion()); + break; + case SEQUENCE_ID: + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.SEQUENCE_ID, + message.getSequenceId()); + break; + case ORDERING_KEY: + if (message.hasOrderingKey()) { + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.ORDERING_KEY, + message.getOrderingKey()); + } + break; + case SIZE: + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.SIZE, message.size()); + break; + case TOPIC_NAME: + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.TOPIC_NAME, + message.getTopicName()); + break; + case INDEX: + message.getIndex().ifPresent( + index -> setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.INDEX, + index)); + break; + case REDELIVERY_COUNT: + setMetadataMapField(metadataMap, PulsarStreamMessageMetadata.PulsarMessageMetadataValue.REDELIVERY_COUNT, + message.getRedeliveryCount()); + break; + default: + throw new IllegalArgumentException("Unsupported metadata field: " + metadataField); + } + } + + private static void setMetadataMapField(Map metadataMap, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataValue, String value) { + if (StringUtils.isNotEmpty(value)) { + metadataMap.put(metadataValue.getKey(), value); + } + } + + private static void setMetadataMapField(Map metadataMap, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataValue, int value) { + setMetadataMapField(metadataMap, metadataValue, Integer.toString(value)); + } + + private static void setMetadataMapField(Map metadataMap, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataValue, long value) { + setMetadataMapField(metadataMap, metadataValue, Long.toString(value)); + } + + private static void setMetadataMapField(Map metadataMap, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue metadataValue, byte[] value) { + if (value != null && value.length > 0) { + setMetadataMapField(metadataMap, metadataValue, Base64.getEncoder().encodeToString(value)); + } } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfigTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfigTest.java index 486944b70384..bca1416cecfb 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfigTest.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfigTest.java @@ -31,14 +31,13 @@ import org.testng.Assert; import org.testng.annotations.Test; -import static org.junit.Assert.assertTrue; public class PulsarConfigTest { public static final String TABLE_NAME_WITH_TYPE = "tableName_REALTIME"; - public static final String STREAM_TYPE = "pulsar"; public static final String STREAM_PULSAR_BROKER_LIST = "pulsar://localhost:6650"; public static final String STREAM_PULSAR_CONSUMER_TYPE = "simple"; + Map getCommonStreamConfigMap() { Map streamConfigMap = new HashMap<>(); streamConfigMap.put("streamType", STREAM_TYPE); @@ -55,7 +54,7 @@ Map getCommonStreamConfigMap() { } @Test - public void testParsingMetadataConfigWithConfiguredFields() throws Exception { + public void testParsingMetadataConfigWithConfiguredFields() { Map streamConfigMap = getCommonStreamConfigMap(); streamConfigMap.put( StreamConfigProperties.constructStreamProperty(STREAM_TYPE, StreamConfigProperties.METADATA_POPULATE), "true"); @@ -66,21 +65,20 @@ public void testParsingMetadataConfigWithConfiguredFields() throws Exception { Set metadataFieldsToExtract = pulsarConfig.getMetadataFields(); Assert.assertEquals(metadataFieldsToExtract.size(), 6); - Assert.assertTrue(metadataFieldsToExtract.containsAll(ImmutableList.of( - PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID_BYTES_B64, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue.PUBLISH_TIME, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue.EVENT_TIME, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_KEY, - PulsarStreamMessageMetadata.PulsarMessageMetadataValue.TOPIC_NAME))); + Assert.assertTrue(metadataFieldsToExtract.containsAll( + ImmutableList.of(PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID_BYTES_B64, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue.PUBLISH_TIME, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue.EVENT_TIME, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_KEY, + PulsarStreamMessageMetadata.PulsarMessageMetadataValue.TOPIC_NAME))); } @Test - public void testParsingMetadataConfigWithoutConfiguredFields() throws Exception { + public void testParsingMetadataConfigWithoutConfiguredFields() { Map streamConfigMap = getCommonStreamConfigMap(); streamConfigMap.put( - StreamConfigProperties.constructStreamProperty(STREAM_TYPE, StreamConfigProperties.METADATA_POPULATE), - "true"); + StreamConfigProperties.constructStreamProperty(STREAM_TYPE, StreamConfigProperties.METADATA_POPULATE), "true"); StreamConfig streamConfig = new StreamConfig(TABLE_NAME_WITH_TYPE, streamConfigMap); PulsarConfig pulsarConfig = new PulsarConfig(streamConfig, "testId"); Set metadataFieldsToExtract = @@ -89,11 +87,10 @@ public void testParsingMetadataConfigWithoutConfiguredFields() throws Exception } @Test - public void testParsingNoMetadataConfig() throws Exception { + public void testParsingNoMetadataConfig() { Map streamConfigMap = getCommonStreamConfigMap(); streamConfigMap.put( - StreamConfigProperties.constructStreamProperty(STREAM_TYPE, StreamConfigProperties.METADATA_POPULATE), - "false"); + StreamConfigProperties.constructStreamProperty(STREAM_TYPE, StreamConfigProperties.METADATA_POPULATE), "false"); StreamConfig streamConfig = new StreamConfig(TABLE_NAME_WITH_TYPE, streamConfigMap); PulsarConfig pulsarConfig = new PulsarConfig(streamConfig, "testId"); Assert.assertFalse(pulsarConfig.isPopulateMetadata()); @@ -103,13 +100,11 @@ public void testParsingNoMetadataConfig() throws Exception { } @Test - public void testParsingNoMetadataConfigWithConfiguredFields() throws Exception { + public void testParsingNoMetadataConfigWithConfiguredFields() { Map streamConfigMap = getCommonStreamConfigMap(); streamConfigMap.put( - StreamConfigProperties.constructStreamProperty(STREAM_TYPE, StreamConfigProperties.METADATA_POPULATE), - "false"); - streamConfigMap.put( - StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.METADATA_FIELDS), + StreamConfigProperties.constructStreamProperty(STREAM_TYPE, StreamConfigProperties.METADATA_POPULATE), "false"); + streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.METADATA_FIELDS), "messageId,messageIdBytes, publishTime, eventTime, key, topicName, "); StreamConfig streamConfig = new StreamConfig(TABLE_NAME_WITH_TYPE, streamConfigMap); PulsarConfig pulsarConfig = new PulsarConfig(streamConfig, "testId"); @@ -120,55 +115,42 @@ public void testParsingNoMetadataConfigWithConfiguredFields() throws Exception { } @Test - public void testParsingConfigForOAuth() throws Exception { + public void testParsingConfigForOAuth() + throws Exception { Path testFile = null; try { testFile = Files.createTempFile("test_cred_file", ".json"); Map streamConfigMap = getCommonStreamConfigMap(); streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.OAUTH_ISSUER_URL), "http://auth.test.com"); - streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, - PulsarConfig.OAUTH_CREDS_FILE_PATH), "file://" + testFile.toFile().getAbsolutePath()); + streamConfigMap.put( + StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.OAUTH_CREDS_FILE_PATH), + "file://" + testFile.toFile().getAbsolutePath()); streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.OAUTH_AUDIENCE), "urn:test:test"); StreamConfig streamConfig = new StreamConfig(TABLE_NAME_WITH_TYPE, streamConfigMap); PulsarConfig pulsarConfig = new PulsarConfig(streamConfig, "testId"); Assert.assertEquals(pulsarConfig.getIssuerUrl(), "http://auth.test.com"); - Assert.assertEquals(pulsarConfig.getCredentialsFilePath(), - "file://" + testFile.toFile().getAbsolutePath()); + Assert.assertEquals(pulsarConfig.getCredentialsFilePath(), "file://" + testFile.toFile().getAbsolutePath()); Assert.assertEquals(pulsarConfig.getAudience(), "urn:test:test"); - PulsarPartitionLevelConnectionHandler pulsarPartitionLevelConnectionHandler = - new PulsarPartitionLevelConnectionHandler("testId", streamConfig); - assertTrue(pulsarPartitionLevelConnectionHandler.getAuthenticationFactory(pulsarConfig).isPresent()); - } catch (Exception e) { - Assert.fail("Should not throw exception", e); } finally { Optional.ofNullable(testFile).map(Path::toFile).ifPresent(File::delete); } } - @Test - public void testParsingConfigFailFileValidationForOAuth() throws Exception { + @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Invalid credentials " + + "file path: .*") + public void testParsingConfigFailFileValidationForOAuth() { String testFilePath = "file://path/to/file.json"; - - try { - Map streamConfigMap = getCommonStreamConfigMap(); - streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.OAUTH_ISSUER_URL), - "http://auth.test.com"); - streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, - PulsarConfig.OAUTH_CREDS_FILE_PATH), - testFilePath); - streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.OAUTH_AUDIENCE), - "urn:test:test"); - StreamConfig streamConfig = new StreamConfig(TABLE_NAME_WITH_TYPE, streamConfigMap); - PulsarConfig pulsarConfig = new PulsarConfig(streamConfig, "testId"); //will throw exception - } catch (IllegalArgumentException mue) { - //expected case. - String errorMessage = String.format("Invalid credentials file path: %s. File does not exist.", testFilePath); - Assert.assertEquals(errorMessage, mue.getMessage()); - } catch (Exception e) { - Assert.fail("Should not throw other exception", e); - } + Map streamConfigMap = getCommonStreamConfigMap(); + streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.OAUTH_ISSUER_URL), + "http://auth.test.com"); + streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.OAUTH_CREDS_FILE_PATH), + testFilePath); + streamConfigMap.put(StreamConfigProperties.constructStreamProperty(STREAM_TYPE, PulsarConfig.OAUTH_AUDIENCE), + "urn:test:test"); + StreamConfig streamConfig = new StreamConfig(TABLE_NAME_WITH_TYPE, streamConfigMap); + new PulsarConfig(streamConfig, "testId"); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerTest.java index 9d59f82fcc8e..5b7fbe4ae168 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerTest.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerTest.java @@ -53,11 +53,13 @@ import org.slf4j.LoggerFactory; import org.testcontainers.containers.PulsarContainer; import org.testcontainers.utility.DockerImageName; -import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.fail; + public class PulsarConsumerTest { private static final Logger LOGGER = LoggerFactory.getLogger(PulsarConsumerTest.class); @@ -230,8 +232,8 @@ private boolean validatePartitionMessageCount(int partition, int expectedMsgCoun new PartitionGroupConsumptionStatus(partition, 1, new MessageIdStreamOffset(MessageId.earliest), null, "CONSUMING")); try { - final MessageBatch messageBatch = - consumer.fetchMessages(new MessageIdStreamOffset(MessageId.earliest), null, CONSUMER_FETCH_TIMEOUT_MILLIS); + MessageBatch messageBatch = + consumer.fetchMessages(new MessageIdStreamOffset(MessageId.earliest), CONSUMER_FETCH_TIMEOUT_MILLIS); LOGGER.info("Partition: " + partition + ", Consumed messageBatch count = " + messageBatch.getMessageCount()); return messageBatch.getMessageCount() == expectedMsgCount; } catch (TimeoutException e) { @@ -268,107 +270,77 @@ protected String getPulsarConsumerFactoryName() { @Test public void testPartitionLevelConsumer() throws Exception { - - final StreamConsumerFactory streamConsumerFactory = - StreamConsumerFactoryProvider.create(getStreamConfig(TEST_TOPIC)); - int numPartitions = new PulsarStreamMetadataProvider(CLIENT_ID, getStreamConfig(TEST_TOPIC)).fetchPartitionCount( - CONSUMER_FETCH_TIMEOUT_MILLIS); + StreamConsumerFactory streamConsumerFactory = StreamConsumerFactoryProvider.create(getStreamConfig(TEST_TOPIC)); + int numPartitions; + try (PulsarStreamMetadataProvider metadataProvider = new PulsarStreamMetadataProvider(CLIENT_ID, + getStreamConfig(TEST_TOPIC))) { + numPartitions = metadataProvider.fetchPartitionCount(CONSUMER_FETCH_TIMEOUT_MILLIS); + } for (int partition = 0; partition < numPartitions; partition++) { PartitionGroupConsumptionStatus partitionGroupConsumptionStatus = - new PartitionGroupConsumptionStatus(partition, 1, new MessageIdStreamOffset(MessageId.earliest), null, + new PartitionGroupConsumptionStatus(partition, 0, new MessageIdStreamOffset(MessageId.earliest), null, "CONSUMING"); + try ( + PulsarPartitionLevelConsumer consumer = + (PulsarPartitionLevelConsumer) streamConsumerFactory.createPartitionGroupConsumer( + CLIENT_ID, partitionGroupConsumptionStatus)) { + PulsarMessageBatch messageBatch = + consumer.fetchMessages(new MessageIdStreamOffset(MessageId.earliest), CONSUMER_FETCH_TIMEOUT_MILLIS); + assertEquals(messageBatch.getMessageCount(), 1000); + for (int i = 0; i < 1000; i++) { + byte[] msg = messageBatch.getStreamMessage(i).getValue(); + assertEquals(new String(msg), "sample_msg_" + i); + } - int totalMessagesReceived = 0; - - final PulsarPartitionLevelConsumer consumer = - (PulsarPartitionLevelConsumer) streamConsumerFactory - .createPartitionGroupConsumer(CLIENT_ID, partitionGroupConsumptionStatus); - final PulsarMessageBatch messageBatch1 = consumer.fetchMessages(new MessageIdStreamOffset(MessageId.earliest), - new MessageIdStreamOffset(getMessageIdForPartitionAndIndex(partition, 500)), CONSUMER_FETCH_TIMEOUT_MILLIS); - Assert.assertEquals(messageBatch1.getMessageCount(), 500); - for (int i = 0; i < messageBatch1.getMessageCount(); i++) { - final byte[] msg = messageBatch1.getMessageAtIndex(i).getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + i); - totalMessagesReceived++; - } - - final PulsarMessageBatch messageBatch2 = - consumer.fetchMessages(new MessageIdStreamOffset(getMessageIdForPartitionAndIndex(partition, 500)), null, - CONSUMER_FETCH_TIMEOUT_MILLIS); - Assert.assertEquals(messageBatch2.getMessageCount(), 500); - for (int i = 0; i < messageBatch2.getMessageCount(); i++) { - final byte[] msg = messageBatch2.getMessageAtIndex(i).getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + (500 + i)); - totalMessagesReceived++; - } - - final PulsarMessageBatch messageBatch3 = - consumer.fetchMessages(new MessageIdStreamOffset(getMessageIdForPartitionAndIndex(partition, 10)), - new MessageIdStreamOffset(getMessageIdForPartitionAndIndex(partition, 35)), - CONSUMER_FETCH_TIMEOUT_MILLIS); - Assert.assertEquals(messageBatch3.getMessageCount(), 25); - for (int i = 0; i < messageBatch3.getMessageCount(); i++) { - final byte[] msg = messageBatch3.getMessageAtIndex(i).getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + (10 + i)); + messageBatch = + consumer.fetchMessages(new MessageIdStreamOffset(getMessageIdForPartitionAndIndex(partition, 500)), + CONSUMER_FETCH_TIMEOUT_MILLIS); + assertEquals(messageBatch.getMessageCount(), 500); + for (int i = 0; i < 500; i++) { + byte[] msg = messageBatch.getStreamMessage(i).getValue(); + assertEquals(new String(msg), "sample_msg_" + (500 + i)); + } } - - Assert.assertEquals(totalMessagesReceived, NUM_RECORDS_PER_PARTITION); } } @Test public void testPartitionLevelConsumerBatchMessages() throws Exception { - - final StreamConsumerFactory streamConsumerFactory = + StreamConsumerFactory streamConsumerFactory = StreamConsumerFactoryProvider.create(getStreamConfig(TEST_TOPIC_BATCH)); - int numPartitions = - new PulsarStreamMetadataProvider(CLIENT_ID, getStreamConfig(TEST_TOPIC_BATCH)).fetchPartitionCount( - CONSUMER_FETCH_TIMEOUT_MILLIS); + int numPartitions; + try (PulsarStreamMetadataProvider metadataProvider = new PulsarStreamMetadataProvider(CLIENT_ID, + getStreamConfig(TEST_TOPIC_BATCH))) { + numPartitions = metadataProvider.fetchPartitionCount(CONSUMER_FETCH_TIMEOUT_MILLIS); + } for (int partition = 0; partition < numPartitions; partition++) { PartitionGroupConsumptionStatus partitionGroupConsumptionStatus = - new PartitionGroupConsumptionStatus(partition, 1, new MessageIdStreamOffset(MessageId.earliest), null, + new PartitionGroupConsumptionStatus(partition, 0, new MessageIdStreamOffset(MessageId.earliest), null, "CONSUMING"); + try ( + PulsarPartitionLevelConsumer consumer = + (PulsarPartitionLevelConsumer) streamConsumerFactory.createPartitionGroupConsumer( + CLIENT_ID, partitionGroupConsumptionStatus)) { + PulsarMessageBatch messageBatch = + consumer.fetchMessages(new MessageIdStreamOffset(MessageId.earliest), CONSUMER_FETCH_TIMEOUT_MILLIS); + assertEquals(messageBatch.getMessageCount(), 1000); + for (int i = 0; i < 1000; i++) { + byte[] msg = messageBatch.getStreamMessage(i).getValue(); + assertEquals(new String(msg), "sample_msg_" + i); + } - int totalMessagesReceived = 0; - - final PulsarPartitionLevelConsumer consumer = - (PulsarPartitionLevelConsumer) streamConsumerFactory.createPartitionGroupConsumer(CLIENT_ID, - partitionGroupConsumptionStatus); - //TODO: This test failed, check it out. - final PulsarMessageBatch messageBatch1 = consumer.fetchMessages(new MessageIdStreamOffset(MessageId.earliest), - new MessageIdStreamOffset(getBatchMessageIdForPartitionAndIndex(partition, 500)), - CONSUMER_FETCH_TIMEOUT_MILLIS); - Assert.assertEquals(messageBatch1.getMessageCount(), 500); - for (int i = 0; i < messageBatch1.getMessageCount(); i++) { - final byte[] msg = messageBatch1.getMessageAtIndex(i).getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + i); - totalMessagesReceived++; - } - - final PulsarMessageBatch messageBatch2 = - consumer.fetchMessages(new MessageIdStreamOffset(getBatchMessageIdForPartitionAndIndex(partition, 500)), null, - CONSUMER_FETCH_TIMEOUT_MILLIS); - Assert.assertEquals(messageBatch2.getMessageCount(), 500); - for (int i = 0; i < messageBatch2.getMessageCount(); i++) { - final byte[] msg = messageBatch2.getMessageAtIndex(i).getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + (500 + i)); - totalMessagesReceived++; - } - - final PulsarMessageBatch messageBatch3 = - consumer.fetchMessages(new MessageIdStreamOffset(getBatchMessageIdForPartitionAndIndex(partition, 10)), - new MessageIdStreamOffset(getBatchMessageIdForPartitionAndIndex(partition, 35)), - CONSUMER_FETCH_TIMEOUT_MILLIS); - Assert.assertEquals(messageBatch3.getMessageCount(), 25); - for (int i = 0; i < messageBatch3.getMessageCount(); i++) { - final byte[] msg = messageBatch3.getMessageAtIndex(i).getValue(); - Assert.assertEquals(new String(msg), "sample_msg_" + (10 + i)); + messageBatch = + consumer.fetchMessages(new MessageIdStreamOffset(getBatchMessageIdForPartitionAndIndex(partition, 500)), + CONSUMER_FETCH_TIMEOUT_MILLIS); + assertEquals(messageBatch.getMessageCount(), 500); + for (int i = 0; i < messageBatch.getMessageCount(); i++) { + byte[] msg = messageBatch.getStreamMessage(i).getValue(); + assertEquals(new String(msg), "sample_msg_" + (500 + i)); + } } - - Assert.assertEquals(totalMessagesReceived, NUM_RECORDS_PER_PARTITION); } } @@ -394,11 +366,11 @@ private void waitForCondition(Function condition, long checkInter } Thread.sleep(checkIntervalMs); } catch (Exception e) { - Assert.fail("Caught exception while checking the condition" + errorMessageSuffix, e); + fail("Caught exception while checking the condition" + errorMessageSuffix, e); } } if (raiseError) { - Assert.fail("Failed to meet condition in " + timeoutMs + "ms" + errorMessageSuffix); + fail("Failed to meet condition in " + timeoutMs + "ms" + errorMessageSuffix); } } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarMessageBatchTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarMessageBatchTest.java index 904dd33a0461..f95dea1279ef 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarMessageBatchTest.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarMessageBatchTest.java @@ -19,34 +19,35 @@ package org.apache.pinot.plugin.stream.pulsar; import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.EnumSet; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Random; -import java.util.stream.Collectors; +import org.apache.pinot.spi.stream.BytesStreamMessage; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.common.api.EncryptionContext; -import org.junit.Assert; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; + public class PulsarMessageBatchTest { - private Random _random = new Random(); - private DummyPulsarMessage _msgWithKeyAndValue; + private static final Random RANDOM = new Random(); + private byte[] _expectedValueBytes; private byte[] _expectedKeyBytes; - private List _messageList; - private PulsarMetadataExtractor _metadataExtractor; + private DummyPulsarMessage _message; public static class DummyPulsarMessage implements Message { private final byte[] _keyData; private final byte[] _valueData; - private Map _properties; + private final Map _properties; public DummyPulsarMessage(byte[] key, byte[] value) { _keyData = key; @@ -111,7 +112,7 @@ public String getProducerName() { @Override public boolean hasKey() { - return _keyData == null ? false : true; + return _keyData != null; } @Override @@ -198,45 +199,39 @@ public Optional getIndex() { public void setup() { _expectedValueBytes = new byte[10]; _expectedKeyBytes = new byte[10]; - _random.nextBytes(_expectedValueBytes); - _random.nextBytes(_expectedKeyBytes); - _msgWithKeyAndValue = new DummyPulsarMessage(_expectedKeyBytes, _expectedValueBytes); - _messageList = new ArrayList<>(); - _metadataExtractor = PulsarMetadataExtractor.build(true, - EnumSet.allOf(PulsarStreamMessageMetadata.PulsarMessageMetadataValue.class)); - _messageList.add(_msgWithKeyAndValue); + RANDOM.nextBytes(_expectedValueBytes); + RANDOM.nextBytes(_expectedKeyBytes); + _message = new DummyPulsarMessage(_expectedKeyBytes, _expectedValueBytes); } @Test public void testMessageBatchNoStitching() { - List streamMessages = _messageList.stream().map(message -> - PulsarUtils.buildPulsarStreamMessage(message, false, _metadataExtractor)) - .collect(Collectors.toList()); - PulsarMessageBatch messageBatch = new PulsarMessageBatch(streamMessages, false); - byte[] valueBytes = messageBatch.getMessageAtIndex(0).getValue(); - Assert.assertArrayEquals(_expectedValueBytes, valueBytes); + PulsarConfig config = mock(PulsarConfig.class); + when(config.getEnableKeyValueStitch()).thenReturn(false); + List streamMessages = List.of(PulsarUtils.buildPulsarStreamMessage(_message, config)); + PulsarMessageBatch messageBatch = new PulsarMessageBatch(streamMessages, mock(MessageIdStreamOffset.class), false); + byte[] valueBytes = messageBatch.getStreamMessage(0).getValue(); + assertEquals(valueBytes, _expectedValueBytes); } @Test public void testMessageBatchWithStitching() { - List streamMessages = _messageList.stream().map(message -> - PulsarUtils.buildPulsarStreamMessage(message, true, _metadataExtractor)) - .collect(Collectors.toList()); - PulsarMessageBatch messageBatch = new PulsarMessageBatch(streamMessages, true); - byte[] keyValueBytes = messageBatch.getMessageAtIndex(0).getValue(); - Assert.assertEquals(keyValueBytes.length, 8 + _expectedKeyBytes.length + _expectedValueBytes.length); - try { - ByteBuffer byteBuffer = ByteBuffer.wrap(keyValueBytes); - int keyLength = byteBuffer.getInt(); - byte[] keyBytes = new byte[keyLength]; - byteBuffer.get(keyBytes); - Assert.assertArrayEquals(_expectedKeyBytes, keyBytes); - int valueLength = byteBuffer.getInt(); - byte[] valueBytes = new byte[valueLength]; - byteBuffer.get(valueBytes); - Assert.assertArrayEquals(_expectedValueBytes, valueBytes); - } catch (Exception e) { - Assert.fail("Could not parse key and value bytes because of exception: " + e.getMessage()); - } + PulsarConfig config = mock(PulsarConfig.class); + when(config.getEnableKeyValueStitch()).thenReturn(true); + List streamMessages = List.of(PulsarUtils.buildPulsarStreamMessage(_message, config)); + PulsarMessageBatch messageBatch = new PulsarMessageBatch(streamMessages, mock(MessageIdStreamOffset.class), false); + BytesStreamMessage streamMessage = messageBatch.getStreamMessage(0); + byte[] keyValueBytes = streamMessage.getValue(); + assertNotNull(keyValueBytes); + assertEquals(keyValueBytes.length, 8 + _expectedKeyBytes.length + _expectedValueBytes.length); + ByteBuffer byteBuffer = ByteBuffer.wrap(keyValueBytes); + int keyLength = byteBuffer.getInt(); + byte[] keyBytes = new byte[keyLength]; + byteBuffer.get(keyBytes); + assertEquals(keyBytes, _expectedKeyBytes); + int valueLength = byteBuffer.getInt(); + byte[] valueBytes = new byte[valueLength]; + byteBuffer.get(valueBytes); + assertEquals(valueBytes, _expectedValueBytes); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarMetadataExtractorTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtilsTest.java similarity index 57% rename from pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarMetadataExtractorTest.java rename to pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtilsTest.java index fd4eb1fa6a61..be1c5050c8fa 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarMetadataExtractorTest.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtilsTest.java @@ -18,76 +18,78 @@ */ package org.apache.pinot.plugin.stream.pulsar; -import com.google.common.collect.ImmutableSet; import java.nio.charset.StandardCharsets; +import java.util.Map; +import java.util.Set; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.stream.BytesStreamMessage; +import org.apache.pinot.spi.stream.StreamMessageMetadata; import org.apache.pulsar.client.api.MessageId; import org.bouncycastle.util.encoders.Base64; -import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; import static org.apache.pinot.plugin.stream.pulsar.PulsarMessageBatchTest.DummyPulsarMessage; import static org.apache.pinot.plugin.stream.pulsar.PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID; import static org.apache.pinot.plugin.stream.pulsar.PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_ID_BYTES_B64; import static org.apache.pinot.plugin.stream.pulsar.PulsarStreamMessageMetadata.PulsarMessageMetadataValue.MESSAGE_KEY; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; -public class PulsarMetadataExtractorTest { - - private PulsarMetadataExtractor _metadataExtractor; - - @BeforeClass - public void setup() { - _metadataExtractor = - PulsarMetadataExtractor.build(true, ImmutableSet.of(MESSAGE_ID, MESSAGE_ID_BYTES_B64, MESSAGE_KEY)); - } +public class PulsarUtilsTest { @Test public void testExtractProperty() throws Exception { + PulsarConfig config = mock(PulsarConfig.class); + when(config.isPopulateMetadata()).thenReturn(true); + when(config.getMetadataFields()).thenReturn(Set.of(MESSAGE_ID, MESSAGE_ID_BYTES_B64, MESSAGE_KEY)); DummyPulsarMessage pulsarMessage = new DummyPulsarMessage("key".getBytes(StandardCharsets.UTF_8), "value".getBytes(StandardCharsets.UTF_8)); pulsarMessage.getProperties().put("test_key", "test_value"); pulsarMessage.getProperties().put("test_key2", "2"); - PulsarStreamMessageMetadata metadata = (PulsarStreamMessageMetadata) _metadataExtractor.extract(pulsarMessage); - assertEquals("test_value", metadata.getHeaders().getValue("test_key")); - assertEquals("2", metadata.getHeaders().getValue("test_key2")); - assertEquals("key", metadata.getRecordMetadata().get(MESSAGE_KEY.getKey())); - String messageIdStr = metadata.getRecordMetadata().get(MESSAGE_ID.getKey()); - assertEquals(pulsarMessage.getMessageId().toString(), messageIdStr); + StreamMessageMetadata metadata = PulsarUtils.extractMessageMetadata(pulsarMessage, config); + GenericRow headers = metadata.getHeaders(); + assertNotNull(headers); + assertEquals(headers.getValue("test_key"), "test_value"); + assertEquals(headers.getValue("test_key2"), "2"); + Map recordMetadata = metadata.getRecordMetadata(); + assertNotNull(recordMetadata); + assertEquals(recordMetadata.get(MESSAGE_KEY.getKey()), "key"); + assertEquals(recordMetadata.get(MESSAGE_ID.getKey()), pulsarMessage.getMessageId().toString()); - byte[] messageIdBytes = Base64.decode(metadata.getRecordMetadata().get(MESSAGE_ID_BYTES_B64.getKey())); + byte[] messageIdBytes = Base64.decode(recordMetadata.get(MESSAGE_ID_BYTES_B64.getKey())); MessageId messageId = MessageId.fromByteArray(messageIdBytes); assertEquals(MessageId.earliest, messageId); } @Test public void testPulsarSteamMessageUnstitched() { + PulsarConfig config = mock(PulsarConfig.class); + when(config.getEnableKeyValueStitch()).thenReturn(false); String key = "key"; String value = "value"; DummyPulsarMessage dummyPulsarMessage = new DummyPulsarMessage(key.getBytes(StandardCharsets.UTF_8), value.getBytes(StandardCharsets.UTF_8)); - PulsarStreamMessage streamMessage = - PulsarUtils.buildPulsarStreamMessage(dummyPulsarMessage, false, _metadataExtractor); - assertEquals(key.getBytes(StandardCharsets.UTF_8), streamMessage.getKey()); - assertEquals(value.getBytes(StandardCharsets.UTF_8), streamMessage.getValue()); - assertEquals(key.getBytes(StandardCharsets.UTF_8).length, streamMessage.getKeyLength()); - assertEquals(value.getBytes(StandardCharsets.UTF_8).length, streamMessage.getValueLength()); + BytesStreamMessage streamMessage = PulsarUtils.buildPulsarStreamMessage(dummyPulsarMessage, config); + assertEquals(streamMessage.getKey(), key.getBytes(StandardCharsets.UTF_8)); + assertEquals(streamMessage.getValue(), value.getBytes(StandardCharsets.UTF_8)); } @Test public void testPulsarSteamMessageStitched() { + PulsarConfig config = mock(PulsarConfig.class); + when(config.getEnableKeyValueStitch()).thenReturn(true); String key = "key"; String value = "value"; byte[] stitchedValueBytes = PulsarUtils.stitchKeyValue(key.getBytes(StandardCharsets.UTF_8), value.getBytes(StandardCharsets.UTF_8)); DummyPulsarMessage dummyPulsarMessage = new DummyPulsarMessage(key.getBytes(StandardCharsets.UTF_8), value.getBytes(StandardCharsets.UTF_8)); - PulsarStreamMessage streamMessage = - PulsarUtils.buildPulsarStreamMessage(dummyPulsarMessage, true, _metadataExtractor); - assertEquals(key.getBytes(StandardCharsets.UTF_8), streamMessage.getKey()); - assertEquals(stitchedValueBytes, streamMessage.getValue()); - assertEquals(key.getBytes(StandardCharsets.UTF_8).length, streamMessage.getKeyLength()); - assertEquals(stitchedValueBytes.length, streamMessage.getValueLength()); + BytesStreamMessage streamMessage = PulsarUtils.buildPulsarStreamMessage(dummyPulsarMessage, config); + assertEquals(streamMessage.getKey(), key.getBytes(StandardCharsets.UTF_8)); + assertEquals(streamMessage.getValue(), stitchedValueBytes); } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMessage.java b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/BytesStreamMessage.java similarity index 62% rename from pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMessage.java rename to pinot-spi/src/main/java/org/apache/pinot/spi/stream/BytesStreamMessage.java index 1f4f7e999d2a..0b3ffb1fcace 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/main/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMessage.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/BytesStreamMessage.java @@ -16,22 +16,22 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pinot.plugin.stream.kinesis; +package org.apache.pinot.spi.stream; import javax.annotation.Nullable; -import org.apache.pinot.spi.stream.StreamMessage; -public class KinesisStreamMessage extends StreamMessage { - private final String _sequenceNumber; +public class BytesStreamMessage extends StreamMessage { - public KinesisStreamMessage(@Nullable byte[] key, byte[] value, String sequenceNumber, - @Nullable KinesisStreamMessageMetadata metadata, int length) { - super(key, value, metadata, length); - _sequenceNumber = sequenceNumber; + public BytesStreamMessage(@Nullable byte[] key, byte[] value, @Nullable StreamMessageMetadata metadata) { + super(key, value, value.length, metadata); } - public String sequenceNumber() { - return _sequenceNumber; + public BytesStreamMessage(byte[] value, @Nullable StreamMessageMetadata metadata) { + this(null, value, metadata); + } + + public BytesStreamMessage(byte[] value) { + this(value, null); } } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/LongMsgOffset.java b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/LongMsgOffset.java index eb1d4f5d166f..db21f9112a60 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/LongMsgOffset.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/LongMsgOffset.java @@ -50,9 +50,4 @@ public int compareTo(StreamPartitionMsgOffset other) { public String toString() { return Long.toString(_offset); } - - @Override - public StreamPartitionMsgOffset fromString(String longOffset) { - return new LongMsgOffset(longOffset); - } } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/LongMsgOffsetFactory.java b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/LongMsgOffsetFactory.java index 23c1f8a41425..800e16672fa6 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/LongMsgOffsetFactory.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/LongMsgOffsetFactory.java @@ -19,6 +19,7 @@ package org.apache.pinot.spi.stream; public class LongMsgOffsetFactory implements StreamPartitionMsgOffsetFactory { + @Override public void init(StreamConfig streamConfig) { } @@ -27,9 +28,4 @@ public void init(StreamConfig streamConfig) { public StreamPartitionMsgOffset create(String offsetStr) { return new LongMsgOffset(offsetStr); } - - @Override - public StreamPartitionMsgOffset create(StreamPartitionMsgOffset other) { - return new LongMsgOffset(other); - } } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/MessageBatch.java b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/MessageBatch.java index 9a8f4e15fc0b..2f00c826574e 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/MessageBatch.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/MessageBatch.java @@ -24,121 +24,110 @@ /** - * Interface wrapping stream consumer. Throws IndexOutOfBoundsException when trying to access a message at an - * invalid index. - * @param + * Interface wrapping streaming messages. Throws IndexOutOfBoundsException when trying to access a message at an invalid + * index. + * @param type of the stream message values. */ @InterfaceAudience.Public @InterfaceStability.Stable public interface MessageBatch { + /** - * @return number of available messages + * Returns the number of available messages (excluding tombstone). */ int getMessageCount(); /** - * @return number of messages returned from the stream + * Returns the number of messages returned from the stream (including tombstone). */ default int getUnfilteredMessageCount() { return getMessageCount(); } /** - * Returns the message at a particular index inside a set of messages returned from the stream. - * @param index - * @return + * Returns the stream message at the given index within the batch. */ - @Deprecated - T getMessageAtIndex(int index); - - // for backward-compatibility - default byte[] getMessageBytesAtIndex(int index) { - return (byte[]) getMessageAtIndex(index); - } - default StreamMessage getStreamMessage(int index) { - return new LegacyStreamMessage(getMessageBytesAtIndex(index), (StreamMessageMetadata) getMetadataAtIndex(index)); + byte[] value = getMessageBytesAtIndex(index); + StreamMessageMetadata metadata = (StreamMessageMetadata) getMetadataAtIndex(index); + //noinspection unchecked + return (StreamMessage) new StreamMessage<>(value, value.length, metadata); } - class LegacyStreamMessage extends StreamMessage { - public LegacyStreamMessage(byte[] value, StreamMessageMetadata metadata) { - super(value, value.length, metadata); - } - } /** - * Returns the offset of the message at a particular index inside a set of messages returned from the stream. - * @param index - * @return + * Returns the start offset of the next batch. */ - int getMessageOffsetAtIndex(int index); + default StreamPartitionMsgOffset getOffsetOfNextBatch() { + return getNextStreamPartitionMsgOffsetAtIndex(getMessageCount() - 1); + } /** - * Returns the length of the message at a particular index inside a set of messages returned from the stream. - * @param index - * @return + * Returns the offset of the first message (including tombstone) in the batch. + * This is useful to determine if there were gaps in the stream. */ - int getMessageLengthAtIndex(int index); + @Nullable + default StreamPartitionMsgOffset getFirstMessageOffset() { + int numMessages = getMessageCount(); + if (numMessages == 0) { + return null; + } + StreamMessageMetadata firstMessageMetadata = getStreamMessage(0).getMetadata(); + return firstMessageMetadata != null ? firstMessageMetadata.getOffset() : null; + } /** - * Returns the metadata associated with the message at a particular index. This typically includes the timestamp - * when the message was ingested by the upstream stream-provider and other relevant metadata. + * Returns the message metadata for the last message (including tombstone) in the batch. + * This is useful while determining ingestion delay for a message batch. */ - default RowMetadata getMetadataAtIndex(int index) { - return null; + @Nullable + default StreamMessageMetadata getLastMessageMetadata() { + int numMessages = getMessageCount(); + if (numMessages == 0) { + return null; + } + return getStreamMessage(numMessages - 1).getMetadata(); } /** - * Returns the offset of the next message. - * @param index - * @return + * Returns {code true} if the current batch is the end of the consumer, and no more messages can be read from this + * partition group. */ + default boolean isEndOfPartitionGroup() { + return false; + } + @Deprecated - long getNextStreamMessageOffsetAtIndex(int index); + default T getMessageAtIndex(int index) { + throw new UnsupportedOperationException(); + } - /** - * Returns the offset of the next message. - * @param index - * @return - */ - default StreamPartitionMsgOffset getNextStreamPartitionMsgOffsetAtIndex(int index) { - return new LongMsgOffset(getNextStreamMessageOffsetAtIndex(index)); + @Deprecated + default byte[] getMessageBytesAtIndex(int index) { + return (byte[]) getMessageAtIndex(index); } - /** - * @return last offset in the batch - */ - default StreamPartitionMsgOffset getOffsetOfNextBatch() { - return getNextStreamPartitionMsgOffsetAtIndex(getMessageCount() - 1); + @Deprecated + default int getMessageLengthAtIndex(int index) { + throw new UnsupportedOperationException(); } - /** - * Returns true if end of the consumer detects that no more records can be read from this partition group for good - */ - default boolean isEndOfPartitionGroup() { - return false; + @Deprecated + default int getMessageOffsetAtIndex(int index) { + throw new UnsupportedOperationException(); } - /** - * Return the offset of the first message in the batch. - * The first offset of the batch is useful to determine if there were gaps in the stream. - * - * @return null by default - */ - @Nullable - default public StreamPartitionMsgOffset getFirstMessageOffset() { + @Deprecated + default RowMetadata getMetadataAtIndex(int index) { return null; } - /** - * This is useful while determining ingestion delay for a message batch. Retaining metadata for last filtered message - * in a batch can enable us to estimate the ingestion delay for the batch. - * Note that a batch can be fully filtered, and we can still retain the metadata for the last filtered message to - * facilitate computing ingestion delay in the face of a fully filtered batch. - * - * @return null by default. - */ - @Nullable - default public StreamMessageMetadata getLastMessageMetadata() { - return null; + @Deprecated + default long getNextStreamMessageOffsetAtIndex(int index) { + throw new UnsupportedOperationException(); + } + + @Deprecated + default StreamPartitionMsgOffset getNextStreamPartitionMsgOffsetAtIndex(int index) { + return new LongMsgOffset(getNextStreamMessageOffsetAtIndex(index)); } } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/PartitionGroupConsumer.java b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/PartitionGroupConsumer.java index 67c35c10b260..1610e1e1fe61 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/PartitionGroupConsumer.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/PartitionGroupConsumer.java @@ -20,6 +20,8 @@ import java.io.Closeable; import java.util.concurrent.TimeoutException; +import javax.annotation.Nullable; + /** * Consumer interface for consuming from a partition group of a stream @@ -35,7 +37,7 @@ public interface PartitionGroupConsumer extends Closeable { * Poll-based consumers can optionally use this to prefetch metadata from the source. * * This method should be invoked by the caller before trying to invoke - * {@link #fetchMessages(StreamPartitionMsgOffset, StreamPartitionMsgOffset, int)}. + * {@link #fetchMessages(StreamPartitionMsgOffset, int)}. * * @param startOffset Offset (inclusive) at which the consumption should begin */ @@ -49,14 +51,22 @@ default void start(StreamPartitionMsgOffset startOffset) { * on the implementation. * * @param startOffset The offset of the first message desired, inclusive - * @param endOffset The offset of the last message desired, exclusive, or null * @param timeoutMs Timeout in milliseconds - * @throws java.util.concurrent.TimeoutException If the operation could not be completed within {@code timeoutMillis} - * milliseconds - * @return An iterable containing messages fetched from the stream partition and their offsets + * @throws TimeoutException If the operation could not be completed within timeout + * @return A batch of messages from the stream partition group */ - MessageBatch fetchMessages(StreamPartitionMsgOffset startOffset, StreamPartitionMsgOffset endOffset, int timeoutMs) - throws TimeoutException; + default MessageBatch fetchMessages(StreamPartitionMsgOffset startOffset, int timeoutMs) + throws TimeoutException { + return fetchMessages(startOffset, null, timeoutMs); + } + + // Deprecated because the offset is not always monotonically increasing + @Deprecated + default MessageBatch fetchMessages(StreamPartitionMsgOffset startOffset, @Nullable StreamPartitionMsgOffset endOffset, + int timeoutMs) + throws TimeoutException { + throw new UnsupportedOperationException(); + } /** * Checkpoints the consumption state of the stream partition group in the source diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/PartitionLevelConsumer.java b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/PartitionLevelConsumer.java index 2ecc21d04278..73953b2aa422 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/PartitionLevelConsumer.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/PartitionLevelConsumer.java @@ -26,6 +26,7 @@ /** * Interface for a consumer which fetches messages at the partition level of a stream, for given offsets */ +@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public interface PartitionLevelConsumer extends Closeable, PartitionGroupConsumer { @@ -41,8 +42,10 @@ public interface PartitionLevelConsumer extends Closeable, PartitionGroupConsume * @throws java.util.concurrent.TimeoutException */ @Deprecated - MessageBatch fetchMessages(long startOffset, long endOffset, int timeoutMillis) - throws java.util.concurrent.TimeoutException; + default MessageBatch fetchMessages(long startOffset, long endOffset, int timeoutMillis) + throws java.util.concurrent.TimeoutException { + throw new UnsupportedOperationException(); + } /** * Fetch messages and the per-partition high watermark from Kafka between the specified offsets. @@ -55,6 +58,7 @@ MessageBatch fetchMessages(long startOffset, long endOffset, int timeoutMillis) * @return An iterable containing messages fetched from the stream partition and their offsets, as well as the * high watermark for this partition. */ + @Deprecated default MessageBatch fetchMessages(StreamPartitionMsgOffset startOffset, StreamPartitionMsgOffset endOffset, int timeoutMillis) throws java.util.concurrent.TimeoutException { diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/RowMetadata.java b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/RowMetadata.java index 8a5eac3981d4..8c67ca71b4e5 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/RowMetadata.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/RowMetadata.java @@ -18,8 +18,8 @@ */ package org.apache.pinot.spi.stream; -import java.util.Collections; import java.util.Map; +import javax.annotation.Nullable; import org.apache.pinot.spi.annotations.InterfaceAudience; import org.apache.pinot.spi.annotations.InterfaceStability; import org.apache.pinot.spi.data.readers.GenericRow; @@ -34,8 +34,6 @@ @InterfaceAudience.Public @InterfaceStability.Evolving public interface RowMetadata { - GenericRow EMPTY_ROW = new GenericRow(); - Map EMPTY_COLLECTION = Collections.emptyMap(); /** * Returns the timestamp associated with the record. This typically refers to the time it was ingested into the @@ -64,24 +62,34 @@ default long getFirstStreamRecordIngestionTimeMs() { } /** - * Returns the stream message headers - * - * @return A {@link GenericRow} that encapsulates the headers in the ingested row + * Returns the stream offset of the message. + */ + @Nullable + default StreamPartitionMsgOffset getOffset() { + return null; + } + + /** + * Returns the next stream offset of the message. */ + @Nullable + default StreamPartitionMsgOffset getNextOffset() { + return null; + } + + /** + * Returns the stream message headers. + */ + @Nullable default GenericRow getHeaders() { - EMPTY_ROW.clear(); - return EMPTY_ROW; + return null; } /** - * Returns the metadata associated with the stream record - * - * Kafka's record offset would be an example of a metadata associated with the record. Record metadata is typically - * stream specific and hence, it is defined as a Map of strings. - * - * @return A Map of record metadata entries. + * Returns the metadata associated with the stream message. */ + @Nullable default Map getRecordMetadata() { - return EMPTY_COLLECTION; + return null; } } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamConsumerFactory.java b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamConsumerFactory.java index 60e04a2431b2..5729dc28142d 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamConsumerFactory.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamConsumerFactory.java @@ -35,29 +35,6 @@ void init(StreamConfig streamConfig) { _streamConfig = streamConfig; } - /** - * Creates a partition level consumer which can fetch messages from a partitioned stream - * @param clientId a client id to identify the creator of this consumer - * @param partition the partition id of the partition for which this consumer is being created - * @return - */ - public abstract PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int partition); - - /** - * Creates a stream level consumer (high level) which can fetch messages from the stream - * @param clientId a client id to identify the creator of this consumer - * @param tableName the table name for the topic of this consumer - * @param fieldsToRead the fields to read from the source stream - * @param groupId consumer group Id - * @return the stream level consumer - */ - @Deprecated - public StreamLevelConsumer createStreamLevelConsumer(String clientId, String tableName, - Set fieldsToRead, String groupId) { - throw new UnsupportedOperationException("Apache pinot no longer supports stream level consumer model. Please " - + "create partition level consumer only"); - } - /** * Creates a metadata provider which provides partition specific metadata * @param clientId a client id to identify the creator of this consumer @@ -84,4 +61,15 @@ public PartitionGroupConsumer createPartitionGroupConsumer(String clientId, PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) { return createPartitionLevelConsumer(clientId, partitionGroupConsumptionStatus.getPartitionGroupId()); } + + @Deprecated + public PartitionLevelConsumer createPartitionLevelConsumer(String clientId, int partition) { + throw new UnsupportedOperationException(); + } + + @Deprecated + public StreamLevelConsumer createStreamLevelConsumer(String clientId, String tableName, Set fieldsToRead, + String groupId) { + throw new UnsupportedOperationException(); + } } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamDataDecoderImpl.java b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamDataDecoderImpl.java index 89c70acaf8c8..127ecfe12156 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamDataDecoderImpl.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamDataDecoderImpl.java @@ -56,14 +56,15 @@ public StreamDataDecoderResult decode(StreamMessage message) { if (message.getKey() != null) { row.putValue(KEY, new String(message.getKey(), StandardCharsets.UTF_8)); } - RowMetadata metadata = message.getMetadata(); + StreamMessageMetadata metadata = message.getMetadata(); if (metadata != null) { if (metadata.getHeaders() != null) { metadata.getHeaders().getFieldToValueMap() .forEach((key, value) -> row.putValue(HEADER_KEY_PREFIX + key, value)); } - metadata.getRecordMetadata() - .forEach((key, value) -> row.putValue(METADATA_KEY_PREFIX + key, value)); + if (metadata.getRecordMetadata() != null) { + metadata.getRecordMetadata().forEach((key, value) -> row.putValue(METADATA_KEY_PREFIX + key, value)); + } } return new StreamDataDecoderResult(row, null); } else { diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamMessage.java b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamMessage.java index 17f66b6a5a45..e973e92a110b 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamMessage.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamMessage.java @@ -25,7 +25,8 @@ * Represents a Stream message which includes the following components: * 1. record key (optional) * 2. record value (required) - * 3. StreamMessageMetadata (optional) - encapsulates record headers and metadata associated with a stream message + * 3. length of the record value (required) + * 4. StreamMessageMetadata (optional) - encapsulates record headers and metadata associated with a stream message * (such as a message identifier, publish timestamp, user-provided headers etc) * * Similar to value decoder, each implementing stream plugin can have a key decoder and header extractor. @@ -39,41 +40,61 @@ * they will not materialize in the pinot table. */ public class StreamMessage { - private final byte[] _key; - private final T _value; + protected final byte[] _key; + protected final T _value; + protected final int _length; protected final StreamMessageMetadata _metadata; - int _length = -1; - public StreamMessage(@Nullable byte[] key, T value, @Nullable StreamMessageMetadata metadata, int length) { + public StreamMessage(@Nullable byte[] key, T value, int length, @Nullable StreamMessageMetadata metadata) { _key = key; _value = value; - _metadata = metadata; _length = length; + _metadata = metadata; + } + + public StreamMessage(T value, int length, @Nullable StreamMessageMetadata metadata) { + this(null, value, length, metadata); } public StreamMessage(T value, int length) { this(value, length, null); } - public StreamMessage(T value, int length, @Nullable StreamMessageMetadata metadata) { - this(null, value, metadata, length); + @Deprecated + public StreamMessage(@Nullable byte[] key, T value, @Nullable StreamMessageMetadata metadata, int length) { + _key = key; + _value = value; + _metadata = metadata; + _length = length; + } + + /** + * Returns the key of the message. + */ + @Nullable + public byte[] getKey() { + return _key; } + /** + * Returns the content of the message. + */ public T getValue() { return _value; } + /** + * Returns the length of the message content. + */ public int getLength() { return _length; } + /** + * Returns the metadata of the message. + */ @Nullable public StreamMessageMetadata getMetadata() { return _metadata; } - - @Nullable - public byte[] getKey() { - return _key; - } } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamMessageMetadata.java b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamMessageMetadata.java index 557069a5815a..e0edcb580178 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamMessageMetadata.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamMessageMetadata.java @@ -18,7 +18,6 @@ */ package org.apache.pinot.spi.stream; -import java.util.Collections; import java.util.Map; import javax.annotation.Nullable; import org.apache.pinot.spi.data.readers.GenericRow; @@ -31,34 +30,39 @@ public class StreamMessageMetadata implements RowMetadata { private final long _recordIngestionTimeMs; private final long _firstStreamRecordIngestionTimeMs; + private final StreamPartitionMsgOffset _offset; + private final StreamPartitionMsgOffset _nextOffset; private final GenericRow _headers; private final Map _metadata; + @Deprecated public StreamMessageMetadata(long recordIngestionTimeMs) { - this(recordIngestionTimeMs, Long.MIN_VALUE, null, Collections.emptyMap()); + this(recordIngestionTimeMs, null); } + @Deprecated public StreamMessageMetadata(long recordIngestionTimeMs, @Nullable GenericRow headers) { - this(recordIngestionTimeMs, Long.MIN_VALUE, headers, Collections.emptyMap()); + this(recordIngestionTimeMs, headers, null); } - public StreamMessageMetadata(long recordIngestionTimeMs, @Nullable GenericRow headers, - Map metadata) { + @Deprecated + public StreamMessageMetadata(long recordIngestionTimeMs, @Nullable GenericRow headers, Map metadata) { this(recordIngestionTimeMs, Long.MIN_VALUE, headers, metadata); } - /** - * Construct the stream based message/row message metadata - * - * @param recordIngestionTimeMs the time that the message was ingested by the stream provider. - * use Long.MIN_VALUE if not applicable - * @param firstStreamRecordIngestionTimeMs the time that the message was ingested by the first stream provider - * in the ingestion pipeline. use Long.MIN_VALUE if not applicable - * @param metadata - */ + + @Deprecated public StreamMessageMetadata(long recordIngestionTimeMs, long firstStreamRecordIngestionTimeMs, @Nullable GenericRow headers, Map metadata) { + this(recordIngestionTimeMs, firstStreamRecordIngestionTimeMs, null, null, headers, metadata); + } + + public StreamMessageMetadata(long recordIngestionTimeMs, long firstStreamRecordIngestionTimeMs, + @Nullable StreamPartitionMsgOffset offset, @Nullable StreamPartitionMsgOffset nextOffset, + @Nullable GenericRow headers, @Nullable Map metadata) { _recordIngestionTimeMs = recordIngestionTimeMs; _firstStreamRecordIngestionTimeMs = firstStreamRecordIngestionTimeMs; + _offset = offset; + _nextOffset = nextOffset; _headers = headers; _metadata = metadata; } @@ -73,13 +77,67 @@ public long getFirstStreamRecordIngestionTimeMs() { return _firstStreamRecordIngestionTimeMs; } + @Nullable + @Override + public StreamPartitionMsgOffset getOffset() { + return _offset; + } + + @Nullable + @Override + public StreamPartitionMsgOffset getNextOffset() { + return _nextOffset; + } + + @Nullable @Override public GenericRow getHeaders() { return _headers; } + @Nullable @Override public Map getRecordMetadata() { return _metadata; } + + public static class Builder { + private long _recordIngestionTimeMs = Long.MIN_VALUE; + private long _firstStreamRecordIngestionTimeMs = Long.MIN_VALUE; + private StreamPartitionMsgOffset _offset; + private StreamPartitionMsgOffset _nextOffset; + private GenericRow _headers; + private Map _metadata; + + public Builder setRecordIngestionTimeMs(long recordIngestionTimeMs) { + _recordIngestionTimeMs = recordIngestionTimeMs; + return this; + } + + public Builder setFirstStreamRecordIngestionTimeMs(long firstStreamRecordIngestionTimeMs) { + _firstStreamRecordIngestionTimeMs = firstStreamRecordIngestionTimeMs; + return this; + } + + public Builder setOffset(StreamPartitionMsgOffset offset, StreamPartitionMsgOffset nextOffset) { + _offset = offset; + _nextOffset = nextOffset; + return this; + } + + public Builder setHeaders(GenericRow headers) { + _headers = headers; + return this; + } + + public Builder setMetadata(Map metadata) { + _metadata = metadata; + return this; + } + + public StreamMessageMetadata build() { + return new StreamMessageMetadata(_recordIngestionTimeMs, _firstStreamRecordIngestionTimeMs, _offset, _nextOffset, + _headers, _metadata); + } + } } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamPartitionMsgOffset.java b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamPartitionMsgOffset.java index 6315debb3ce7..1822067da0bc 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamPartitionMsgOffset.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/stream/StreamPartitionMsgOffset.java @@ -48,7 +48,10 @@ public interface StreamPartitionMsgOffset extends Comparable message = new StreamMessage(value.getBytes(StandardCharsets.UTF_8), - value.getBytes(StandardCharsets.UTF_8).length, null); + BytesStreamMessage message = new BytesStreamMessage(value.getBytes(StandardCharsets.UTF_8)); StreamDataDecoderResult result = new StreamDataDecoderImpl(messageDecoder).decode(message); Assert.assertNotNull(result); Assert.assertNull(result.getException()); @@ -63,9 +62,8 @@ public void testDecodeKeyAndHeaders() headers.putValue(AGE_HEADER_KEY, 3); Map recordMetadata = Collections.singletonMap(SEQNO_RECORD_METADATA, "1"); StreamMessageMetadata metadata = new StreamMessageMetadata(1234L, headers, recordMetadata); - byte[] valueBytes = value.getBytes(StandardCharsets.UTF_8); - StreamMessage message = - new StreamMessage(key.getBytes(StandardCharsets.UTF_8), valueBytes, metadata, value.length()); + BytesStreamMessage message = + new BytesStreamMessage(key.getBytes(StandardCharsets.UTF_8), value.getBytes(StandardCharsets.UTF_8), metadata); StreamDataDecoderResult result = new StreamDataDecoderImpl(messageDecoder).decode(message); Assert.assertNotNull(result); @@ -86,8 +84,7 @@ public void testNoExceptionIsThrown() ThrowingDecoder messageDecoder = new ThrowingDecoder(); messageDecoder.init(Collections.emptyMap(), ImmutableSet.of(NAME_FIELD), ""); String value = "Alice"; - StreamMessage message = new StreamMessage(value.getBytes(StandardCharsets.UTF_8), - value.getBytes(StandardCharsets.UTF_8).length, null); + BytesStreamMessage message = new BytesStreamMessage(value.getBytes(StandardCharsets.UTF_8)); StreamDataDecoderResult result = new StreamDataDecoderImpl(messageDecoder).decode(message); Assert.assertNotNull(result); Assert.assertNotNull(result.getException()); @@ -98,7 +95,8 @@ class ThrowingDecoder implements StreamMessageDecoder { @Override public void init(Map props, Set fieldsToRead, String topicName) - throws Exception { } + throws Exception { + } @Nullable @Override @@ -113,11 +111,11 @@ public GenericRow decode(byte[] payload, int offset, int length, GenericRow dest } } - class TestDecoder implements StreamMessageDecoder { @Override public void init(Map props, Set fieldsToRead, String topicName) - throws Exception { } + throws Exception { + } @Nullable @Override diff --git a/pinot-spi/src/test/java/org/apache/pinot/spi/stream/StreamMessageTest.java b/pinot-spi/src/test/java/org/apache/pinot/spi/stream/StreamMessageTest.java deleted file mode 100644 index d0d028709fda..000000000000 --- a/pinot-spi/src/test/java/org/apache/pinot/spi/stream/StreamMessageTest.java +++ /dev/null @@ -1,55 +0,0 @@ -/** - * 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.spi.stream; - -import java.nio.charset.StandardCharsets; -import org.testng.Assert; -import org.testng.annotations.Test; - - -public class StreamMessageTest { - - @Test - public void testAllowNullKeyAndMetadata() { - String value = "hello"; - byte[] valBytes = value.getBytes(StandardCharsets.UTF_8); - StreamMessage msg = new StreamMessage(valBytes, valBytes.length, null); - Assert.assertNull(msg.getKey()); - Assert.assertNull(msg.getMetadata()); - Assert.assertEquals(new String(msg.getValue()), value); - - value = "value"; - valBytes = value.getBytes(StandardCharsets.UTF_8); - StreamMessage msg1 = - new StreamMessage("key".getBytes(StandardCharsets.UTF_8), valBytes, null, valBytes.length); - Assert.assertNotNull(msg1.getKey()); - Assert.assertEquals(new String(msg1.getKey()), "key"); - Assert.assertNotNull(msg1.getValue()); - Assert.assertEquals(new String(msg1.getValue()), value); - Assert.assertNull(msg1.getMetadata()); - - StreamMessage msg2 = new StreamMessage<>("key".getBytes(StandardCharsets.UTF_8), value, null, - value.length()); - Assert.assertNotNull(msg2.getKey()); - Assert.assertEquals(new String(msg2.getKey()), "key"); - Assert.assertNotNull(msg2.getValue()); - Assert.assertEquals(msg2.getValue(), value); - Assert.assertNull(msg2.getMetadata()); - } -} From 32a02bc1a31de0ff02851e0b983df69e15bdd271 Mon Sep 17 00:00:00 2001 From: "Xiaotian (Jackie) Jiang" <17555551+Jackie-Jiang@users.noreply.github.com> Date: Sun, 7 Apr 2024 18:21:24 -0700 Subject: [PATCH 02/72] Fix some resource leak in tests (#12794) --- ...leSmartTDigestAggregationFunctionTest.java | 5 - ...leSegmentImplIngestionAggregationTest.java | 2 + .../VarByteChunkSVForwardIndexWriterTest.java | 10 +- .../segment/index/ColumnMetadataTest.java | 18 +- .../MultiValueVarByteRawIndexCreatorTest.java | 36 ++-- .../SegmentGenerationWithBytesTypeTest.java | 155 +++++++----------- .../spi/memory/unsafe/DirectMemory.java | 14 +- .../segment/spi/memory/unsafe/MmapMemory.java | 13 +- 8 files changed, 99 insertions(+), 154 deletions(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/function/PercentileSmartTDigestAggregationFunctionTest.java b/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/function/PercentileSmartTDigestAggregationFunctionTest.java index b1eb471c704e..68a180ea886d 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/function/PercentileSmartTDigestAggregationFunctionTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/function/PercentileSmartTDigestAggregationFunctionTest.java @@ -78,10 +78,5 @@ String expectedAggrWithoutNull75(Scenario scenario) { String expectedAggrWithoutNull90(Scenario scenario) { return "7.100000000000001"; } - - @Override - String expectedAggrWithoutNull100(Scenario scenario) { - return super.expectedAggrWithoutNull100(scenario); - } } } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplIngestionAggregationTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplIngestionAggregationTest.java index 5e048520c0ed..711634ea179c 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplIngestionAggregationTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplIngestionAggregationTest.java @@ -480,6 +480,8 @@ public void testBigDecimalTooBig() { Assert.assertThrows(IllegalArgumentException.class, () -> { mutableSegmentImpl.index(row, defaultMetadata); }); + + mutableSegmentImpl.destroy(); } private BigDecimal generateRandomBigDecimal(Random random, int maxPrecision, int scale) { diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkSVForwardIndexWriterTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkSVForwardIndexWriterTest.java index 5a46c9d3f380..a9b8c34bb538 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkSVForwardIndexWriterTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkSVForwardIndexWriterTest.java @@ -35,7 +35,7 @@ import org.apache.pinot.segment.spi.V1Constants; import org.apache.pinot.segment.spi.compression.ChunkCompressionType; import org.apache.pinot.segment.spi.memory.PinotDataBuffer; -import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.FieldSpec.DataType; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.DataProvider; @@ -90,8 +90,8 @@ public void testPutStrings(ChunkCompressionType compressionType, int totalDocs, writer.putStringMV(array); } } - try (VarByteChunkSVForwardIndexReader reader = new VarByteChunkSVForwardIndexReader( - PinotDataBuffer.loadBigEndianFile(file), FieldSpec.DataType.STRING); + try (PinotDataBuffer dataBuffer = PinotDataBuffer.loadBigEndianFile(file); + VarByteChunkSVForwardIndexReader reader = new VarByteChunkSVForwardIndexReader(dataBuffer, DataType.STRING); ChunkReaderContext context = reader.createContext()) { for (int i = 0; i < arrays.size(); i++) { String[] array = arrays.get(i); @@ -125,8 +125,8 @@ public void testPutBytes(ChunkCompressionType compressionType, int totalDocs, in writer.putBytesMV(Arrays.stream(array).map(str -> str.getBytes(UTF_8)).toArray(byte[][]::new)); } } - try (VarByteChunkSVForwardIndexReader reader = new VarByteChunkSVForwardIndexReader( - PinotDataBuffer.loadBigEndianFile(file), FieldSpec.DataType.BYTES); + try (PinotDataBuffer dataBuffer = PinotDataBuffer.loadBigEndianFile(file); + VarByteChunkSVForwardIndexReader reader = new VarByteChunkSVForwardIndexReader(dataBuffer, DataType.BYTES); ChunkReaderContext context = reader.createContext()) { for (int i = 0; i < arrays.size(); i++) { String[] array = arrays.get(i); diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/ColumnMetadataTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/ColumnMetadataTest.java index b8c1f5851948..dbee83216c91 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/ColumnMetadataTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/ColumnMetadataTest.java @@ -29,22 +29,20 @@ import org.apache.commons.configuration2.PropertiesConfiguration; import org.apache.commons.configuration2.ex.ConfigurationException; import org.apache.commons.io.FileUtils; -import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader; import org.apache.pinot.segment.local.segment.creator.SegmentTestUtils; import org.apache.pinot.segment.local.segment.creator.impl.SegmentCreationDriverFactory; import org.apache.pinot.segment.spi.ColumnMetadata; -import org.apache.pinot.segment.spi.IndexSegment; import org.apache.pinot.segment.spi.SegmentMetadata; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver; import org.apache.pinot.segment.spi.index.metadata.ColumnMetadataImpl; +import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; import org.apache.pinot.segment.spi.partition.BoundedColumnValuePartitionFunction; import org.apache.pinot.spi.config.table.ColumnPartitionConfig; import org.apache.pinot.spi.config.table.SegmentPartitionConfig; import org.apache.pinot.spi.data.DimensionFieldSpec; import org.apache.pinot.spi.data.FieldSpec.DataType; import org.apache.pinot.spi.env.CommonsConfigurationUtils; -import org.apache.pinot.spi.utils.ReadMode; import org.apache.pinot.util.TestUtils; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -140,8 +138,7 @@ public void testAllFieldsInitialized() driver.build(); // Load segment metadata. - IndexSegment segment = ImmutableSegmentLoader.load(INDEX_DIR.listFiles()[0], ReadMode.mmap); - SegmentMetadata segmentMetadata = segment.getSegmentMetadata(); + SegmentMetadata segmentMetadata = new SegmentMetadataImpl(INDEX_DIR.listFiles()[0]); verifySegmentAfterLoading(segmentMetadata); // Make sure we got the creator name as well. @@ -159,8 +156,7 @@ public void testAllFieldsExceptCreatorName() driver.build(); // Load segment metadata. - IndexSegment segment = ImmutableSegmentLoader.load(INDEX_DIR.listFiles()[0], ReadMode.mmap); - SegmentMetadata segmentMetadata = segment.getSegmentMetadata(); + SegmentMetadata segmentMetadata = new SegmentMetadataImpl(INDEX_DIR.listFiles()[0]); verifySegmentAfterLoading(segmentMetadata); // Make sure we get null for creator name. @@ -177,9 +173,8 @@ public void testPaddingCharacter() driver.build(); // Load segment metadata. - IndexSegment segment = ImmutableSegmentLoader.load(INDEX_DIR.listFiles()[0], ReadMode.mmap); - SegmentMetadata metadata = segment.getSegmentMetadata(); - verifySegmentAfterLoading(metadata); + SegmentMetadata segmentMetadata = new SegmentMetadataImpl(INDEX_DIR.listFiles()[0]); + verifySegmentAfterLoading(segmentMetadata); } @Test @@ -198,8 +193,7 @@ public void testSegmentPartitionedWithBoundedColumnValue() driver.build(); // Load segment metadata. - IndexSegment segment = ImmutableSegmentLoader.load(INDEX_DIR.listFiles()[0], ReadMode.mmap); - SegmentMetadata segmentMetadata = segment.getSegmentMetadata(); + SegmentMetadata segmentMetadata = new SegmentMetadataImpl(INDEX_DIR.listFiles()[0]); verifySegmentAfterLoading(segmentMetadata); // Make sure we get null for creator name. Assert.assertNull(segmentMetadata.getCreatorName()); diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/MultiValueVarByteRawIndexCreatorTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/MultiValueVarByteRawIndexCreatorTest.java index 32f4ff1a2c83..7b382b4dc5e4 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/MultiValueVarByteRawIndexCreatorTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/MultiValueVarByteRawIndexCreatorTest.java @@ -126,14 +126,15 @@ public void testMVString(ChunkCompressionType compressionType, boolean useFullSi } //read - final PinotDataBuffer buffer = PinotDataBuffer.mapFile(file, true, 0, file.length(), ByteOrder.BIG_ENDIAN, ""); - ForwardIndexReader reader = ForwardIndexReaderFactory.createRawIndexReader(buffer, DataType.STRING, false); - final ForwardIndexReaderContext context = reader.createContext(); - String[] values = new String[maxElements]; - for (int i = 0; i < numDocs; i++) { - int length = reader.getStringMV(i, values, context); - String[] readValue = Arrays.copyOf(values, length); - Assert.assertEquals(inputs.get(i), readValue); + try (PinotDataBuffer buffer = PinotDataBuffer.mapFile(file, true, 0, file.length(), ByteOrder.BIG_ENDIAN, ""); + ForwardIndexReader reader = ForwardIndexReaderFactory.createRawIndexReader(buffer, DataType.STRING, false); + ForwardIndexReaderContext context = reader.createContext()) { + String[] values = new String[maxElements]; + for (int i = 0; i < numDocs; i++) { + int length = reader.getStringMV(i, values, context); + String[] readValue = Arrays.copyOf(values, length); + Assert.assertEquals(inputs.get(i), readValue); + } } } @@ -177,15 +178,16 @@ public void testMVBytes(ChunkCompressionType compressionType, boolean useFullSiz } //read - final PinotDataBuffer buffer = PinotDataBuffer.mapFile(file, true, 0, file.length(), ByteOrder.BIG_ENDIAN, ""); - ForwardIndexReader reader = ForwardIndexReaderFactory.createRawIndexReader(buffer, DataType.BYTES, false); - final ForwardIndexReaderContext context = reader.createContext(); - byte[][] values = new byte[maxElements][]; - for (int i = 0; i < numDocs; i++) { - int length = reader.getBytesMV(i, values, context); - byte[][] readValue = Arrays.copyOf(values, length); - for (int j = 0; j < length; j++) { - Assert.assertTrue(Arrays.equals(inputs.get(i)[j], readValue[j])); + try (PinotDataBuffer buffer = PinotDataBuffer.mapFile(file, true, 0, file.length(), ByteOrder.BIG_ENDIAN, ""); + ForwardIndexReader reader = ForwardIndexReaderFactory.createRawIndexReader(buffer, DataType.BYTES, false); + ForwardIndexReaderContext context = reader.createContext()) { + byte[][] values = new byte[maxElements][]; + for (int i = 0; i < numDocs; i++) { + int length = reader.getBytesMV(i, values, context); + byte[][] readValue = Arrays.copyOf(values, length); + for (int j = 0; j < length; j++) { + Assert.assertTrue(Arrays.equals(inputs.get(i)[j], readValue[j])); + } } } } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/SegmentGenerationWithBytesTypeTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/SegmentGenerationWithBytesTypeTest.java index 5c96f4b39600..7e33ad173ed8 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/SegmentGenerationWithBytesTypeTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/SegmentGenerationWithBytesTypeTest.java @@ -27,7 +27,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Random; import org.apache.avro.file.DataFileWriter; import org.apache.avro.generic.GenericData; @@ -37,7 +36,6 @@ import org.apache.pinot.segment.local.aggregator.PercentileTDigestValueAggregator; import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader; import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; -import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.segment.local.segment.index.readers.BaseImmutableDictionary; import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.local.segment.readers.PinotSegmentRecordReader; @@ -45,26 +43,25 @@ import org.apache.pinot.segment.spi.IndexSegment; import org.apache.pinot.segment.spi.SegmentMetadata; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; -import org.apache.pinot.segment.spi.loader.SegmentDirectoryLoaderContext; -import org.apache.pinot.segment.spi.loader.SegmentDirectoryLoaderRegistry; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; -import org.apache.pinot.spi.data.DimensionFieldSpec; -import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.FieldSpec.DataType; import org.apache.pinot.spi.data.MetricFieldSpec; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.data.readers.GenericRow; import org.apache.pinot.spi.data.readers.RecordReader; -import org.apache.pinot.spi.env.PinotConfiguration; import org.apache.pinot.spi.utils.ByteArray; import org.apache.pinot.spi.utils.BytesUtils; import org.apache.pinot.spi.utils.ReadMode; import org.apache.pinot.spi.utils.builder.TableConfigBuilder; -import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; + /** * Class for testing segment generation with byte[] data type. @@ -89,92 +86,70 @@ public class SegmentGenerationWithBytesTypeTest { private static final String VARIABLE_BYTES_COLUMN = "variableBytes"; private Random _random; - private RecordReader _recordReader; - private Schema _schema; + private List _rows; private TableConfig _tableConfig; + private Schema _schema; private ImmutableSegment _segment; /** * Setup to build a segment with raw indexes (no-dictionary) of various data types. - * - * @throws Exception */ @BeforeClass - public void setup() + public void setUp() throws Exception { - - _schema = new Schema(); - _schema.addField(new DimensionFieldSpec(FIXED_BYTE_SORTED_COLUMN, FieldSpec.DataType.BYTES, true)); - _schema.addField(new DimensionFieldSpec(FIXED_BYTES_UNSORTED_COLUMN, FieldSpec.DataType.BYTES, true)); - _schema.addField(new DimensionFieldSpec(FIXED_BYTES_NO_DICT_COLUMN, FieldSpec.DataType.BYTES, true)); - _schema.addField(new DimensionFieldSpec(VARIABLE_BYTES_COLUMN, FieldSpec.DataType.BYTES, true)); - - _tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName("test").build(); - _random = new Random(System.nanoTime()); - _recordReader = buildIndex(_schema); + _rows = generateRows(); + _tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName("test").build(); + _schema = new Schema.SchemaBuilder().setSchemaName("test") + .addSingleValueDimension(FIXED_BYTE_SORTED_COLUMN, DataType.BYTES) + .addSingleValueDimension(FIXED_BYTES_UNSORTED_COLUMN, DataType.BYTES) + .addSingleValueDimension(FIXED_BYTES_NO_DICT_COLUMN, DataType.BYTES) + .addSingleValueDimension(VARIABLE_BYTES_COLUMN, DataType.BYTES).build(); + buildSegment(); _segment = ImmutableSegmentLoader.load(new File(SEGMENT_DIR_NAME, SEGMENT_NAME), ReadMode.heap); } - /** - * Clean up after test - */ @AfterClass - public void cleanup() + public void tearDown() throws IOException { - _recordReader.close(); _segment.destroy(); FileUtils.deleteQuietly(new File(SEGMENT_DIR_NAME)); FileUtils.deleteQuietly(new File(AVRO_DIR_NAME)); } @Test - public void test() + public void testRecords() throws Exception { - PinotSegmentRecordReader pinotReader = new PinotSegmentRecordReader(new File(SEGMENT_DIR_NAME, SEGMENT_NAME)); - - _recordReader.rewind(); - while (pinotReader.hasNext()) { - GenericRow expectedRow = _recordReader.next(); - GenericRow actualRow = pinotReader.next(); - - for (String column : _schema.getColumnNames()) { - byte[] actual = (byte[]) actualRow.getValue(column); - byte[] expected = (byte[]) expectedRow.getValue(column); - - if (ByteArray.compare(actual, expected) != 0) { - Assert.assertEquals(actualRow.getValue(column), expectedRow.getValue(column)); - } + try (PinotSegmentRecordReader recordReader = new PinotSegmentRecordReader()) { + recordReader.init(_segment); + for (int i = 0; i < NUM_ROWS; i++) { + assertEquals(recordReader.next(), _rows.get(i)); } + assertFalse(recordReader.hasNext()); } - - // Ensure both record readers are exhausted, ie same number of rows. - Assert.assertFalse(_recordReader.hasNext()); - pinotReader.close(); } @Test public void testMetadata() { - Assert.assertTrue(_segment.getDataSource(FIXED_BYTE_SORTED_COLUMN).getDataSourceMetadata().isSorted()); - Assert.assertFalse(_segment.getSegmentMetadata().getColumnMetadataFor(FIXED_BYTES_NO_DICT_COLUMN).hasDictionary()); + assertTrue(_segment.getDataSource(FIXED_BYTE_SORTED_COLUMN).getDataSourceMetadata().isSorted()); + assertFalse(_segment.getSegmentMetadata().getColumnMetadataFor(FIXED_BYTES_NO_DICT_COLUMN).hasDictionary()); } @Test public void testDictionary() { BaseImmutableDictionary dictionary = (BaseImmutableDictionary) _segment.getDictionary(FIXED_BYTE_SORTED_COLUMN); - Assert.assertEquals(dictionary.length(), NUM_SORTED_VALUES); + assertEquals(dictionary.length(), NUM_SORTED_VALUES); // Test dictionary indexing. for (int i = 0; i < NUM_ROWS; i++) { int value = (i * NUM_SORTED_VALUES) / NUM_ROWS; // For sorted columns, values are written as 0, 0, 0.., 1, 1, 1...n, n, n - Assert.assertEquals(dictionary.indexOf(BytesUtils.toHexString(Ints.toByteArray(value))), - value % NUM_SORTED_VALUES); + assertEquals(dictionary.indexOf(BytesUtils.toHexString(Ints.toByteArray(value))), value % NUM_SORTED_VALUES); } // Test value not in dictionary. - Assert.assertEquals(dictionary.indexOf(BytesUtils.toHexString(Ints.toByteArray(NUM_SORTED_VALUES + 1))), -1); - Assert.assertEquals(dictionary.insertionIndexOf(BytesUtils.toHexString(Ints.toByteArray(NUM_SORTED_VALUES + 1))), + assertEquals(dictionary.indexOf(BytesUtils.toHexString(Ints.toByteArray(NUM_SORTED_VALUES + 1))), -1); + assertEquals(dictionary.insertionIndexOf(BytesUtils.toHexString(Ints.toByteArray(NUM_SORTED_VALUES + 1))), -(NUM_SORTED_VALUES + 1)); int[] dictIds = new int[NUM_SORTED_VALUES]; @@ -186,7 +161,7 @@ public void testDictionary() { dictionary.readBytesValues(dictIds, NUM_SORTED_VALUES, values); for (int expected = 0; expected < NUM_SORTED_VALUES; expected++) { int actual = ByteBuffer.wrap(values[expected]).asIntBuffer().get(); - Assert.assertEquals(actual, expected); + assertEquals(actual, expected); } } @@ -197,8 +172,8 @@ public void testDictionary() { public void testTDigestAvro() throws Exception { Schema schema = new Schema(); - schema.addField(new MetricFieldSpec(FIXED_BYTES_UNSORTED_COLUMN, FieldSpec.DataType.BYTES)); - schema.addField(new MetricFieldSpec(VARIABLE_BYTES_COLUMN, FieldSpec.DataType.BYTES)); + schema.addField(new MetricFieldSpec(FIXED_BYTES_UNSORTED_COLUMN, DataType.BYTES)); + schema.addField(new MetricFieldSpec(VARIABLE_BYTES_COLUMN, DataType.BYTES)); List fixedExpected = new ArrayList<>(NUM_ROWS); List varExpected = new ArrayList<>(NUM_ROWS); @@ -207,38 +182,24 @@ public void testTDigestAvro() IndexSegment segment = buildSegmentFromAvro(schema, AVRO_DIR_NAME, AVRO_NAME, SEGMENT_NAME); SegmentMetadata metadata = segment.getSegmentMetadata(); - - Assert.assertTrue(metadata.getColumnMetadataFor(FIXED_BYTES_UNSORTED_COLUMN).hasDictionary()); - Assert.assertTrue(metadata.getColumnMetadataFor(VARIABLE_BYTES_COLUMN).hasDictionary()); - - PinotSegmentRecordReader reader = new PinotSegmentRecordReader(new File(AVRO_DIR_NAME, SEGMENT_NAME)); - GenericRow row = new GenericRow(); - - int i = 0; - while (reader.hasNext()) { - row = reader.next(row); - Assert.assertEquals(ByteArray.compare((byte[]) row.getValue(FIXED_BYTES_UNSORTED_COLUMN), fixedExpected.get(i)), - 0); - Assert.assertEquals(ByteArray.compare((byte[]) row.getValue(VARIABLE_BYTES_COLUMN), varExpected.get(i++)), 0); + assertTrue(metadata.getColumnMetadataFor(FIXED_BYTES_UNSORTED_COLUMN).hasDictionary()); + assertTrue(metadata.getColumnMetadataFor(VARIABLE_BYTES_COLUMN).hasDictionary()); + + try (PinotSegmentRecordReader reader = new PinotSegmentRecordReader()) { + reader.init(segment); + GenericRow row = new GenericRow(); + int i = 0; + while (reader.hasNext()) { + row = reader.next(row); + assertEquals(ByteArray.compare((byte[]) row.getValue(FIXED_BYTES_UNSORTED_COLUMN), fixedExpected.get(i)), 0); + assertEquals(ByteArray.compare((byte[]) row.getValue(VARIABLE_BYTES_COLUMN), varExpected.get(i++)), 0); + } } + segment.destroy(); } - /** - * Helper method to build a segment containing a single valued string column with RAW (no-dictionary) index. - * - * @return Array of string values for the rows in the generated index. - * @throws Exception - */ - - private RecordReader buildIndex(Schema schema) - throws Exception { - SegmentGeneratorConfig config = new SegmentGeneratorConfig(_tableConfig, schema); - - config.setOutDir(SEGMENT_DIR_NAME); - config.setSegmentName(SEGMENT_NAME); - config.setRawIndexCreationColumns(Collections.singletonList(FIXED_BYTES_NO_DICT_COLUMN)); - + private List generateRows() { List rows = new ArrayList<>(NUM_ROWS); for (int i = 0; i < NUM_ROWS; i++) { HashMap map = new HashMap<>(); @@ -264,19 +225,21 @@ private RecordReader buildIndex(Schema schema) genericRow.init(map); rows.add(genericRow); } + return rows; + } - RecordReader recordReader = new GenericRowRecordReader(rows); - SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl(); - driver.init(config, recordReader); - driver.build(); + private void buildSegment() + throws Exception { + SegmentGeneratorConfig config = new SegmentGeneratorConfig(_tableConfig, _schema); + config.setOutDir(SEGMENT_DIR_NAME); + config.setSegmentName(SEGMENT_NAME); + config.setRawIndexCreationColumns(Collections.singletonList(FIXED_BYTES_NO_DICT_COLUMN)); - Map props = new HashMap<>(); - props.put(IndexLoadingConfig.READ_MODE_KEY, ReadMode.mmap.toString()); - SegmentDirectoryLoaderRegistry.getDefaultSegmentDirectoryLoader().load(driver.getOutputDirectory().toURI(), - new SegmentDirectoryLoaderContext.Builder().setTableConfig(_tableConfig) - .setSegmentDirectoryConfigs(new PinotConfiguration(props)).build()); - recordReader.rewind(); - return recordReader; + try (RecordReader recordReader = new GenericRowRecordReader(_rows)) { + SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl(); + driver.init(config, recordReader); + driver.build(); + } } /** diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/memory/unsafe/DirectMemory.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/memory/unsafe/DirectMemory.java index aef92af1dffd..04e13a2572cd 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/memory/unsafe/DirectMemory.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/memory/unsafe/DirectMemory.java @@ -18,7 +18,6 @@ */ package org.apache.pinot.segment.spi.memory.unsafe; -import java.io.IOException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,15 +54,10 @@ public void flush() { } @Override - public void close() - throws IOException { + public synchronized void close() { if (!_closed) { - synchronized (this) { - if (!_closed) { - Unsafer.UNSAFE.freeMemory(_address); - _closed = true; - } - } + Unsafer.UNSAFE.freeMemory(_address); + _closed = true; } } @@ -71,7 +65,7 @@ public void close() protected void finalize() throws Throwable { if (!_closed) { - LOGGER.warn("Mmap section of " + _size + " wasn't explicitly closed"); + LOGGER.warn("Direct memory of size: {} wasn't explicitly closed", _size); close(); } super.finalize(); diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/memory/unsafe/MmapMemory.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/memory/unsafe/MmapMemory.java index 521d7f4cdf35..ce91c255fa02 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/memory/unsafe/MmapMemory.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/memory/unsafe/MmapMemory.java @@ -98,16 +98,11 @@ public void flush() { } @Override - public void close() - throws IOException { + public synchronized void close() { try { if (!_closed) { - synchronized (this) { - if (!_closed) { - _section._unmapFun.unmap(); - _closed = true; - } - } + _section._unmapFun.unmap(); + _closed = true; } } catch (InvocationTargetException | IllegalAccessException e) { throw new RuntimeException("Error while calling unmap", e); @@ -118,7 +113,7 @@ public void close() protected void finalize() throws Throwable { if (!_closed) { - LOGGER.warn("Mmap section of " + _size + " wasn't explicitly closed"); + LOGGER.warn("Mmap section of size: {} wasn't explicitly closed", _size); close(); } super.finalize(); From a0a2171d349979461c34e58b1ec7f98f6e8e615b Mon Sep 17 00:00:00 2001 From: "Xiaotian (Jackie) Jiang" <17555551+Jackie-Jiang@users.noreply.github.com> Date: Sun, 7 Apr 2024 18:22:10 -0700 Subject: [PATCH 03/72] Fix the time segment pruner on TIMESTAMP data type (#12789) --- .../segmentpruner/SegmentPrunerFactory.java | 44 +- .../segmentpruner/TimeSegmentPruner.java | 136 ++-- .../segmentpruner/SegmentPrunerTest.java | 621 ++++++++---------- .../apache/pinot/common/data/SchemaTest.java | 29 +- .../pinot/spi/data/DateTimeFieldSpec.java | 17 +- 5 files changed, 375 insertions(+), 472 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerFactory.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerFactory.java index 6135982e185c..423eb527faee 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerFactory.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerFactory.java @@ -18,8 +18,6 @@ */ package org.apache.pinot.broker.routing.segmentpruner; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -37,7 +35,6 @@ import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.data.DateTimeFieldSpec; -import org.apache.pinot.spi.data.DateTimeFormatSpec; import org.apache.pinot.spi.data.Schema; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,7 +65,7 @@ public static List getSegmentPruners(TableConfig tableConfig, List configuredSegmentPruners = new ArrayList<>(segmentPrunerTypes.size()); for (String segmentPrunerType : segmentPrunerTypes) { if (RoutingConfig.PARTITION_SEGMENT_PRUNER_TYPE.equalsIgnoreCase(segmentPrunerType)) { - SegmentPruner partitionSegmentPruner = getPartitionSegmentPruner(tableConfig, propertyStore); + SegmentPruner partitionSegmentPruner = getPartitionSegmentPruner(tableConfig); if (partitionSegmentPruner != null) { configuredSegmentPruners.add(partitionSegmentPruner); } @@ -91,7 +88,7 @@ public static List getSegmentPruners(TableConfig tableConfig, if ((tableType == TableType.OFFLINE && LEGACY_PARTITION_AWARE_OFFLINE_ROUTING.equalsIgnoreCase( routingTableBuilderName)) || (tableType == TableType.REALTIME && LEGACY_PARTITION_AWARE_REALTIME_ROUTING.equalsIgnoreCase(routingTableBuilderName))) { - SegmentPruner partitionSegmentPruner = getPartitionSegmentPruner(tableConfig, propertyStore); + SegmentPruner partitionSegmentPruner = getPartitionSegmentPruner(tableConfig); if (partitionSegmentPruner != null) { segmentPruners.add(partitionSegmentPruner); } @@ -102,8 +99,7 @@ public static List getSegmentPruners(TableConfig tableConfig, } @Nullable - private static SegmentPruner getPartitionSegmentPruner(TableConfig tableConfig, - ZkHelixPropertyStore propertyStore) { + private static SegmentPruner getPartitionSegmentPruner(TableConfig tableConfig) { String tableNameWithType = tableConfig.getTableName(); SegmentPartitionConfig segmentPartitionConfig = tableConfig.getIndexingConfig().getSegmentPartitionConfig(); if (segmentPartitionConfig == null) { @@ -137,26 +133,20 @@ private static TimeSegmentPruner getTimeSegmentPruner(TableConfig tableConfig, LOGGER.warn("Cannot enable time range pruning without time column for table: {}", tableNameWithType); return null; } - return createTimeSegmentPruner(tableConfig, propertyStore); - } - - @VisibleForTesting - static TimeSegmentPruner createTimeSegmentPruner(TableConfig tableConfig, - ZkHelixPropertyStore propertyStore) { - String tableNameWithType = tableConfig.getTableName(); - String timeColumn = tableConfig.getValidationConfig().getTimeColumnName(); - Preconditions.checkNotNull(timeColumn, "Time column must be configured in table config for table: %s", - tableNameWithType); - Schema schema = ZKMetadataProvider.getTableSchema(propertyStore, tableNameWithType); - Preconditions.checkNotNull(schema, "Failed to find schema for table: %s", tableNameWithType); - DateTimeFieldSpec dateTimeSpec = schema.getSpecForTimeColumn(timeColumn); - Preconditions.checkNotNull(dateTimeSpec, "Field spec must be specified in schema for time column: %s of table: %s", - timeColumn, tableNameWithType); - DateTimeFormatSpec timeFormatSpec = dateTimeSpec.getFormatSpec(); - - LOGGER.info("Using TimeRangePruner on time column: {} for table: {} with DateTimeFormatSpec: {}", - timeColumn, tableNameWithType, timeFormatSpec); - return new TimeSegmentPruner(tableConfig, timeColumn, timeFormatSpec); + Schema schema = ZKMetadataProvider.getTableSchema(propertyStore, tableConfig); + if (schema == null) { + LOGGER.warn("Cannot enable time range pruning without schema for table: {}", tableNameWithType); + return null; + } + DateTimeFieldSpec timeFieldSpec = schema.getSpecForTimeColumn(timeColumn); + if (timeFieldSpec == null) { + LOGGER.warn("Cannot enable time range pruning without field spec for table: {}, time column: {}", + tableNameWithType, timeColumn); + return null; + } + LOGGER.info("Using TimeRangePruner on time column: {} for table: {} with DateTimeFieldSpec: {}", timeColumn, + tableNameWithType, timeFieldSpec); + return new TimeSegmentPruner(tableConfig, timeFieldSpec); } private static List sortSegmentPruners(List pruners) { diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/TimeSegmentPruner.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/TimeSegmentPruner.java index a7ac4fce4bdf..c2e6b20cce54 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/TimeSegmentPruner.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/TimeSegmentPruner.java @@ -18,6 +18,7 @@ */ package org.apache.pinot.broker.routing.segmentpruner; +import com.google.common.base.Preconditions; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -37,7 +38,9 @@ import org.apache.pinot.common.request.Expression; import org.apache.pinot.common.request.Function; import org.apache.pinot.common.request.Identifier; +import org.apache.pinot.common.request.Literal; import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.DateTimeFieldSpec; import org.apache.pinot.spi.data.DateTimeFormatSpec; import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.CommonConstants.Query.Range; @@ -64,10 +67,10 @@ public class TimeSegmentPruner implements SegmentPruner { private volatile IntervalTree _intervalTree; private final Map _intervalMap = new HashMap<>(); - public TimeSegmentPruner(TableConfig tableConfig, String timeColumn, DateTimeFormatSpec timeFormatSpec) { + public TimeSegmentPruner(TableConfig tableConfig, DateTimeFieldSpec timeFieldSpec) { _tableNameWithType = tableConfig.getTableName(); - _timeColumn = timeColumn; - _timeFormatSpec = timeFormatSpec; + _timeColumn = timeFieldSpec.getName(); + _timeFormatSpec = timeFieldSpec.getFormatSpec(); } @Override @@ -206,97 +209,53 @@ private List getFilterTimeIntervals(Expression filterExpression) { } else { return getComplementSortedIntervals(childIntervals); } - case EQUALS: { - Identifier identifier = operands.get(0).getIdentifier(); - if (identifier != null && identifier.getName().equals(_timeColumn)) { - long timeStamp = _timeFormatSpec.fromFormatToMillis(operands.get(1).getLiteral().getFieldValue().toString()); - return Collections.singletonList(new Interval(timeStamp, timeStamp)); - } else { - return null; + case EQUALS: + if (isTimeColumn(operands.get(0))) { + long timestamp = toMillisSinceEpoch(operands.get(1)); + return List.of(new Interval(timestamp, timestamp)); } - } - case IN: { - Identifier identifier = operands.get(0).getIdentifier(); - if (identifier != null && identifier.getName().equals(_timeColumn)) { + return null; + case IN: + if (isTimeColumn(operands.get(0))) { int numOperands = operands.size(); List intervals = new ArrayList<>(numOperands - 1); for (int i = 1; i < numOperands; i++) { - long timeStamp = - _timeFormatSpec.fromFormatToMillis(operands.get(i).getLiteral().getFieldValue().toString()); - intervals.add(new Interval(timeStamp, timeStamp)); + long timestamp = toMillisSinceEpoch(operands.get(i)); + intervals.add(new Interval(timestamp, timestamp)); } return intervals; - } else { - return null; } - } - case GREATER_THAN: { - Identifier identifier = operands.get(0).getIdentifier(); - if (identifier != null && identifier.getName().equals(_timeColumn)) { - long timeStamp = _timeFormatSpec.fromFormatToMillis(operands.get(1).getLiteral().getFieldValue().toString()); - return Collections.singletonList(new Interval(timeStamp + 1, MAX_END_TIME)); - } else { - return null; + return null; + case GREATER_THAN: + if (isTimeColumn(operands.get(0))) { + return getInterval(toMillisSinceEpoch(operands.get(1)) + 1, MAX_END_TIME); } - } - case GREATER_THAN_OR_EQUAL: { - Identifier identifier = operands.get(0).getIdentifier(); - if (identifier != null && identifier.getName().equals(_timeColumn)) { - long timeStamp = _timeFormatSpec.fromFormatToMillis(operands.get(1).getLiteral().getFieldValue().toString()); - return Collections.singletonList(new Interval(timeStamp, MAX_END_TIME)); - } else { - return null; + return null; + case GREATER_THAN_OR_EQUAL: + if (isTimeColumn(operands.get(0))) { + return getInterval(toMillisSinceEpoch(operands.get(1)), MAX_END_TIME); } - } - case LESS_THAN: { - Identifier identifier = operands.get(0).getIdentifier(); - if (identifier != null && identifier.getName().equals(_timeColumn)) { - long timeStamp = _timeFormatSpec.fromFormatToMillis(operands.get(1).getLiteral().getFieldValue().toString()); - if (timeStamp > MIN_START_TIME) { - return Collections.singletonList(new Interval(MIN_START_TIME, timeStamp - 1)); - } else { - return Collections.emptyList(); - } - } else { - return null; + return null; + case LESS_THAN: + if (isTimeColumn(operands.get(0))) { + return getInterval(MIN_START_TIME, toMillisSinceEpoch(operands.get(1)) - 1); } - } - case LESS_THAN_OR_EQUAL: { - Identifier identifier = operands.get(0).getIdentifier(); - if (identifier != null && identifier.getName().equals(_timeColumn)) { - long timeStamp = _timeFormatSpec.fromFormatToMillis(operands.get(1).getLiteral().getFieldValue().toString()); - if (timeStamp >= MIN_START_TIME) { - return Collections.singletonList(new Interval(MIN_START_TIME, timeStamp)); - } else { - return Collections.emptyList(); - } - } else { - return null; + return null; + case LESS_THAN_OR_EQUAL: + if (isTimeColumn(operands.get(0))) { + return getInterval(MIN_START_TIME, toMillisSinceEpoch(operands.get(1))); } - } - case BETWEEN: { - Identifier identifier = operands.get(0).getIdentifier(); - if (identifier != null && identifier.getName().equals(_timeColumn)) { - long startTimestamp = - _timeFormatSpec.fromFormatToMillis(operands.get(1).getLiteral().getFieldValue().toString()); - long endTimestamp = - _timeFormatSpec.fromFormatToMillis(operands.get(2).getLiteral().getFieldValue().toString()); - if (endTimestamp >= startTimestamp) { - return Collections.singletonList(new Interval(startTimestamp, endTimestamp)); - } else { - return Collections.emptyList(); - } - } else { - return null; + return null; + case BETWEEN: + if (isTimeColumn(operands.get(0))) { + return getInterval(toMillisSinceEpoch(operands.get(1)), toMillisSinceEpoch(operands.get(2))); } - } - case RANGE: { - Identifier identifier = operands.get(0).getIdentifier(); - if (identifier != null && identifier.getName().equals(_timeColumn)) { + return null; + case RANGE: + if (isTimeColumn(operands.get(0))) { return parseInterval(operands.get(1).getLiteral().getFieldValue().toString()); } return null; - } default: return null; } @@ -408,6 +367,17 @@ private List getComplementSortedIntervals(List intervals) { return res; } + private boolean isTimeColumn(Expression expression) { + Identifier identifier = expression.getIdentifier(); + return identifier != null && identifier.getName().equals(_timeColumn); + } + + private long toMillisSinceEpoch(Expression expression) { + Literal literal = expression.getLiteral(); + Preconditions.checkArgument(literal != null, "Literal is required for time column filter, got: %s", expression); + return _timeFormatSpec.fromFormatToMillis(literal.getFieldValue().toString()); + } + /** * Parse interval to millisecond as [min, max] with both sides included. * E.g. '(* 16311]' is parsed as [0, 16311], '(1455 16311)' is parsed as [1456, 16310] @@ -432,10 +402,10 @@ private List parseInterval(String rangeString) { endTime--; } } + return getInterval(startTime, endTime); + } - if (startTime > endTime) { - return Collections.emptyList(); - } - return Collections.singletonList(new Interval(startTime, endTime)); + private static List getInterval(long inclusiveStart, long inclusiveEnd) { + return inclusiveStart <= inclusiveEnd ? List.of(new Interval(inclusiveStart, inclusiveEnd)) : List.of(); } } diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerTest.java index feaad35169ba..5e48a981ccc4 100644 --- a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerTest.java +++ b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerTest.java @@ -18,8 +18,7 @@ */ package org.apache.pinot.broker.routing.segmentpruner; -import java.util.Arrays; -import java.util.Collections; +import java.sql.Timestamp; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -50,11 +49,11 @@ import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; +import org.apache.pinot.spi.data.DateTimeFieldSpec; import org.apache.pinot.spi.data.DateTimeFormatSpec; -import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.FieldSpec.DataType; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.stream.StreamConfigProperties; -import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.builder.TableConfigBuilder; import org.apache.pinot.sql.parsers.CalciteSqlCompiler; import org.mockito.Mockito; @@ -78,29 +77,45 @@ public class SegmentPrunerTest extends ControllerTest { private static final String SDF_PATTERN = "yyyyMMdd"; private static final String QUERY_1 = "SELECT * FROM testTable"; - private static final String QUERY_2 = "SELECT * FROM testTable where memberId = 0"; - private static final String QUERY_3 = "SELECT * FROM testTable where memberId IN (1, 2)"; - private static final String QUERY_4 = "SELECT * FROM testTable where memberId = 0 AND memberName='xyz'"; - - private static final String TIME_QUERY_1 = "SELECT * FROM testTable where timeColumn = 40"; - private static final String TIME_QUERY_2 = "SELECT * FROM testTable where timeColumn BETWEEN 20 AND 30"; - private static final String TIME_QUERY_3 = "SELECT * FROM testTable where 30 < timeColumn AND timeColumn <= 50"; - private static final String TIME_QUERY_4 = "SELECT * FROM testTable where timeColumn < 15 OR timeColumn > 45"; + private static final String QUERY_2 = "SELECT * FROM testTable WHERE memberId = 0"; + private static final String QUERY_3 = "SELECT * FROM testTable WHERE memberId IN (1, 2)"; + private static final String QUERY_4 = "SELECT * FROM testTable WHERE memberId = 0 AND memberName = 'xyz'"; + + private static final String TIME_QUERY_1 = "SELECT * FROM testTable WHERE timeColumn = 40"; + private static final String TIME_QUERY_2 = "SELECT * FROM testTable WHERE timeColumn BETWEEN 20 AND 30"; + private static final String TIME_QUERY_3 = "SELECT * FROM testTable WHERE 30 < timeColumn AND timeColumn <= 50"; + private static final String TIME_QUERY_4 = "SELECT * FROM testTable WHERE timeColumn < 15 OR timeColumn > 45"; private static final String TIME_QUERY_5 = - "SELECT * FROM testTable where timeColumn < 15 OR (60 < timeColumn AND timeColumn < 70)"; - private static final String TIME_QUERY_6 = "SELECT * FROM testTable where timeColumn < 0 AND timeColumn > 0"; + "SELECT * FROM testTable WHERE timeColumn < 15 OR (60 < timeColumn AND timeColumn < 70)"; + private static final String TIME_QUERY_6 = "SELECT * FROM testTable WHERE timeColumn NOT BETWEEN 20 AND 30"; + private static final String TIME_QUERY_7 = "SELECT * FROM testTable WHERE NOT timeColumn > 30"; + private static final String TIME_QUERY_8 = "SELECT * FROM testTable WHERE timeColumn < 0 AND timeColumn > 0"; - private static final String SDF_QUERY_1 = "SELECT * FROM testTable where timeColumn = 20200131"; - private static final String SDF_QUERY_2 = "SELECT * FROM testTable where timeColumn BETWEEN 20200101 AND 20200331"; + private static final String SDF_QUERY_1 = "SELECT * FROM testTable WHERE timeColumn = 20200131"; + private static final String SDF_QUERY_2 = "SELECT * FROM testTable WHERE timeColumn BETWEEN 20200101 AND 20200331"; private static final String SDF_QUERY_3 = - "SELECT * FROM testTable where 20200430 < timeColumn AND timeColumn < 20200630"; + "SELECT * FROM testTable WHERE 20200430 < timeColumn AND timeColumn < 20200630"; private static final String SDF_QUERY_4 = - "SELECT * FROM testTable where timeColumn <= 20200101 OR timeColumn in (20200201, 20200401)"; + "SELECT * FROM testTable WHERE timeColumn <= 20200101 OR timeColumn IN (20200201, 20200401)"; private static final String SDF_QUERY_5 = - "SELECT * FROM testTable where timeColumn in (20200101, 20200102) AND timeColumn >= 20200530"; - - private static final String SQL_TIME_QUERY_1 = "SELECT * FROM testTable WHERE timeColumn NOT BETWEEN 20 AND 30"; - private static final String SQL_TIME_QUERY_2 = "SELECT * FROM testTable WHERE NOT timeColumn > 30"; + "SELECT * FROM testTable WHERE timeColumn IN (20200101, 20200102) AND timeColumn >= 20200530"; + + // Timestamp can be passed as string or long + private static final String TIMESTAMP_QUERY_1 = "SELECT * FROM testTable WHERE timeColumn = '2020-01-31 00:00:00'"; + private static final String TIMESTAMP_QUERY_2 = String.format("SELECT * FROM testTable WHERE timeColumn = %d", + Timestamp.valueOf("2020-01-31 00:00:00").getTime()); + private static final String TIMESTAMP_QUERY_3 = + "SELECT * FROM testTable WHERE timeColumn BETWEEN '2020-01-01 00:00:00' AND '2020-03-31 00:00:00'"; + private static final String TIMESTAMP_QUERY_4 = + String.format("SELECT * FROM testTable WHERE timeColumn BETWEEN %d AND %d", + Timestamp.valueOf("2020-01-01 00:00:00").getTime(), Timestamp.valueOf("2020-03-31 00:00:00").getTime()); + private static final String TIMESTAMP_QUERY_5 = + "SELECT * FROM testTable WHERE timeColumn <= '2020-01-01 00:00:00' OR timeColumn IN ('2020-02-01 00:00:00', " + + "'2020-04-01 00:00:00')"; + private static final String TIMESTAMP_QUERY_6 = + String.format("SELECT * FROM testTable WHERE timeColumn <= %d OR timeColumn IN (%d, %d)", + Timestamp.valueOf("2020-01-01 00:00:00").getTime(), Timestamp.valueOf("2020-02-01 00:00:00").getTime(), + Timestamp.valueOf("2020-04-01 00:00:00").getTime()); // this is duplicate with KinesisConfig.STREAM_TYPE, while instead of use KinesisConfig.STREAM_TYPE directly, we // hardcode the value here to avoid pulling the entire pinot-kinesis module as dependency. @@ -127,6 +142,7 @@ public void tearDown() { @Test public void testSegmentPrunerFactoryForPartitionPruner() { TableConfig tableConfig = mock(TableConfig.class); + when(tableConfig.getTableName()).thenReturn(OFFLINE_TABLE_NAME); IndexingConfig indexingConfig = mock(IndexingConfig.class); when(tableConfig.getIndexingConfig()).thenReturn(indexingConfig); @@ -141,8 +157,7 @@ public void testSegmentPrunerFactoryForPartitionPruner() { assertEquals(segmentPruners.size(), 0); // Segment partition config is missing - when(routingConfig.getSegmentPrunerTypes()).thenReturn( - Collections.singletonList(RoutingConfig.PARTITION_SEGMENT_PRUNER_TYPE)); + when(routingConfig.getSegmentPrunerTypes()).thenReturn(List.of(RoutingConfig.PARTITION_SEGMENT_PRUNER_TYPE)); segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); assertEquals(segmentPruners.size(), 0); @@ -189,8 +204,7 @@ public void testSegmentPrunerFactoryForPartitionPruner() { @Test public void testSegmentPrunerFactoryForTimeRangePruner() { TableConfig tableConfig = mock(TableConfig.class); - when(tableConfig.getTableName()).thenReturn(RAW_TABLE_NAME); - setSchemaDateTimeFieldSpec(RAW_TABLE_NAME, TimeUnit.HOURS); + when(tableConfig.getTableName()).thenReturn(OFFLINE_TABLE_NAME); // Routing config is missing List segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); @@ -203,8 +217,7 @@ public void testSegmentPrunerFactoryForTimeRangePruner() { assertEquals(segmentPruners.size(), 0); // Validation config is missing - when(routingConfig.getSegmentPrunerTypes()).thenReturn( - Collections.singletonList(RoutingConfig.TIME_SEGMENT_PRUNER_TYPE)); + when(routingConfig.getSegmentPrunerTypes()).thenReturn(List.of(RoutingConfig.TIME_SEGMENT_PRUNER_TYPE)); segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); assertEquals(segmentPruners.size(), 0); @@ -214,41 +227,54 @@ public void testSegmentPrunerFactoryForTimeRangePruner() { segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); assertEquals(segmentPruners.size(), 0); - // Time range pruner should be returned + // Schema is missing when(validationConfig.getTimeColumnName()).thenReturn(TIME_COLUMN); segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); + assertEquals(segmentPruners.size(), 0); + + // Field spec is missing + Schema schema = new Schema.SchemaBuilder().setSchemaName(RAW_TABLE_NAME).build(); + ZKMetadataProvider.setSchema(_propertyStore, schema); + segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); + assertEquals(segmentPruners.size(), 0); + + // Time range pruner should be returned + schema = new Schema.SchemaBuilder().setSchemaName(RAW_TABLE_NAME) + .addDateTimeField(TIME_COLUMN, DataType.TIMESTAMP, "TIMESTAMP", "1:MILLISECONDS").build(); + ZKMetadataProvider.setSchema(_propertyStore, schema); + segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); assertEquals(segmentPruners.size(), 1); assertTrue(segmentPruners.get(0) instanceof TimeSegmentPruner); } @Test - public void testEnablingEmptySegmentPruner() { + public void testSegmentPrunerFactoryForEmptySegmentPruner() { TableConfig tableConfig = mock(TableConfig.class); + when(tableConfig.getTableName()).thenReturn(REALTIME_TABLE_NAME); IndexingConfig indexingConfig = mock(IndexingConfig.class); + when(tableConfig.getIndexingConfig()).thenReturn(indexingConfig); RoutingConfig routingConfig = mock(RoutingConfig.class); - StreamIngestionConfig streamIngestionConfig = mock(StreamIngestionConfig.class); + when(tableConfig.getRoutingConfig()).thenReturn(routingConfig); + ZkHelixPropertyStore propertyStore = mock(ZkHelixPropertyStore.class); // When routingConfig is configured with EmptySegmentPruner, EmptySegmentPruner should be returned. - when(tableConfig.getRoutingConfig()).thenReturn(routingConfig); - when(routingConfig.getSegmentPrunerTypes()).thenReturn( - Collections.singletonList(RoutingConfig.EMPTY_SEGMENT_PRUNER_TYPE)); - List segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); + when(routingConfig.getSegmentPrunerTypes()).thenReturn(List.of(RoutingConfig.EMPTY_SEGMENT_PRUNER_TYPE)); + List segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, propertyStore); assertEquals(segmentPruners.size(), 1); assertTrue(segmentPruners.get(0) instanceof EmptySegmentPruner); // When indexingConfig is configured with Kinesis streaming, EmptySegmentPruner should be returned. - when(indexingConfig.getStreamConfigs()).thenReturn( - Collections.singletonMap(StreamConfigProperties.STREAM_TYPE, KINESIS_STREAM_TYPE)); - segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); + when(indexingConfig.getStreamConfigs()).thenReturn(Map.of(StreamConfigProperties.STREAM_TYPE, KINESIS_STREAM_TYPE)); + segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, propertyStore); assertEquals(segmentPruners.size(), 1); assertTrue(segmentPruners.get(0) instanceof EmptySegmentPruner); // When streamIngestionConfig is configured with Kinesis streaming, EmptySegmentPruner should be returned. + StreamIngestionConfig streamIngestionConfig = mock(StreamIngestionConfig.class); when(streamIngestionConfig.getStreamConfigMaps()).thenReturn( - Collections.singletonList(Collections.singletonMap(StreamConfigProperties.STREAM_TYPE, KINESIS_STREAM_TYPE))); - when(indexingConfig.getStreamConfigs()).thenReturn( - Collections.singletonMap(StreamConfigProperties.STREAM_TYPE, KINESIS_STREAM_TYPE)); - segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, _propertyStore); + List.of(Map.of(StreamConfigProperties.STREAM_TYPE, KINESIS_STREAM_TYPE))); + when(indexingConfig.getStreamConfigs()).thenReturn(Map.of(StreamConfigProperties.STREAM_TYPE, KINESIS_STREAM_TYPE)); + segmentPruners = SegmentPrunerFactory.getSegmentPruners(tableConfig, propertyStore); assertEquals(segmentPruners.size(), 1); assertTrue(segmentPruners.get(0) instanceof EmptySegmentPruner); } @@ -259,95 +285,76 @@ public void testPartitionAwareSegmentPruner() { BrokerRequest brokerRequest2 = CalciteSqlCompiler.compileToBrokerRequest(QUERY_2); BrokerRequest brokerRequest3 = CalciteSqlCompiler.compileToBrokerRequest(QUERY_3); BrokerRequest brokerRequest4 = CalciteSqlCompiler.compileToBrokerRequest(QUERY_4); + // NOTE: Ideal state and external view are not used in the current implementation IdealState idealState = Mockito.mock(IdealState.class); ExternalView externalView = Mockito.mock(ExternalView.class); SinglePartitionColumnSegmentPruner singlePartitionColumnSegmentPruner = new SinglePartitionColumnSegmentPruner(OFFLINE_TABLE_NAME, PARTITION_COLUMN_1); - SegmentZkMetadataFetcher segmentZkMetadataFetcher = new SegmentZkMetadataFetcher(OFFLINE_TABLE_NAME, - _propertyStore); + SegmentZkMetadataFetcher segmentZkMetadataFetcher = + new SegmentZkMetadataFetcher(OFFLINE_TABLE_NAME, _propertyStore); segmentZkMetadataFetcher.register(singlePartitionColumnSegmentPruner); Set onlineSegments = new HashSet<>(); segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); - assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest1, Collections.emptySet()), - Collections.emptySet()); - assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest2, Collections.emptySet()), - Collections.emptySet()); - assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest3, Collections.emptySet()), - Collections.emptySet()); + + Set input = Set.of(); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest1, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest2, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest3, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest4, input), input); // Segments without metadata (not updated yet) should not be pruned String newSegment = "newSegment"; - assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest1, Collections.singleton(newSegment)), - Collections.singletonList(newSegment)); - assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest2, Collections.singleton(newSegment)), - Collections.singletonList(newSegment)); - assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest3, Collections.singleton(newSegment)), - Collections.singletonList(newSegment)); + onlineSegments.add(newSegment); + segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); + input = Set.of(newSegment); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest1, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest2, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest3, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest4, input), input); // Segments without partition metadata should not be pruned String segmentWithoutPartitionMetadata = "segmentWithoutPartitionMetadata"; - onlineSegments.add(segmentWithoutPartitionMetadata); - SegmentZKMetadata segmentZKMetadataWithoutPartitionMetadata = - new SegmentZKMetadata(segmentWithoutPartitionMetadata); ZKMetadataProvider.setSegmentZKMetadata(_propertyStore, OFFLINE_TABLE_NAME, - segmentZKMetadataWithoutPartitionMetadata); + new SegmentZKMetadata(segmentWithoutPartitionMetadata)); + onlineSegments.add(segmentWithoutPartitionMetadata); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest1, - new HashSet<>(Collections.singletonList(segmentWithoutPartitionMetadata))), - Collections.singletonList(segmentWithoutPartitionMetadata)); - assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest2, - new HashSet<>(Collections.singletonList(segmentWithoutPartitionMetadata))), - Collections.singletonList(segmentWithoutPartitionMetadata)); - assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest3, - new HashSet<>(Collections.singletonList(segmentWithoutPartitionMetadata))), - Collections.singletonList(segmentWithoutPartitionMetadata)); + input = Set.of(segmentWithoutPartitionMetadata); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest1, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest2, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest3, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest4, input), input); // Test different partition functions and number of partitions // 0 % 5 = 0; 1 % 5 = 1; 2 % 5 = 2 String segment0 = "segment0"; - onlineSegments.add(segment0); setSegmentZKPartitionMetadata(OFFLINE_TABLE_NAME, segment0, "Modulo", 5, 0); + onlineSegments.add(segment0); // Murmur(0) % 4 = 0; Murmur(1) % 4 = 3; Murmur(2) % 4 = 0 String segment1 = "segment1"; - onlineSegments.add(segment1); setSegmentZKPartitionMetadata(OFFLINE_TABLE_NAME, segment1, "Murmur", 4, 0); + onlineSegments.add(segment1); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals( - singlePartitionColumnSegmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals( - singlePartitionColumnSegmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals( - singlePartitionColumnSegmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Collections.singletonList(segment1))); + input = Set.of(segment0, segment1); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest1, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest2, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest3, input), Set.of(segment1)); // Update partition metadata without refreshing should have no effect setSegmentZKPartitionMetadata(OFFLINE_TABLE_NAME, segment0, "Modulo", 4, 1); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals( - singlePartitionColumnSegmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals( - singlePartitionColumnSegmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals( - singlePartitionColumnSegmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Collections.singletonList(segment1))); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest1, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest2, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest3, input), Set.of(segment1)); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest4, input), input); // Refresh the changed segment should update the segment pruner segmentZkMetadataFetcher.refreshSegment(segment0); - assertEquals( - singlePartitionColumnSegmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals( - singlePartitionColumnSegmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Collections.singletonList(segment1))); - assertEquals( - singlePartitionColumnSegmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Arrays.asList(segment0, segment1))); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest1, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest2, input), Set.of(segment1)); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest3, input), input); + assertEquals(singlePartitionColumnSegmentPruner.prune(brokerRequest4, input), Set.of(segment1)); // Multi-column partitioned segment. MultiPartitionColumnsSegmentPruner multiPartitionColumnsSegmentPruner = @@ -356,38 +363,25 @@ public void testPartitionAwareSegmentPruner() { segmentZkMetadataFetcher = new SegmentZkMetadataFetcher(OFFLINE_TABLE_NAME, _propertyStore); segmentZkMetadataFetcher.register(multiPartitionColumnsSegmentPruner); segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); - assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest1, Collections.emptySet()), - Collections.emptySet()); - assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest2, Collections.emptySet()), - Collections.emptySet()); - assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest3, Collections.emptySet()), - Collections.emptySet()); - assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest4, Collections.emptySet()), - Collections.emptySet()); + + assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest1, input), input); + assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest2, input), Set.of(segment1)); + assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest3, input), input); + assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest4, input), Set.of(segment1)); String segment2 = "segment2"; - onlineSegments.add(segment2); Map columnPartitionMetadataMap = new HashMap<>(); - columnPartitionMetadataMap.put(PARTITION_COLUMN_1, - new ColumnPartitionMetadata("Modulo", 4, Collections.singleton(0), null)); - Map partitionColumn2FunctionConfig = new HashMap<>(); - partitionColumn2FunctionConfig.put("columnValues", "xyz|abc"); - partitionColumn2FunctionConfig.put("columnValuesDelimiter", "|"); - columnPartitionMetadataMap.put(PARTITION_COLUMN_2, new ColumnPartitionMetadata( - "BoundedColumnValue", 3, Collections.singleton(1), partitionColumn2FunctionConfig)); + columnPartitionMetadataMap.put(PARTITION_COLUMN_1, new ColumnPartitionMetadata("Modulo", 4, Set.of(0), null)); + columnPartitionMetadataMap.put(PARTITION_COLUMN_2, new ColumnPartitionMetadata("BoundedColumnValue", 3, Set.of(1), + Map.of("columnValues", "xyz|abc", "columnValuesDelimiter", "|"))); setSegmentZKPartitionMetadata(OFFLINE_TABLE_NAME, segment2, columnPartitionMetadataMap); + onlineSegments.add(segment2); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals( - multiPartitionColumnsSegmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals( - multiPartitionColumnsSegmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Collections.singletonList(segment1))); - assertEquals( - multiPartitionColumnsSegmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest4, - new HashSet<>(Arrays.asList(segment0, segment1, segment2))), new HashSet<>(Arrays.asList(segment1, segment2))); + input = Set.of(segment0, segment1, segment2); + assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest1, input), input); + assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest2, input), Set.of(segment1, segment2)); + assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest3, input), Set.of(segment0, segment1)); + assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest4, input), Set.of(segment1, segment2)); } @Test @@ -399,143 +393,112 @@ public void testTimeSegmentPruner() { BrokerRequest brokerRequest5 = CalciteSqlCompiler.compileToBrokerRequest(TIME_QUERY_4); BrokerRequest brokerRequest6 = CalciteSqlCompiler.compileToBrokerRequest(TIME_QUERY_5); BrokerRequest brokerRequest7 = CalciteSqlCompiler.compileToBrokerRequest(TIME_QUERY_6); + BrokerRequest brokerRequest8 = CalciteSqlCompiler.compileToBrokerRequest(TIME_QUERY_7); + BrokerRequest brokerRequest9 = CalciteSqlCompiler.compileToBrokerRequest(TIME_QUERY_8); + // NOTE: Ideal state and external view are not used in the current implementation IdealState idealState = Mockito.mock(IdealState.class); ExternalView externalView = Mockito.mock(ExternalView.class); - TableConfig tableConfig = getTableConfig(RAW_TABLE_NAME, TableType.REALTIME); - setSchemaDateTimeFieldSpec(RAW_TABLE_NAME, TimeUnit.DAYS); - TimeSegmentPruner segmentPruner = SegmentPrunerFactory.createTimeSegmentPruner(tableConfig, - _propertyStore); - Set onlineSegments = new HashSet<>(); - SegmentZkMetadataFetcher segmentZkMetadataFetcher = new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, - _propertyStore); + TableConfig tableConfig = + new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setTimeColumnName(TIME_COLUMN).build(); + DateTimeFieldSpec timeFieldSpec = new DateTimeFieldSpec(TIME_COLUMN, DataType.INT, "EPOCH|DAYS", "1:DAYS"); + TimeSegmentPruner segmentPruner = new TimeSegmentPruner(tableConfig, timeFieldSpec); + SegmentZkMetadataFetcher segmentZkMetadataFetcher = + new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, _propertyStore); segmentZkMetadataFetcher.register(segmentPruner); + Set onlineSegments = new HashSet<>(); segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, Collections.emptySet()), Collections.emptySet()); - assertEquals(segmentPruner.prune(brokerRequest2, Collections.emptySet()), Collections.emptySet()); - assertEquals(segmentPruner.prune(brokerRequest3, Collections.emptySet()), Collections.emptySet()); - assertEquals(segmentPruner.prune(brokerRequest4, Collections.emptySet()), Collections.emptySet()); - assertEquals(segmentPruner.prune(brokerRequest5, Collections.emptySet()), Collections.emptySet()); - assertEquals(segmentPruner.prune(brokerRequest6, Collections.emptySet()), Collections.emptySet()); - assertEquals(segmentPruner.prune(brokerRequest7, Collections.emptySet()), Collections.emptySet()); - - // Initialize with non-empty onlineSegments + + Set input = Set.of(); + assertEquals(segmentPruner.prune(brokerRequest1, input), input); + assertEquals(segmentPruner.prune(brokerRequest2, input), input); + assertEquals(segmentPruner.prune(brokerRequest3, input), input); + assertEquals(segmentPruner.prune(brokerRequest4, input), input); + assertEquals(segmentPruner.prune(brokerRequest5, input), input); + assertEquals(segmentPruner.prune(brokerRequest6, input), input); + assertEquals(segmentPruner.prune(brokerRequest7, input), input); + assertEquals(segmentPruner.prune(brokerRequest8, input), input); + assertEquals(segmentPruner.prune(brokerRequest9, input), input); + // Segments without metadata (not updated yet) should not be pruned - segmentPruner = SegmentPrunerFactory.createTimeSegmentPruner(tableConfig, _propertyStore); - segmentZkMetadataFetcher = new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, _propertyStore); - segmentZkMetadataFetcher.register(segmentPruner); String newSegment = "newSegment"; onlineSegments.add(newSegment); - segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, Collections.singleton(newSegment)), - Collections.singletonList(newSegment)); - assertEquals(segmentPruner.prune(brokerRequest2, Collections.singleton(newSegment)), - Collections.singletonList(newSegment)); - assertEquals(segmentPruner.prune(brokerRequest3, Collections.singleton(newSegment)), - Collections.singletonList(newSegment)); - assertEquals(segmentPruner.prune(brokerRequest4, Collections.singleton(newSegment)), - Collections.singletonList(newSegment)); - assertEquals(segmentPruner.prune(brokerRequest5, Collections.singleton(newSegment)), - Collections.singletonList(newSegment)); - assertEquals(segmentPruner.prune(brokerRequest6, Collections.singleton(newSegment)), - Collections.singletonList(newSegment)); - assertEquals(segmentPruner.prune(brokerRequest7, Collections.singleton(newSegment)), - Collections.emptySet()); // query with invalid range will always have empty filtered result + segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); + input = Set.of(newSegment); + assertEquals(segmentPruner.prune(brokerRequest1, input), input); + assertEquals(segmentPruner.prune(brokerRequest2, input), input); + assertEquals(segmentPruner.prune(brokerRequest3, input), input); + assertEquals(segmentPruner.prune(brokerRequest4, input), input); + assertEquals(segmentPruner.prune(brokerRequest5, input), input); + assertEquals(segmentPruner.prune(brokerRequest6, input), input); + assertEquals(segmentPruner.prune(brokerRequest7, input), input); + assertEquals(segmentPruner.prune(brokerRequest8, input), input); + assertEquals(segmentPruner.prune(brokerRequest9, input), Set.of()); // Query with invalid range // Segments without time range metadata should not be pruned String segmentWithoutTimeRangeMetadata = "segmentWithoutTimeRangeMetadata"; - onlineSegments.add(segmentWithoutTimeRangeMetadata); SegmentZKMetadata segmentZKMetadataWithoutTimeRangeMetadata = new SegmentZKMetadata(segmentWithoutTimeRangeMetadata); - segmentZKMetadataWithoutTimeRangeMetadata.setStatus(CommonConstants.Segment.Realtime.Status.DONE); ZKMetadataProvider.setSegmentZKMetadata(_propertyStore, REALTIME_TABLE_NAME, segmentZKMetadataWithoutTimeRangeMetadata); + onlineSegments.add(segmentWithoutTimeRangeMetadata); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals( - segmentPruner.prune(brokerRequest1, new HashSet<>(Collections.singletonList(segmentWithoutTimeRangeMetadata))), - Collections.singletonList(segmentWithoutTimeRangeMetadata)); - assertEquals( - segmentPruner.prune(brokerRequest2, new HashSet<>(Collections.singletonList(segmentWithoutTimeRangeMetadata))), - Collections.singletonList(segmentWithoutTimeRangeMetadata)); - assertEquals( - segmentPruner.prune(brokerRequest3, new HashSet<>(Collections.singletonList(segmentWithoutTimeRangeMetadata))), - Collections.singletonList(segmentWithoutTimeRangeMetadata)); - assertEquals( - segmentPruner.prune(brokerRequest4, new HashSet<>(Collections.singletonList(segmentWithoutTimeRangeMetadata))), - Collections.singletonList(segmentWithoutTimeRangeMetadata)); - assertEquals( - segmentPruner.prune(brokerRequest5, new HashSet<>(Collections.singletonList(segmentWithoutTimeRangeMetadata))), - Collections.singletonList(segmentWithoutTimeRangeMetadata)); - assertEquals( - segmentPruner.prune(brokerRequest6, new HashSet<>(Collections.singletonList(segmentWithoutTimeRangeMetadata))), - Collections.singletonList(segmentWithoutTimeRangeMetadata)); - assertEquals( - segmentPruner.prune(brokerRequest7, new HashSet<>(Collections.singletonList(segmentWithoutTimeRangeMetadata))), - Collections.emptySet()); + assertEquals(segmentPruner.prune(brokerRequest1, input), input); + assertEquals(segmentPruner.prune(brokerRequest2, input), input); + assertEquals(segmentPruner.prune(brokerRequest3, input), input); + assertEquals(segmentPruner.prune(brokerRequest4, input), input); + assertEquals(segmentPruner.prune(brokerRequest5, input), input); + assertEquals(segmentPruner.prune(brokerRequest6, input), input); + assertEquals(segmentPruner.prune(brokerRequest7, input), input); + assertEquals(segmentPruner.prune(brokerRequest8, input), input); + assertEquals(segmentPruner.prune(brokerRequest9, input), Set.of()); // Query with invalid range // Test different time range String segment0 = "segment0"; - onlineSegments.add(segment0); setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment0, 10, 60, TimeUnit.DAYS); - + onlineSegments.add(segment0); String segment1 = "segment1"; - onlineSegments.add(segment1); setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment1, 20, 30, TimeUnit.DAYS); - + onlineSegments.add(segment1); String segment2 = "segment2"; - onlineSegments.add(segment2); setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment2, 50, 65, TimeUnit.DAYS); - + onlineSegments.add(segment2); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment1, segment2))); - assertEquals(segmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - Collections.singleton(segment0)); - assertEquals(segmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals(segmentPruner.prune(brokerRequest4, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest5, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest6, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest7, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - Collections.emptySet()); + input = Set.of(segment0, segment1, segment2); + assertEquals(segmentPruner.prune(brokerRequest1, input), input); + assertEquals(segmentPruner.prune(brokerRequest2, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest3, input), Set.of(segment0, segment1)); + assertEquals(segmentPruner.prune(brokerRequest4, input), Set.of(segment0, segment2)); + assertEquals(segmentPruner.prune(brokerRequest5, input), Set.of(segment0, segment2)); + assertEquals(segmentPruner.prune(brokerRequest6, input), Set.of(segment0, segment2)); + assertEquals(segmentPruner.prune(brokerRequest7, input), Set.of(segment0, segment2)); + assertEquals(segmentPruner.prune(brokerRequest8, input), Set.of(segment0, segment1)); + assertEquals(segmentPruner.prune(brokerRequest9, input), Set.of()); // Query with invalid range // Update metadata without external view change or refreshing should have no effect setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment2, 20, 30, TimeUnit.DAYS); - assertEquals(segmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment1, segment2))); - assertEquals(segmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - Collections.singleton(segment0)); - assertEquals(segmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals(segmentPruner.prune(brokerRequest4, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest5, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest6, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest7, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - Collections.emptySet()); + assertEquals(segmentPruner.prune(brokerRequest1, input), input); + assertEquals(segmentPruner.prune(brokerRequest2, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest3, input), Set.of(segment0, segment1)); + assertEquals(segmentPruner.prune(brokerRequest4, input), Set.of(segment0, segment2)); + assertEquals(segmentPruner.prune(brokerRequest5, input), Set.of(segment0, segment2)); + assertEquals(segmentPruner.prune(brokerRequest6, input), Set.of(segment0, segment2)); + assertEquals(segmentPruner.prune(brokerRequest7, input), Set.of(segment0, segment2)); + assertEquals(segmentPruner.prune(brokerRequest8, input), Set.of(segment0, segment1)); + assertEquals(segmentPruner.prune(brokerRequest9, input), Set.of()); // Query with invalid range // Refresh the changed segment should update the segment pruner segmentZkMetadataFetcher.refreshSegment(segment2); - assertEquals(segmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment1, segment2))); - assertEquals(segmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - Collections.singleton(segment0)); - assertEquals(segmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment1, segment2))); - assertEquals(segmentPruner.prune(brokerRequest4, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - Collections.singleton(segment0)); - assertEquals(segmentPruner.prune(brokerRequest5, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - Collections.singleton(segment0)); - assertEquals(segmentPruner.prune(brokerRequest6, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - Collections.singleton(segment0)); - assertEquals(segmentPruner.prune(brokerRequest7, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - Collections.emptySet()); + assertEquals(segmentPruner.prune(brokerRequest1, input), input); + assertEquals(segmentPruner.prune(brokerRequest2, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest3, input), input); + assertEquals(segmentPruner.prune(brokerRequest4, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest5, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest6, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest7, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest8, input), input); + assertEquals(segmentPruner.prune(brokerRequest9, input), Set.of()); // Query with invalid range } @Test @@ -545,215 +508,175 @@ public void testTimeSegmentPrunerSimpleDateFormat() { BrokerRequest brokerRequest3 = CalciteSqlCompiler.compileToBrokerRequest(SDF_QUERY_3); BrokerRequest brokerRequest4 = CalciteSqlCompiler.compileToBrokerRequest(SDF_QUERY_4); BrokerRequest brokerRequest5 = CalciteSqlCompiler.compileToBrokerRequest(SDF_QUERY_5); + // NOTE: Ideal state and external view are not used in the current implementation IdealState idealState = Mockito.mock(IdealState.class); ExternalView externalView = Mockito.mock(ExternalView.class); - TableConfig tableConfig = getTableConfig(RAW_TABLE_NAME, TableType.REALTIME); - setSchemaDateTimeFieldSpecSDF(RAW_TABLE_NAME, SDF_PATTERN); - - TimeSegmentPruner segmentPruner = SegmentPrunerFactory.createTimeSegmentPruner(tableConfig, _propertyStore); - SegmentZkMetadataFetcher segmentZkMetadataFetcher = new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, - _propertyStore); + TableConfig tableConfig = + new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setTimeColumnName(TIME_COLUMN).build(); + DateTimeFieldSpec timeFieldSpec = + new DateTimeFieldSpec(TIME_COLUMN, DataType.STRING, "SIMPLE_DATE_FORMAT|" + SDF_PATTERN, "1:DAYS"); + TimeSegmentPruner segmentPruner = new TimeSegmentPruner(tableConfig, timeFieldSpec); + SegmentZkMetadataFetcher segmentZkMetadataFetcher = + new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, _propertyStore); segmentZkMetadataFetcher.register(segmentPruner); - Schema schema = ZKMetadataProvider.getTableSchema(_propertyStore, RAW_TABLE_NAME); - DateTimeFormatSpec dateTimeFormatSpec = schema.getSpecForTimeColumn(TIME_COLUMN).getFormatSpec(); - + DateTimeFormatSpec timeFormatSpec = timeFieldSpec.getFormatSpec(); Set onlineSegments = new HashSet<>(); String segment0 = "segment0"; + setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment0, timeFormatSpec.fromFormatToMillis("20200101"), + timeFormatSpec.fromFormatToMillis("20200228"), TimeUnit.MILLISECONDS); onlineSegments.add(segment0); - setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment0, dateTimeFormatSpec.fromFormatToMillis("20200101"), - dateTimeFormatSpec.fromFormatToMillis("20200228"), TimeUnit.MILLISECONDS); - String segment1 = "segment1"; + setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment1, timeFormatSpec.fromFormatToMillis("20200201"), + timeFormatSpec.fromFormatToMillis("20200530"), TimeUnit.MILLISECONDS); onlineSegments.add(segment1); - setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment1, dateTimeFormatSpec.fromFormatToMillis("20200201"), - dateTimeFormatSpec.fromFormatToMillis("20200530"), TimeUnit.MILLISECONDS); - String segment2 = "segment2"; + setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment2, timeFormatSpec.fromFormatToMillis("20200401"), + timeFormatSpec.fromFormatToMillis("20200430"), TimeUnit.MILLISECONDS); onlineSegments.add(segment2); - setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment2, dateTimeFormatSpec.fromFormatToMillis("20200401"), - dateTimeFormatSpec.fromFormatToMillis("20200430"), TimeUnit.MILLISECONDS); - segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), Collections.singleton(segment0)); - assertEquals(segmentPruner.prune(brokerRequest2, onlineSegments), new HashSet<>(Arrays.asList(segment0, segment1))); - assertEquals(segmentPruner.prune(brokerRequest3, onlineSegments), Collections.singleton(segment1)); - assertEquals(segmentPruner.prune(brokerRequest4, onlineSegments), - new HashSet<>(Arrays.asList(segment0, segment1, segment2))); - assertEquals(segmentPruner.prune(brokerRequest5, onlineSegments), Collections.emptySet()); + + Set input = Set.of(segment0, segment1, segment2); + assertEquals(segmentPruner.prune(brokerRequest1, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest2, input), Set.of(segment0, segment1)); + assertEquals(segmentPruner.prune(brokerRequest3, input), Set.of(segment1)); + assertEquals(segmentPruner.prune(brokerRequest4, input), input); + assertEquals(segmentPruner.prune(brokerRequest5, input), Set.of()); } @Test - public void testTimeSegmentPrunerSql() { - BrokerRequest brokerRequest1 = CalciteSqlCompiler.compileToBrokerRequest(SQL_TIME_QUERY_1); - BrokerRequest brokerRequest2 = CalciteSqlCompiler.compileToBrokerRequest(SQL_TIME_QUERY_2); + public void testTimeSegmentPrunerTimestampFormat() { + BrokerRequest brokerRequest1 = CalciteSqlCompiler.compileToBrokerRequest(TIMESTAMP_QUERY_1); + BrokerRequest brokerRequest2 = CalciteSqlCompiler.compileToBrokerRequest(TIMESTAMP_QUERY_2); + BrokerRequest brokerRequest3 = CalciteSqlCompiler.compileToBrokerRequest(TIMESTAMP_QUERY_3); + BrokerRequest brokerRequest4 = CalciteSqlCompiler.compileToBrokerRequest(TIMESTAMP_QUERY_4); + BrokerRequest brokerRequest5 = CalciteSqlCompiler.compileToBrokerRequest(TIMESTAMP_QUERY_5); + BrokerRequest brokerRequest6 = CalciteSqlCompiler.compileToBrokerRequest(TIMESTAMP_QUERY_6); + // NOTE: Ideal state and external view are not used in the current implementation IdealState idealState = Mockito.mock(IdealState.class); ExternalView externalView = Mockito.mock(ExternalView.class); - TableConfig tableConfig = getTableConfig(RAW_TABLE_NAME, TableType.REALTIME); - setSchemaDateTimeFieldSpec(RAW_TABLE_NAME, TimeUnit.DAYS); - - TimeSegmentPruner segmentPruner = SegmentPrunerFactory.createTimeSegmentPruner(tableConfig, _propertyStore); - SegmentZkMetadataFetcher segmentZkMetadataFetcher = new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, - _propertyStore); + TableConfig tableConfig = + new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).setTimeColumnName(TIME_COLUMN).build(); + // Intentionally put EPOCH as the format which Pinot should handle + DateTimeFieldSpec timeFieldSpec = + new DateTimeFieldSpec(TIME_COLUMN, DataType.TIMESTAMP, "EPOCH|MILLISECONDS", "1:DAYS"); + TimeSegmentPruner segmentPruner = new TimeSegmentPruner(tableConfig, timeFieldSpec); + SegmentZkMetadataFetcher segmentZkMetadataFetcher = + new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, _propertyStore); segmentZkMetadataFetcher.register(segmentPruner); + DateTimeFormatSpec timeFormatSpec = timeFieldSpec.getFormatSpec(); Set onlineSegments = new HashSet<>(); String segment0 = "segment0"; + setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment0, + timeFormatSpec.fromFormatToMillis("2020-01-01 00:00:00"), + timeFormatSpec.fromFormatToMillis("2020-02-28 00:00:00"), TimeUnit.MILLISECONDS); onlineSegments.add(segment0); - setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment0, 10, 60, TimeUnit.DAYS); String segment1 = "segment1"; + setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment1, + timeFormatSpec.fromFormatToMillis("2020-02-01 00:00:00"), + timeFormatSpec.fromFormatToMillis("2020-05-30 00:00:00"), TimeUnit.MILLISECONDS); onlineSegments.add(segment1); - setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment1, 20, 30, TimeUnit.DAYS); String segment2 = "segment2"; + setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment2, + timeFormatSpec.fromFormatToMillis("2020-04-01 00:00:00"), + timeFormatSpec.fromFormatToMillis("2020-04-30 00:00:00"), TimeUnit.MILLISECONDS); onlineSegments.add(segment2); - setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment2, 50, 65, TimeUnit.DAYS); segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest2, onlineSegments), new HashSet<>(Arrays.asList(segment0, segment1))); + Set input = Set.of(segment0, segment1, segment2); + assertEquals(segmentPruner.prune(brokerRequest1, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest2, input), Set.of(segment0)); + assertEquals(segmentPruner.prune(brokerRequest3, input), Set.of(segment0, segment1)); + assertEquals(segmentPruner.prune(brokerRequest4, input), Set.of(segment0, segment1)); + assertEquals(segmentPruner.prune(brokerRequest5, input), input); + assertEquals(segmentPruner.prune(brokerRequest6, input), input); } @Test public void testEmptySegmentPruner() { BrokerRequest brokerRequest1 = CalciteSqlCompiler.compileToBrokerRequest(QUERY_1); - BrokerRequest brokerRequest2 = CalciteSqlCompiler.compileToBrokerRequest(QUERY_2); - BrokerRequest brokerRequest3 = CalciteSqlCompiler.compileToBrokerRequest(QUERY_3); + // NOTE: Ideal state and external view are not used in the current implementation IdealState idealState = Mockito.mock(IdealState.class); ExternalView externalView = Mockito.mock(ExternalView.class); - TableConfig tableConfig = getTableConfig(RAW_TABLE_NAME, TableType.REALTIME); + TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).build(); - // init with list of segments + // Init with a list of segments EmptySegmentPruner segmentPruner = new EmptySegmentPruner(tableConfig); - SegmentZkMetadataFetcher segmentZkMetadataFetcher = new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, - _propertyStore); + SegmentZkMetadataFetcher segmentZkMetadataFetcher = + new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, _propertyStore); segmentZkMetadataFetcher.register(segmentPruner); Set onlineSegments = new HashSet<>(); String segment0 = "segment0"; - onlineSegments.add(segment0); setSegmentZKTotalDocsMetadata(REALTIME_TABLE_NAME, segment0, 10); + onlineSegments.add(segment0); String segment1 = "segment1"; - onlineSegments.add(segment1); setSegmentZKTotalDocsMetadata(REALTIME_TABLE_NAME, segment1, 0); + onlineSegments.add(segment1); segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Collections.singletonList(segment0))); - assertEquals(segmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Collections.singletonList(segment0))); - assertEquals(segmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1))), - new HashSet<>(Collections.singletonList(segment0))); - - // init with empty list of segments + assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), Set.of(segment0)); + + // Init with no segment segmentPruner = new EmptySegmentPruner(tableConfig); segmentZkMetadataFetcher = new SegmentZkMetadataFetcher(REALTIME_TABLE_NAME, _propertyStore); segmentZkMetadataFetcher.register(segmentPruner); onlineSegments.clear(); segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, Collections.emptySet()), Collections.emptySet()); - assertEquals(segmentPruner.prune(brokerRequest2, Collections.emptySet()), Collections.emptySet()); - assertEquals(segmentPruner.prune(brokerRequest3, Collections.emptySet()), Collections.emptySet()); + assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), onlineSegments); // Segments without metadata (not updated yet) should not be pruned String newSegment = "newSegment"; onlineSegments.add(newSegment); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, Collections.singleton(newSegment)), - Collections.singleton(newSegment)); - assertEquals(segmentPruner.prune(brokerRequest2, Collections.singleton(newSegment)), - Collections.singleton(newSegment)); - assertEquals(segmentPruner.prune(brokerRequest3, Collections.singleton(newSegment)), - Collections.singleton(newSegment)); + assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), onlineSegments); // Segments without totalDocs metadata should not be pruned - onlineSegments.clear(); String segmentWithoutTotalDocsMetadata = "segmentWithoutTotalDocsMetadata"; - onlineSegments.add(segmentWithoutTotalDocsMetadata); SegmentZKMetadata segmentZKMetadataWithoutTotalDocsMetadata = new SegmentZKMetadata(segmentWithoutTotalDocsMetadata); - segmentZKMetadataWithoutTotalDocsMetadata.setStatus(CommonConstants.Segment.Realtime.Status.IN_PROGRESS); ZKMetadataProvider.setSegmentZKMetadata(_propertyStore, REALTIME_TABLE_NAME, segmentZKMetadataWithoutTotalDocsMetadata); + onlineSegments.add(segmentWithoutTotalDocsMetadata); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, Collections.singleton(segmentWithoutTotalDocsMetadata)), - Collections.singleton(segmentWithoutTotalDocsMetadata)); - assertEquals(segmentPruner.prune(brokerRequest2, Collections.singleton(segmentWithoutTotalDocsMetadata)), - Collections.singleton(segmentWithoutTotalDocsMetadata)); - assertEquals(segmentPruner.prune(brokerRequest3, Collections.singleton(segmentWithoutTotalDocsMetadata)), - Collections.singleton(segmentWithoutTotalDocsMetadata)); + assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), onlineSegments); // Segments with -1 totalDocs should not be pruned - onlineSegments.clear(); String segmentWithNegativeTotalDocsMetadata = "segmentWithNegativeTotalDocsMetadata"; - onlineSegments.add(segmentWithNegativeTotalDocsMetadata); setSegmentZKTotalDocsMetadata(REALTIME_TABLE_NAME, segmentWithNegativeTotalDocsMetadata, -1); + onlineSegments.add(segmentWithNegativeTotalDocsMetadata); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, Collections.singleton(segmentWithNegativeTotalDocsMetadata)), - Collections.singleton(segmentWithNegativeTotalDocsMetadata)); - assertEquals(segmentPruner.prune(brokerRequest2, Collections.singleton(segmentWithNegativeTotalDocsMetadata)), - Collections.singleton(segmentWithNegativeTotalDocsMetadata)); - assertEquals(segmentPruner.prune(brokerRequest3, Collections.singleton(segmentWithNegativeTotalDocsMetadata)), - Collections.singleton(segmentWithNegativeTotalDocsMetadata)); + assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), onlineSegments); // Prune segments with 0 total docs onlineSegments.clear(); - onlineSegments.add(segment0); setSegmentZKTotalDocsMetadata(REALTIME_TABLE_NAME, segment0, 10); - onlineSegments.add(segment1); + onlineSegments.add(segment0); setSegmentZKTotalDocsMetadata(REALTIME_TABLE_NAME, segment1, 0); + onlineSegments.add(segment1); String segment2 = "segment2"; - onlineSegments.add(segment2); setSegmentZKTotalDocsMetadata(REALTIME_TABLE_NAME, segment2, -1); - + onlineSegments.add(segment2); segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); - assertEquals(segmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); + assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), Set.of(segment0, segment2)); // Update metadata without external view change or refreshing should have no effect - setSegmentZKTimeRangeMetadata(REALTIME_TABLE_NAME, segment2, 20, 30, TimeUnit.DAYS); setSegmentZKTotalDocsMetadata(REALTIME_TABLE_NAME, segment2, 0); - assertEquals(segmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); - assertEquals(segmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Arrays.asList(segment0, segment2))); + assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), Set.of(segment0, segment2)); // Refresh the changed segment should update the segment pruner segmentZkMetadataFetcher.refreshSegment(segment2); - assertEquals(segmentPruner.prune(brokerRequest1, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Collections.singletonList(segment0))); - assertEquals(segmentPruner.prune(brokerRequest2, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Collections.singletonList(segment0))); - assertEquals(segmentPruner.prune(brokerRequest3, new HashSet<>(Arrays.asList(segment0, segment1, segment2))), - new HashSet<>(Collections.singletonList(segment0))); - } - - private TableConfig getTableConfig(String rawTableName, TableType type) { - return new TableConfigBuilder(type).setTableName(rawTableName).setTimeColumnName(TIME_COLUMN).build(); - } - - private void setSchemaDateTimeFieldSpec(String rawTableName, TimeUnit timeUnit) { - ZKMetadataProvider.setSchema(_propertyStore, new Schema.SchemaBuilder().setSchemaName(rawTableName) - .addDateTime(TIME_COLUMN, FieldSpec.DataType.LONG, "1:" + timeUnit + ":EPOCH", "1:" + timeUnit).build()); - } - - private void setSchemaDateTimeFieldSpecSDF(String rawTableName, String format) { - ZKMetadataProvider.setSchema(_propertyStore, new Schema.SchemaBuilder().setSchemaName(rawTableName) - .addDateTime(TIME_COLUMN, FieldSpec.DataType.STRING, "1:DAYS:SIMPLE_DATE_FORMAT:" + format, "1:DAYS").build()); + assertEquals(segmentPruner.prune(brokerRequest1, onlineSegments), Set.of(segment0)); } private void setSegmentZKPartitionMetadata(String tableNameWithType, String segment, String partitionFunction, int numPartitions, int partitionId) { SegmentZKMetadata segmentZKMetadata = new SegmentZKMetadata(segment); - segmentZKMetadata.setPartitionMetadata(new SegmentPartitionMetadata(Collections.singletonMap(PARTITION_COLUMN_1, - new ColumnPartitionMetadata(partitionFunction, numPartitions, Collections.singleton(partitionId), null)))); + segmentZKMetadata.setPartitionMetadata(new SegmentPartitionMetadata(Map.of(PARTITION_COLUMN_1, + new ColumnPartitionMetadata(partitionFunction, numPartitions, Set.of(partitionId), null)))); ZKMetadataProvider.setSegmentZKMetadata(_propertyStore, tableNameWithType, segmentZKMetadata); } diff --git a/pinot-common/src/test/java/org/apache/pinot/common/data/SchemaTest.java b/pinot-common/src/test/java/org/apache/pinot/common/data/SchemaTest.java index 626a09100555..e8fd1287293c 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/data/SchemaTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/data/SchemaTest.java @@ -221,8 +221,7 @@ public void testFetchFieldSpecForTime() { .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.DAYS, "time"), null) .addDateTime("dateTime0", FieldSpec.DataType.LONG, "1:HOURS:EPOCH", "1:HOURS") .addDateTime("dateTime1", FieldSpec.DataType.TIMESTAMP, "1:MILLISECONDS:EPOCH", "1:MILLISECONDS") - .addDateTime("dateTime2", FieldSpec.DataType.INT, "1:DAYS:EPOCH", "1:DAYS") - .build(); + .addDateTime("dateTime2", FieldSpec.DataType.INT, "1:DAYS:EPOCH", "1:DAYS").build(); // Test method which fetches the DateTimeFieldSpec given the timeColumnName // Test is on TIME @@ -254,7 +253,7 @@ public void testFetchFieldSpecForTime() { Assert.assertEquals(dateTimeFieldSpec.getDataType(), FieldSpec.DataType.TIMESTAMP); Assert.assertTrue(dateTimeFieldSpec.isSingleValueField()); Assert.assertEquals(dateTimeFieldSpec.getDefaultNullValue(), 0L); - Assert.assertEquals(dateTimeFieldSpec.getFormat(), "1:MILLISECONDS:EPOCH"); + Assert.assertEquals(dateTimeFieldSpec.getFormat(), "TIMESTAMP"); Assert.assertEquals(dateTimeFieldSpec.getGranularity(), "1:MILLISECONDS"); dateTimeFieldSpec = schema.getSpecForTimeColumn("dateTime2"); @@ -326,15 +325,10 @@ public void testSerializeDeserialize() @Test public void testSerializeDeserializeOptions() throws IOException { - String json = "{\n" - + " \"primaryKeyColumns\" : null,\n" - + " \"timeFieldSpec\" : null,\n" - + " \"schemaName\" : null,\n" - + " \"enableColumnBasedNullHandling\" : true,\n" - + " \"dimensionFieldSpecs\" : [ ],\n" - + " \"metricFieldSpecs\" : [ ],\n" - + " \"dateTimeFieldSpecs\" : [ ]\n" - + "}"; + String json = + "{\n" + " \"primaryKeyColumns\" : null,\n" + " \"timeFieldSpec\" : null,\n" + " \"schemaName\" : null,\n" + + " \"enableColumnBasedNullHandling\" : true,\n" + " \"dimensionFieldSpecs\" : [ ],\n" + + " \"metricFieldSpecs\" : [ ],\n" + " \"dateTimeFieldSpecs\" : [ ]\n" + "}"; JsonNode expectedNode = JsonUtils.stringToJsonNode(json); Schema schema = JsonUtils.jsonNodeToObject(expectedNode, Schema.class); @@ -363,6 +357,17 @@ public void testSimpleDateFormat() Assert.assertEquals(schemaFromJson.hashCode(), schema.hashCode()); } + @Test + public void testTimestampFormatOverride() + throws Exception { + URL resourceUrl = getClass().getClassLoader().getResource("schemaTest.schema"); + Assert.assertNotNull(resourceUrl); + Schema schema = Schema.fromFile(new File(resourceUrl.getFile())); + DateTimeFieldSpec fieldSpec = schema.getDateTimeSpec("dateTime3"); + Assert.assertNotNull(fieldSpec); + Assert.assertEquals(fieldSpec.getFormat(), "TIMESTAMP"); + } + @Test public void testByteType() throws Exception { diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFieldSpec.java b/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFieldSpec.java index ea9285a10487..dbb92090d177 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFieldSpec.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFieldSpec.java @@ -25,6 +25,7 @@ import javax.annotation.Nullable; import org.apache.pinot.spi.utils.EqualityUtils; + @SuppressWarnings("unused") @JsonIgnoreProperties(ignoreUnknown = true) public final class DateTimeFieldSpec extends FieldSpec { @@ -74,6 +75,10 @@ public DateTimeFieldSpec(String name, DataType dataType, String format, String g @Nullable Object sampleValue) { super(name, dataType, true); + // Override format to be "TIMESTAMP" for TIMESTAMP data type because the format is implicit + if (dataType == DataType.TIMESTAMP) { + format = TimeFormat.TIMESTAMP.name(); + } _format = format; _granularity = granularity; _formatSpec = new DateTimeFormatSpec(format); @@ -119,13 +124,23 @@ public void setSingleValueField(boolean isSingleValueField) { Preconditions.checkArgument(isSingleValueField, "Unsupported multi-value for date time field."); } + @Override + public void setDataType(DataType dataType) { + super.setDataType(dataType); + if (dataType == DataType.TIMESTAMP) { + _format = TimeFormat.TIMESTAMP.name(); + } + } + public String getFormat() { return _format; } // Required by JSON de-serializer. DO NOT REMOVE. public void setFormat(String format) { - _format = format; + if (_dataType != DataType.TIMESTAMP) { + _format = format; + } } @JsonIgnore From 00b5a73b20415aaea0f9bb4f44c07ad21f563193 Mon Sep 17 00:00:00 2001 From: "Xiaotian (Jackie) Jiang" <17555551+Jackie-Jiang@users.noreply.github.com> Date: Mon, 8 Apr 2024 01:10:31 -0700 Subject: [PATCH 04/72] Keep get tables API with and without database (#12804) --- .../controller/BaseControllerStarter.java | 125 +++--- .../resources/PinotBrokerRestletResource.java | 13 +- .../PinotLeadControllerRestletResource.java | 3 +- .../helix/core/PinotHelixResourceManager.java | 79 ++-- .../cleanup/StaleInstancesCleanupTask.java | 9 +- .../helix/core/minion/PinotTaskManager.java | 25 +- .../periodictask/ControllerPeriodicTask.java | 9 +- .../helix/RealtimeConsumerMonitorTest.java | 85 ++-- .../helix/SegmentStatusCheckerTest.java | 425 ++++++++---------- .../ControllerPeriodicTaskTest.java | 20 +- .../core/retention/RetentionManagerTest.java | 71 +-- 11 files changed, 376 insertions(+), 488 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/BaseControllerStarter.java b/pinot-controller/src/main/java/org/apache/pinot/controller/BaseControllerStarter.java index f78a49f2f927..91df2e07431d 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/BaseControllerStarter.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/BaseControllerStarter.java @@ -201,8 +201,8 @@ public void init(PinotConfiguration pinotConfiguration) _helixClusterName = _config.getHelixClusterName(); ServiceStartableUtils.applyClusterConfig(_config, _helixZkURL, _helixClusterName, ServiceRole.CONTROLLER); - PinotInsecureMode.setPinotInInsecureMode( - Boolean.valueOf(_config.getProperty(CommonConstants.CONFIG_OF_PINOT_INSECURE_MODE, + PinotInsecureMode.setPinotInInsecureMode(Boolean.valueOf( + _config.getProperty(CommonConstants.CONFIG_OF_PINOT_INSECURE_MODE, CommonConstants.DEFAULT_PINOT_INSECURE_MODE))); setupHelixSystemProperties(); @@ -531,8 +531,8 @@ protected void configure() { if (tableConfig != null) { Map streamConfigMap = IngestionConfigUtils.getStreamConfigMap(tableConfig); try { - StreamConfig.validateConsumerType( - streamConfigMap.getOrDefault(StreamConfigProperties.STREAM_TYPE, "kafka"), streamConfigMap); + StreamConfig.validateConsumerType(streamConfigMap.getOrDefault(StreamConfigProperties.STREAM_TYPE, "kafka"), + streamConfigMap); } catch (Exception e) { existingHlcTables.add(rt); } @@ -587,66 +587,63 @@ public void fixSchemaNameInTableConfig() { AtomicInteger failedToUpdateTableConfigCount = new AtomicInteger(); ZkHelixPropertyStore propertyStore = _helixResourceManager.getPropertyStore(); - _helixResourceManager.getDatabaseNames().stream() - .map(_helixResourceManager::getAllTables) - .flatMap(List::stream) - .forEach(tableNameWithType -> { - Pair tableConfigWithVersion = - ZKMetadataProvider.getTableConfigWithVersion(propertyStore, tableNameWithType); - if (tableConfigWithVersion == null) { - // This might due to table deletion, just log it here. - LOGGER.warn("Failed to find table config for table: {}, the table likely already got deleted", - tableNameWithType); - return; - } - TableConfig tableConfig = tableConfigWithVersion.getLeft(); - String rawTableName = TableNameBuilder.extractRawTableName(tableNameWithType); - String schemaPath = ZKMetadataProvider.constructPropertyStorePathForSchema(rawTableName); - boolean schemaExists = propertyStore.exists(schemaPath, AccessOption.PERSISTENT); - String existSchemaName = tableConfig.getValidationConfig().getSchemaName(); - if (existSchemaName == null || existSchemaName.equals(rawTableName)) { - // Although the table config is valid, we still need to ensure the schema exists - if (!schemaExists) { - LOGGER.warn("Failed to find schema for table: {}", tableNameWithType); - tableWithoutSchemaCount.getAndIncrement(); - return; - } - // Table config is already in good status - return; - } - misconfiguredTableCount.getAndIncrement(); - if (schemaExists) { - // If a schema named `rawTableName` already exists, then likely this is a misconfiguration. - // Reset schema name in table config to null to let the table point to the existing schema. - LOGGER.warn("Schema: {} already exists, fix the schema name in table config from {} to null", rawTableName, - existSchemaName); - } else { - // Copy the schema current table referring to to `rawTableName` if it does not exist - Schema schema = _helixResourceManager.getSchema(existSchemaName); - if (schema == null) { - LOGGER.warn("Failed to find schema: {} for table: {}", existSchemaName, tableNameWithType); - tableWithoutSchemaCount.getAndIncrement(); - return; - } - schema.setSchemaName(rawTableName); - if (propertyStore.create(schemaPath, SchemaUtils.toZNRecord(schema), AccessOption.PERSISTENT)) { - LOGGER.info("Copied schema: {} to {}", existSchemaName, rawTableName); - } else { - LOGGER.warn("Failed to copy schema: {} to {}", existSchemaName, rawTableName); - failedToCopySchemaCount.getAndIncrement(); - return; - } - } - // Update table config to remove schema name - tableConfig.getValidationConfig().setSchemaName(null); - if (ZKMetadataProvider.setTableConfig(propertyStore, tableConfig, tableConfigWithVersion.getRight())) { - LOGGER.info("Removed schema name from table config for table: {}", tableNameWithType); - fixedSchemaTableCount.getAndIncrement(); - } else { - LOGGER.warn("Failed to update table config for table: {}", tableNameWithType); - failedToUpdateTableConfigCount.getAndIncrement(); - } - }); + _helixResourceManager.getAllTables().forEach(tableNameWithType -> { + Pair tableConfigWithVersion = + ZKMetadataProvider.getTableConfigWithVersion(propertyStore, tableNameWithType); + if (tableConfigWithVersion == null) { + // This might due to table deletion, just log it here. + LOGGER.warn("Failed to find table config for table: {}, the table likely already got deleted", + tableNameWithType); + return; + } + TableConfig tableConfig = tableConfigWithVersion.getLeft(); + String rawTableName = TableNameBuilder.extractRawTableName(tableNameWithType); + String schemaPath = ZKMetadataProvider.constructPropertyStorePathForSchema(rawTableName); + boolean schemaExists = propertyStore.exists(schemaPath, AccessOption.PERSISTENT); + String existSchemaName = tableConfig.getValidationConfig().getSchemaName(); + if (existSchemaName == null || existSchemaName.equals(rawTableName)) { + // Although the table config is valid, we still need to ensure the schema exists + if (!schemaExists) { + LOGGER.warn("Failed to find schema for table: {}", tableNameWithType); + tableWithoutSchemaCount.getAndIncrement(); + return; + } + // Table config is already in good status + return; + } + misconfiguredTableCount.getAndIncrement(); + if (schemaExists) { + // If a schema named `rawTableName` already exists, then likely this is a misconfiguration. + // Reset schema name in table config to null to let the table point to the existing schema. + LOGGER.warn("Schema: {} already exists, fix the schema name in table config from {} to null", rawTableName, + existSchemaName); + } else { + // Copy the schema current table referring to to `rawTableName` if it does not exist + Schema schema = _helixResourceManager.getSchema(existSchemaName); + if (schema == null) { + LOGGER.warn("Failed to find schema: {} for table: {}", existSchemaName, tableNameWithType); + tableWithoutSchemaCount.getAndIncrement(); + return; + } + schema.setSchemaName(rawTableName); + if (propertyStore.create(schemaPath, SchemaUtils.toZNRecord(schema), AccessOption.PERSISTENT)) { + LOGGER.info("Copied schema: {} to {}", existSchemaName, rawTableName); + } else { + LOGGER.warn("Failed to copy schema: {} to {}", existSchemaName, rawTableName); + failedToCopySchemaCount.getAndIncrement(); + return; + } + } + // Update table config to remove schema name + tableConfig.getValidationConfig().setSchemaName(null); + if (ZKMetadataProvider.setTableConfig(propertyStore, tableConfig, tableConfigWithVersion.getRight())) { + LOGGER.info("Removed schema name from table config for table: {}", tableNameWithType); + fixedSchemaTableCount.getAndIncrement(); + } else { + LOGGER.warn("Failed to update table config for table: {}", tableNameWithType); + failedToUpdateTableConfigCount.getAndIncrement(); + } + }); LOGGER.info( "Found {} tables misconfigured, {} tables without schema. Successfully fixed schema for {} tables, failed to " + "fix {} tables due to copy schema failure, failed to fix {} tables due to update table config failure.", diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotBrokerRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotBrokerRestletResource.java index 1863819ab97e..4fdca71a8200 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotBrokerRestletResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotBrokerRestletResource.java @@ -127,10 +127,8 @@ public List getBrokersForTenant( @ApiOperation(value = "List tables to brokers mappings", notes = "List tables to brokers mappings") public Map> getTablesToBrokersMapping( @ApiParam(value = "ONLINE|OFFLINE") @QueryParam("state") String state, @Context HttpHeaders headers) { - Map> resultMap = new HashMap<>(); - _pinotHelixResourceManager.getAllRawTables(headers.getHeaderString(DATABASE)) - .forEach(table -> resultMap.put(table, getBrokersForTable(table, null, state, headers))); - return resultMap; + return _pinotHelixResourceManager.getAllRawTables(headers.getHeaderString(DATABASE)).stream() + .collect(Collectors.toMap(table -> table, table -> getBrokersForTable(table, null, state, headers))); } @GET @@ -201,11 +199,8 @@ public List getBrokersForTenantV2( @ApiOperation(value = "List tables to brokers mappings", notes = "List tables to brokers mappings") public Map> getTablesToBrokersMappingV2( @ApiParam(value = "ONLINE|OFFLINE") @QueryParam("state") String state, @Context HttpHeaders headers) { - Map> resultMap = new HashMap<>(); - String databaseName = headers.getHeaderString(DATABASE); - _pinotHelixResourceManager.getAllRawTables(databaseName).stream() - .forEach(table -> resultMap.put(table, getBrokersForTableV2(table, null, state, headers))); - return resultMap; + return _pinotHelixResourceManager.getAllRawTables(headers.getHeaderString(DATABASE)).stream() + .collect(Collectors.toMap(table -> table, table -> getBrokersForTableV2(table, null, state, headers))); } @GET diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotLeadControllerRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotLeadControllerRestletResource.java index 5f09f74336a9..16f13e047165 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotLeadControllerRestletResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotLeadControllerRestletResource.java @@ -109,8 +109,7 @@ public LeadControllerResponse getLeadersForAllTables(@Context HttpHeaders header } // Assigns all the tables to the relevant partitions. - List tableNames = _pinotHelixResourceManager.getAllTables( - headers.getHeaderString(DATABASE)); + List tableNames = _pinotHelixResourceManager.getAllTables(headers.getHeaderString(DATABASE)); for (String tableName : tableNames) { String rawTableName = TableNameBuilder.extractRawTableName(tableName); int partitionId = LeadControllerUtils.getPartitionIdForTable(rawTableName); diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java index f1761ce8666e..680e475d6353 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java @@ -716,13 +716,12 @@ public List getAllResources() { } /** - * Get all table names (with type suffix) in default database. + * Get all table names (with type suffix) in all databases. * - * @return List of table names in default database + * @return List of table names */ - @Deprecated public List getAllTables() { - return getAllTables(null); + return getAllResources().stream().filter(TableNameBuilder::isTableResource).collect(Collectors.toList()); } /** @@ -732,23 +731,18 @@ public List getAllTables() { * @return List of table names in provided database name */ public List getAllTables(@Nullable String databaseName) { - List tableNames = new ArrayList<>(); - for (String resourceName : getAllResources()) { - if (TableNameBuilder.isTableResource(resourceName) - && DatabaseUtils.isPartOfDatabase(resourceName, databaseName)) { - tableNames.add(resourceName); - } - } - return tableNames; + return getAllResources().stream().filter( + resourceName -> TableNameBuilder.isTableResource(resourceName) && DatabaseUtils.isPartOfDatabase(resourceName, + databaseName)).collect(Collectors.toList()); } /** - * Get all offline table names from default database. + * Get all offline table names from all databases. * - * @return List of offline table names in default database + * @return List of offline table names */ public List getAllOfflineTables() { - return getAllOfflineTables(null); + return getAllResources().stream().filter(TableNameBuilder::isOfflineTableResource).collect(Collectors.toList()); } /** @@ -758,23 +752,18 @@ public List getAllOfflineTables() { * @return List of offline table names in provided database name */ public List getAllOfflineTables(@Nullable String databaseName) { - List offlineTableNames = new ArrayList<>(); - for (String resourceName : getAllResources()) { - if (DatabaseUtils.isPartOfDatabase(resourceName, databaseName) - && TableNameBuilder.isOfflineTableResource(resourceName)) { - offlineTableNames.add(resourceName); - } - } - return offlineTableNames; + return getAllResources().stream().filter( + resourceName -> TableNameBuilder.isOfflineTableResource(resourceName) && DatabaseUtils.isPartOfDatabase( + resourceName, databaseName)).collect(Collectors.toList()); } /** - * Get all dimension table names from default database. + * Get all dimension table names from all databases. * - * @return List of dimension table names in default database + * @return List of dimension table names */ public List getAllDimensionTables() { - return getAllDimensionTables(null); + return _tableCache.getAllDimensionTables(); } /** @@ -785,17 +774,16 @@ public List getAllDimensionTables() { */ public List getAllDimensionTables(@Nullable String databaseName) { return _tableCache.getAllDimensionTables().stream() - .filter(table -> DatabaseUtils.isPartOfDatabase(table, databaseName)) - .collect(Collectors.toList()); + .filter(table -> DatabaseUtils.isPartOfDatabase(table, databaseName)).collect(Collectors.toList()); } /** - * Get all realtime table names from default database. + * Get all realtime table names from all databases. * - * @return List of realtime table names in default database + * @return List of realtime table names */ public List getAllRealtimeTables() { - return getAllRealtimeTables(null); + return getAllResources().stream().filter(TableNameBuilder::isRealtimeTableResource).collect(Collectors.toList()); } /** @@ -805,23 +793,19 @@ public List getAllRealtimeTables() { * @return List of realtime table names in provided database name */ public List getAllRealtimeTables(@Nullable String databaseName) { - List realtimeTableNames = new ArrayList<>(); - for (String resourceName : getAllResources()) { - if (DatabaseUtils.isPartOfDatabase(resourceName, databaseName) - && TableNameBuilder.isRealtimeTableResource(resourceName)) { - realtimeTableNames.add(resourceName); - } - } - return realtimeTableNames; + return getAllResources().stream().filter( + resourceName -> TableNameBuilder.isRealtimeTableResource(resourceName) && DatabaseUtils.isPartOfDatabase( + resourceName, databaseName)).collect(Collectors.toList()); } /** - * Get all raw table names in default database. + * Get all raw table names in all databases. * - * @return List of raw table names in default database + * @return List of raw table names */ public List getAllRawTables() { - return getAllRawTables(null); + return getAllResources().stream().filter(TableNameBuilder::isTableResource) + .map(TableNameBuilder::extractRawTableName).distinct().collect(Collectors.toList()); } /** @@ -831,14 +815,9 @@ public List getAllRawTables() { * @return List of raw table names in provided database name */ public List getAllRawTables(@Nullable String databaseName) { - Set rawTableNames = new HashSet<>(); - for (String resourceName : getAllResources()) { - if (TableNameBuilder.isTableResource(resourceName) - && DatabaseUtils.isPartOfDatabase(resourceName, databaseName)) { - rawTableNames.add(TableNameBuilder.extractRawTableName(resourceName)); - } - } - return new ArrayList<>(rawTableNames); + return getAllResources().stream().filter( + resourceName -> TableNameBuilder.isTableResource(resourceName) && DatabaseUtils.isPartOfDatabase(resourceName, + databaseName)).map(TableNameBuilder::extractRawTableName).distinct().collect(Collectors.toList()); } /** diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/cleanup/StaleInstancesCleanupTask.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/cleanup/StaleInstancesCleanupTask.java index 027712bde88b..b25746298549 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/cleanup/StaleInstancesCleanupTask.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/cleanup/StaleInstancesCleanupTask.java @@ -138,12 +138,9 @@ private Set getBrokerInstancesInUse() { private Set getServerInstancesInUse() { Set serverInstancesInUse = new HashSet<>(); - _pinotHelixResourceManager.getDatabaseNames().stream() - .map(_pinotHelixResourceManager::getAllTables) - .flatMap(List::stream) - .forEach(tableName -> serverInstancesInUse.addAll( - Optional.ofNullable(_pinotHelixResourceManager.getTableIdealState(tableName)) - .map(is -> is.getInstanceSet(tableName)).orElse(Collections.emptySet()))); + _pinotHelixResourceManager.getAllTables().forEach(tableName -> serverInstancesInUse.addAll( + Optional.ofNullable(_pinotHelixResourceManager.getTableIdealState(tableName)) + .map(is -> is.getInstanceSet(tableName)).orElse(Collections.emptySet()))); return serverInstancesInUse; } } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java index 9b2fced8c28a..2cdbf8c1dfe2 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java @@ -480,13 +480,12 @@ public void registerTaskGenerator(PinotTaskGenerator taskGenerator) { } /** - * Public API to schedule tasks (all task types) for all tables in default database. + * Public API to schedule tasks (all task types) for all tables in all databases. * It might be called from the non-leader controller. * Returns a map from the task type to the list of tasks scheduled. */ - @Deprecated public synchronized Map> scheduleTasks() { - return scheduleTasks(_pinotHelixResourceManager.getAllTables(CommonConstants.DEFAULT_DATABASE), false); + return scheduleTasks(_pinotHelixResourceManager.getAllTables(), false); } /** @@ -494,7 +493,7 @@ public synchronized Map> scheduleTasks() { * It might be called from the non-leader controller. * Returns a map from the task type to the list of tasks scheduled. */ - public synchronized Map> scheduleTasksForDatabase(String database) { + public synchronized Map> scheduleTasksForDatabase(@Nullable String database) { return scheduleTasks(_pinotHelixResourceManager.getAllTables(database), false); } @@ -605,8 +604,8 @@ private List scheduleTask(PinotTaskGenerator taskGenerator, List 0) { @@ -629,14 +628,13 @@ public synchronized Map> scheduleTasks(String tableNameWith } /** - * Public API to schedule task for the given task type in default database. + * Public API to schedule task for the given task type in all databases. * It might be called from the non-leader controller. * Returns the list of task names, or {@code null} if no task is scheduled. */ - @Deprecated @Nullable public synchronized List scheduleTask(String taskType) { - return scheduleTaskForDatabase(taskType, CommonConstants.DEFAULT_DATABASE); + return scheduleTask(taskType, _pinotHelixResourceManager.getAllTables()); } /** @@ -645,13 +643,18 @@ public synchronized List scheduleTask(String taskType) { * Returns the list of task name, or {@code null} if no task is scheduled. */ @Nullable - public synchronized List scheduleTaskForDatabase(String taskType, String database) { + public synchronized List scheduleTaskForDatabase(String taskType, @Nullable String database) { + return scheduleTask(taskType, _pinotHelixResourceManager.getAllTables(database)); + } + + @Nullable + private List scheduleTask(String taskType, List tables) { PinotTaskGenerator taskGenerator = _taskGeneratorRegistry.getTaskGenerator(taskType); Preconditions.checkState(taskGenerator != null, "Task type: %s is not registered", taskType); // Scan all table configs to get the tables with task enabled List enabledTableConfigs = new ArrayList<>(); - for (String tableNameWithType : _pinotHelixResourceManager.getAllTables(database)) { + for (String tableNameWithType : tables) { TableConfig tableConfig = _pinotHelixResourceManager.getTableConfig(tableNameWithType); if (tableConfig != null && tableConfig.getTaskConfig() != null && tableConfig.getTaskConfig() .isTaskTypeEnabled(taskType)) { diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTask.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTask.java index 47dc218f43e5..d3584df0682e 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTask.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTask.java @@ -20,7 +20,6 @@ import com.google.common.collect.Sets; import java.util.ArrayList; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Properties; @@ -69,12 +68,8 @@ protected final void runTask(Properties periodicTaskProperties) { // Check if we have a specific table against which this task needs to be run. String propTableNameWithType = (String) periodicTaskProperties.get(PeriodicTask.PROPERTY_KEY_TABLE_NAME); // Process the tables that are managed by this controller - List allTables = propTableNameWithType == null - ? _pinotHelixResourceManager.getDatabaseNames().stream() - .map(_pinotHelixResourceManager::getAllTables) - .flatMap(List::stream) - .collect(Collectors.toList()) - : Collections.singletonList(propTableNameWithType); + List allTables = + propTableNameWithType != null ? List.of(propTableNameWithType) : _pinotHelixResourceManager.getAllTables(); Set currentLeaderOfTables = allTables.stream() .filter(_leadControllerManager::isLeaderForTable) diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/RealtimeConsumerMonitorTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/RealtimeConsumerMonitorTest.java index 051fd784b685..26928deb3c0f 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/RealtimeConsumerMonitorTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/RealtimeConsumerMonitorTest.java @@ -18,9 +18,6 @@ */ package org.apache.pinot.controller.helix; -import com.google.common.collect.ImmutableMap; -import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.TreeMap; @@ -43,15 +40,14 @@ import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.builder.TableConfigBuilder; import org.apache.pinot.spi.utils.builder.TableNameBuilder; -import org.testng.Assert; import org.testng.annotations.Test; -import static org.apache.pinot.spi.utils.CommonConstants.DEFAULT_DATABASE; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; public class RealtimeConsumerMonitorTest { @@ -59,17 +55,16 @@ public class RealtimeConsumerMonitorTest { @Test public void realtimeBasicTest() throws Exception { - final String tableName = "myTable_REALTIME"; - final String rawTableName = TableNameBuilder.extractRawTableName(tableName); - List allTableNames = new ArrayList(); - allTableNames.add(tableName); + String rawTableName = "myTable"; + String realtimeTableName = TableNameBuilder.REALTIME.tableNameWithType(rawTableName); TableConfig tableConfig = - new TableConfigBuilder(TableType.REALTIME).setTableName(tableName).setTimeColumnName("timeColumn") + new TableConfigBuilder(TableType.REALTIME).setTableName(rawTableName).setTimeColumnName("timeColumn") .setNumReplicas(2).setStreamConfigs(getStreamConfigMap()).build(); + LLCSegmentName segmentPartition1Seq0 = new LLCSegmentName(rawTableName, 1, 0, System.currentTimeMillis()); LLCSegmentName segmentPartition1Seq1 = new LLCSegmentName(rawTableName, 1, 1, System.currentTimeMillis()); LLCSegmentName segmentPartition2Seq0 = new LLCSegmentName(rawTableName, 2, 0, System.currentTimeMillis()); - IdealState idealState = new IdealState(tableName); + IdealState idealState = new IdealState(realtimeTableName); idealState.setPartitionState(segmentPartition1Seq0.getSegmentName(), "pinot1", "ONLINE"); idealState.setPartitionState(segmentPartition1Seq0.getSegmentName(), "pinot2", "ONLINE"); idealState.setPartitionState(segmentPartition1Seq1.getSegmentName(), "pinot1", "CONSUMING"); @@ -79,7 +74,7 @@ public void realtimeBasicTest() idealState.setReplicas("3"); idealState.setRebalanceMode(IdealState.RebalanceMode.CUSTOMIZED); - ExternalView externalView = new ExternalView(tableName); + ExternalView externalView = new ExternalView(realtimeTableName); externalView.setState(segmentPartition1Seq0.getSegmentName(), "pinot1", "ONLINE"); externalView.setState(segmentPartition1Seq0.getSegmentName(), "pinot2", "ONLINE"); externalView.setState(segmentPartition1Seq1.getSegmentName(), "pinot1", "CONSUMING"); @@ -91,13 +86,11 @@ public void realtimeBasicTest() { helixResourceManager = mock(PinotHelixResourceManager.class); ZkHelixPropertyStore helixPropertyStore = mock(ZkHelixPropertyStore.class); - when(helixResourceManager.getTableConfig(tableName)).thenReturn(tableConfig); + when(helixResourceManager.getTableConfig(realtimeTableName)).thenReturn(tableConfig); when(helixResourceManager.getPropertyStore()).thenReturn(helixPropertyStore); - when(helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(helixResourceManager.getTableIdealState(tableName)).thenReturn(idealState); - when(helixResourceManager.getTableExternalView(tableName)).thenReturn(externalView); + when(helixResourceManager.getAllTables()).thenReturn(List.of(realtimeTableName)); + when(helixResourceManager.getTableIdealState(realtimeTableName)).thenReturn(idealState); + when(helixResourceManager.getTableExternalView(realtimeTableName)).thenReturn(externalView); ZNRecord znRecord = new ZNRecord("0"); znRecord.setSimpleField(CommonConstants.Segment.Realtime.END_OFFSET, "10000"); when(helixPropertyStore.get(anyString(), any(), anyInt())).thenReturn(znRecord); @@ -121,61 +114,53 @@ public void realtimeBasicTest() // So, the consumer monitor should show: 1. partition-1 has 0 lag; partition-2 has some non-zero lag. // Segment 1 in replicas: TreeMap> response = new TreeMap<>(); - List part1ServerConsumingSegmentInfo = new ArrayList<>(2); - part1ServerConsumingSegmentInfo.add( - getConsumingSegmentInfoForServer("pinot1", "1", "100", "100", "0")); - part1ServerConsumingSegmentInfo.add( - getConsumingSegmentInfoForServer("pinot2", "1", "100", "100", "0")); - + List part1ServerConsumingSegmentInfo = + List.of(getConsumingSegmentInfoForServer("pinot1", "1", "100", "100", "0"), + getConsumingSegmentInfoForServer("pinot2", "1", "100", "100", "0")); response.put(segmentPartition1Seq1.getSegmentName(), part1ServerConsumingSegmentInfo); // Segment 2 in replicas - List part2ServerConsumingSegmentInfo = new ArrayList<>(2); - part2ServerConsumingSegmentInfo.add( - getConsumingSegmentInfoForServer("pinot1", "2", "120", "120", "0")); - part2ServerConsumingSegmentInfo.add( - getConsumingSegmentInfoForServer("pinot2", "2", "80", "120", "60000")); - + List part2ServerConsumingSegmentInfo = + List.of(getConsumingSegmentInfoForServer("pinot1", "2", "120", "120", "0"), + getConsumingSegmentInfoForServer("pinot2", "2", "80", "120", "60000")); response.put(segmentPartition2Seq0.getSegmentName(), part2ServerConsumingSegmentInfo); ConsumingSegmentInfoReader consumingSegmentReader = mock(ConsumingSegmentInfoReader.class); - when(consumingSegmentReader.getConsumingSegmentsInfo(tableName, 10000)) - .thenReturn(new ConsumingSegmentInfoReader.ConsumingSegmentsInfoMap(response, 0, 0)); + when(consumingSegmentReader.getConsumingSegmentsInfo(realtimeTableName, 10000)).thenReturn( + new ConsumingSegmentInfoReader.ConsumingSegmentsInfoMap(response, 0, 0)); RealtimeConsumerMonitor realtimeConsumerMonitor = - new RealtimeConsumerMonitor(config, helixResourceManager, leadControllerManager, - controllerMetrics, consumingSegmentReader); + new RealtimeConsumerMonitor(config, helixResourceManager, leadControllerManager, controllerMetrics, + consumingSegmentReader); realtimeConsumerMonitor.start(); realtimeConsumerMonitor.run(); - Assert.assertEquals(MetricValueUtils.getPartitionGaugeValue(controllerMetrics, tableName, 1, + + assertEquals(MetricValueUtils.getPartitionGaugeValue(controllerMetrics, realtimeTableName, 1, ControllerGauge.MAX_RECORDS_LAG), 0); - Assert.assertEquals(MetricValueUtils.getPartitionGaugeValue(controllerMetrics, tableName, 2, + assertEquals(MetricValueUtils.getPartitionGaugeValue(controllerMetrics, realtimeTableName, 2, ControllerGauge.MAX_RECORDS_LAG), 40); - Assert.assertEquals(MetricValueUtils.getPartitionGaugeValue(controllerMetrics, tableName, 1, - ControllerGauge.MAX_RECORD_AVAILABILITY_LAG_MS), 0); - Assert.assertEquals(MetricValueUtils.getPartitionGaugeValue(controllerMetrics, tableName, 2, + assertEquals(MetricValueUtils.getPartitionGaugeValue(controllerMetrics, realtimeTableName, 1, + ControllerGauge.MAX_RECORD_AVAILABILITY_LAG_MS), 0); + assertEquals(MetricValueUtils.getPartitionGaugeValue(controllerMetrics, realtimeTableName, 2, ControllerGauge.MAX_RECORD_AVAILABILITY_LAG_MS), 60000); } ConsumingSegmentInfoReader.ConsumingSegmentInfo getConsumingSegmentInfoForServer(String serverName, String partitionId, String currentOffset, String upstreamLatestOffset, String availabilityLagMs) { - Map currentOffsetMap = Collections.singletonMap(partitionId, currentOffset); - Map latestUpstreamOffsetMap = Collections.singletonMap(partitionId, upstreamLatestOffset); - Map recordsLagMap = Collections.singletonMap(partitionId, String.valueOf( - Long.parseLong(upstreamLatestOffset) - Long.parseLong(currentOffset))); - Map availabilityLagMsMap = Collections.singletonMap(partitionId, availabilityLagMs); + Map currentOffsetMap = Map.of(partitionId, currentOffset); + Map latestUpstreamOffsetMap = Map.of(partitionId, upstreamLatestOffset); + Map recordsLagMap = + Map.of(partitionId, String.valueOf(Long.parseLong(upstreamLatestOffset) - Long.parseLong(currentOffset))); + Map availabilityLagMsMap = Map.of(partitionId, availabilityLagMs); ConsumingSegmentInfoReader.PartitionOffsetInfo partitionOffsetInfo = new ConsumingSegmentInfoReader.PartitionOffsetInfo(currentOffsetMap, latestUpstreamOffsetMap, recordsLagMap, availabilityLagMsMap); - return new ConsumingSegmentInfoReader.ConsumingSegmentInfo(serverName, "CONSUMING", -1, - currentOffsetMap, partitionOffsetInfo); + return new ConsumingSegmentInfoReader.ConsumingSegmentInfo(serverName, "CONSUMING", -1, currentOffsetMap, + partitionOffsetInfo); } Map getStreamConfigMap() { - return ImmutableMap.of( - "streamType", "kafka", - "stream.kafka.consumer.type", "simple", - "stream.kafka.topic.name", "test", + return Map.of("streamType", "kafka", "stream.kafka.consumer.type", "simple", "stream.kafka.topic.name", "test", "stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaAvroMessageDecoder", "stream.kafka.consumer.factory.class.name", "org.apache.pinot.core.realtime.impl.fakestream.FakeStreamConsumerFactory"); diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/SegmentStatusCheckerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/SegmentStatusCheckerTest.java index a1dd8f2697b0..3161c9da200a 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/SegmentStatusCheckerTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/SegmentStatusCheckerTest.java @@ -18,10 +18,6 @@ */ package org.apache.pinot.controller.helix; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; @@ -52,19 +48,21 @@ import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.builder.TableConfigBuilder; import org.apache.pinot.spi.utils.builder.TableNameBuilder; -import org.testng.Assert; import org.testng.annotations.Test; -import static org.apache.pinot.spi.utils.CommonConstants.DEFAULT_DATABASE; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; public class SegmentStatusCheckerTest { + private final ExecutorService _executorService = Executors.newFixedThreadPool(1); + private SegmentStatusChecker _segmentStatusChecker; private PinotHelixResourceManager _helixResourceManager; private ZkHelixPropertyStore _helixPropertyStore; @@ -73,18 +71,15 @@ public class SegmentStatusCheckerTest { private ControllerMetrics _controllerMetrics; private ControllerConf _config; private TableSizeReader _tableSizeReader; - private ExecutorService _executorService = Executors.newFixedThreadPool(1); @Test public void offlineBasicTest() throws Exception { - final String tableName = "myTable_OFFLINE"; - List allTableNames = new ArrayList(); - allTableNames.add(tableName); + String offlineTableName = "myTable_OFFLINE"; TableConfig tableConfig = - new TableConfigBuilder(TableType.OFFLINE).setTableName(tableName).setNumReplicas(2).build(); + new TableConfigBuilder(TableType.OFFLINE).setTableName(offlineTableName).setNumReplicas(2).build(); - IdealState idealState = new IdealState(tableName); + IdealState idealState = new IdealState(offlineTableName); idealState.setPartitionState("myTable_0", "pinot1", "ONLINE"); idealState.setPartitionState("myTable_0", "pinot2", "ONLINE"); idealState.setPartitionState("myTable_0", "pinot3", "ONLINE"); @@ -101,7 +96,7 @@ public void offlineBasicTest() idealState.setReplicas("2"); idealState.setRebalanceMode(IdealState.RebalanceMode.CUSTOMIZED); - ExternalView externalView = new ExternalView(tableName); + ExternalView externalView = new ExternalView(offlineTableName); externalView.setState("myTable_0", "pinot1", "ONLINE"); externalView.setState("myTable_0", "pinot2", "ONLINE"); externalView.setState("myTable_1", "pinot1", "ERROR"); @@ -114,27 +109,23 @@ public void offlineBasicTest() { _helixResourceManager = mock(PinotHelixResourceManager.class); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(_helixResourceManager.getTableConfig(tableName)).thenReturn(tableConfig); - when(_helixResourceManager.getTableIdealState(tableName)).thenReturn(idealState); - when(_helixResourceManager.getTableExternalView(tableName)).thenReturn(externalView); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(offlineTableName)); + when(_helixResourceManager.getTableConfig(offlineTableName)).thenReturn(tableConfig); + when(_helixResourceManager.getTableIdealState(offlineTableName)).thenReturn(idealState); + when(_helixResourceManager.getTableExternalView(offlineTableName)).thenReturn(externalView); } { _helixPropertyStore = mock(ZkHelixPropertyStore.class); when(_helixResourceManager.getPropertyStore()).thenReturn(_helixPropertyStore); // Based on the lineage entries: {myTable_1 -> myTable_3, COMPLETED}, {myTable_3 -> myTable_4, IN_PROGRESS}, // myTable_1 and myTable_4 will be skipped for the metrics. - SegmentLineage segmentLineage = new SegmentLineage(tableName); + SegmentLineage segmentLineage = new SegmentLineage(offlineTableName); segmentLineage.addLineageEntry(SegmentLineageUtils.generateLineageEntryId(), - new LineageEntry(Collections.singletonList("myTable_1"), Collections.singletonList("myTable_3"), - LineageEntryState.COMPLETED, 11111L)); + new LineageEntry(List.of("myTable_1"), List.of("myTable_3"), LineageEntryState.COMPLETED, 11111L)); segmentLineage.addLineageEntry(SegmentLineageUtils.generateLineageEntryId(), - new LineageEntry(Collections.singletonList("myTable_3"), Collections.singletonList("myTable_4"), - LineageEntryState.IN_PROGRESS, 11111L)); - when(_helixPropertyStore.get(eq("/SEGMENT_LINEAGE/" + tableName), any(), eq(AccessOption.PERSISTENT))) - .thenReturn(segmentLineage.toZNRecord()); + new LineageEntry(List.of("myTable_3"), List.of("myTable_4"), LineageEntryState.IN_PROGRESS, 11111L)); + when(_helixPropertyStore.get(eq("/SEGMENT_LINEAGE/" + offlineTableName), any(), + eq(AccessOption.PERSISTENT))).thenReturn(segmentLineage.toZNRecord()); } { _config = mock(ControllerConf.class); @@ -158,40 +149,41 @@ public void offlineBasicTest() _segmentStatusChecker.setTableSizeReader(_tableSizeReader); _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, + + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, offlineTableName, ControllerGauge.REPLICATION_FROM_CONFIG), 2); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.SEGMENT_COUNT), 3); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.SEGMENT_COUNT_INCLUDING_REPLACED), 5); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.SEGMENTS_IN_ERROR_STATE), 1); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + assertEquals( + MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), ControllerGauge.SEGMENT_COUNT), + 3); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.SEGMENT_COUNT_INCLUDING_REPLACED), 5); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.SEGMENTS_IN_ERROR_STATE), 1); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS), 2); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.NUMBER_OF_REPLICAS), 2); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.PERCENT_OF_REPLICAS), 66); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 100); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.TABLE_COMPRESSED_SIZE), 0); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.NUMBER_OF_REPLICAS), 2); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.PERCENT_OF_REPLICAS), 66); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 100); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.TABLE_COMPRESSED_SIZE), 0); } @Test public void realtimeBasicTest() throws Exception { - final String tableName = "myTable_REALTIME"; - final String rawTableName = TableNameBuilder.extractRawTableName(tableName); - List allTableNames = new ArrayList(); - allTableNames.add(tableName); + String rawTableName = "myTable"; + String realtimeTableName = TableNameBuilder.REALTIME.tableNameWithType(rawTableName); TableConfig tableConfig = - new TableConfigBuilder(TableType.REALTIME).setTableName(tableName).setTimeColumnName("timeColumn") + new TableConfigBuilder(TableType.REALTIME).setTableName(rawTableName).setTimeColumnName("timeColumn") .setNumReplicas(3).setStreamConfigs(getStreamConfigMap()).build(); - final LLCSegmentName seg1 = new LLCSegmentName(rawTableName, 1, 0, System.currentTimeMillis()); - final LLCSegmentName seg2 = new LLCSegmentName(rawTableName, 1, 1, System.currentTimeMillis()); - final LLCSegmentName seg3 = new LLCSegmentName(rawTableName, 2, 1, System.currentTimeMillis()); - IdealState idealState = new IdealState(tableName); + + LLCSegmentName seg1 = new LLCSegmentName(rawTableName, 1, 0, System.currentTimeMillis()); + LLCSegmentName seg2 = new LLCSegmentName(rawTableName, 1, 1, System.currentTimeMillis()); + LLCSegmentName seg3 = new LLCSegmentName(rawTableName, 2, 1, System.currentTimeMillis()); + IdealState idealState = new IdealState(realtimeTableName); idealState.setPartitionState(seg1.getSegmentName(), "pinot1", "ONLINE"); idealState.setPartitionState(seg1.getSegmentName(), "pinot2", "ONLINE"); idealState.setPartitionState(seg1.getSegmentName(), "pinot3", "ONLINE"); @@ -204,7 +196,7 @@ public void realtimeBasicTest() idealState.setReplicas("3"); idealState.setRebalanceMode(IdealState.RebalanceMode.CUSTOMIZED); - ExternalView externalView = new ExternalView(tableName); + ExternalView externalView = new ExternalView(realtimeTableName); externalView.setState(seg1.getSegmentName(), "pinot1", "ONLINE"); externalView.setState(seg1.getSegmentName(), "pinot2", "ONLINE"); externalView.setState(seg1.getSegmentName(), "pinot3", "ONLINE"); @@ -218,13 +210,11 @@ public void realtimeBasicTest() { _helixResourceManager = mock(PinotHelixResourceManager.class); _helixPropertyStore = mock(ZkHelixPropertyStore.class); - when(_helixResourceManager.getTableConfig(tableName)).thenReturn(tableConfig); + when(_helixResourceManager.getTableConfig(realtimeTableName)).thenReturn(tableConfig); when(_helixResourceManager.getPropertyStore()).thenReturn(_helixPropertyStore); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(_helixResourceManager.getTableIdealState(tableName)).thenReturn(idealState); - when(_helixResourceManager.getTableExternalView(tableName)).thenReturn(externalView); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(realtimeTableName)); + when(_helixResourceManager.getTableIdealState(realtimeTableName)).thenReturn(idealState); + when(_helixResourceManager.getTableExternalView(realtimeTableName)).thenReturn(externalView); ZNRecord znRecord = new ZNRecord("0"); znRecord.setSimpleField(CommonConstants.Segment.Realtime.END_OFFSET, "10000"); when(_helixPropertyStore.get(anyString(), any(), anyInt())).thenReturn(znRecord); @@ -251,27 +241,25 @@ public void realtimeBasicTest() _segmentStatusChecker.setTableSizeReader(_tableSizeReader); _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, - ControllerGauge.REPLICATION_FROM_CONFIG), 3); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.SEGMENTS_IN_ERROR_STATE), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, + ControllerGauge.REPLICATION_FROM_CONFIG), 3); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.SEGMENTS_IN_ERROR_STATE), 0); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.NUMBER_OF_REPLICAS), 3); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.PERCENT_OF_REPLICAS), 100); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 100); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.MISSING_CONSUMING_SEGMENT_TOTAL_COUNT), 2); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.NUMBER_OF_REPLICAS), 3); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.PERCENT_OF_REPLICAS), 100); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 100); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.MISSING_CONSUMING_SEGMENT_TOTAL_COUNT), 2); } Map getStreamConfigMap() { - return ImmutableMap.of( - "streamType", "kafka", - "stream.kafka.consumer.type", "simple", - "stream.kafka.topic.name", "test", + return Map.of("streamType", "kafka", "stream.kafka.consumer.type", "simple", "stream.kafka.topic.name", "test", "stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaAvroMessageDecoder", "stream.kafka.consumer.factory.class.name", "org.apache.pinot.core.realtime.impl.fakestream.FakeStreamConsumerFactory"); @@ -281,8 +269,7 @@ Map getStreamConfigMap() { public void missingEVPartitionTest() throws Exception { String offlineTableName = "myTable_OFFLINE"; - List allTableNames = new ArrayList(); - allTableNames.add(offlineTableName); + IdealState idealState = new IdealState(offlineTableName); idealState.setPartitionState("myTable_0", "pinot1", "ONLINE"); idealState.setPartitionState("myTable_0", "pinot2", "ONLINE"); @@ -317,21 +304,19 @@ public void missingEVPartitionTest() ZkHelixPropertyStore propertyStore; { propertyStore = (ZkHelixPropertyStore) mock(ZkHelixPropertyStore.class); - when(propertyStore.get("/SEGMENTS/myTable_OFFLINE/myTable_3", null, AccessOption.PERSISTENT)) - .thenReturn(znrecord); + when(propertyStore.get("/SEGMENTS/myTable_OFFLINE/myTable_3", null, AccessOption.PERSISTENT)).thenReturn( + znrecord); } { _helixResourceManager = mock(PinotHelixResourceManager.class); _helixPropertyStore = mock(ZkHelixPropertyStore.class); when(_helixResourceManager.getPropertyStore()).thenReturn(_helixPropertyStore); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(offlineTableName)); when(_helixResourceManager.getTableIdealState(offlineTableName)).thenReturn(idealState); when(_helixResourceManager.getTableExternalView(offlineTableName)).thenReturn(externalView); - when(_helixResourceManager.getSegmentZKMetadata(offlineTableName, "myTable_3")) - .thenReturn(new SegmentZKMetadata(znrecord)); + when(_helixResourceManager.getSegmentZKMetadata(offlineTableName, "myTable_3")).thenReturn( + new SegmentZKMetadata(znrecord)); } { _config = mock(ControllerConf.class); @@ -355,25 +340,25 @@ public void missingEVPartitionTest() _segmentStatusChecker.setTableSizeReader(_tableSizeReader); _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.SEGMENTS_IN_ERROR_STATE), 1); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.SEGMENTS_IN_ERROR_STATE), 1); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS), 2); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.NUMBER_OF_REPLICAS), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 75); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.TABLE_COMPRESSED_SIZE), 1111); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.NUMBER_OF_REPLICAS), 0); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 75); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.TABLE_COMPRESSED_SIZE), 1111); } @Test public void missingEVTest() throws Exception { - final String tableName = "myTable_REALTIME"; - List allTableNames = new ArrayList(); - allTableNames.add(tableName); - IdealState idealState = new IdealState(tableName); + String realtimeTableName = "myTable_REALTIME"; + + IdealState idealState = new IdealState(realtimeTableName); idealState.setPartitionState("myTable_0", "pinot1", "ONLINE"); idealState.setPartitionState("myTable_0", "pinot2", "ONLINE"); idealState.setPartitionState("myTable_0", "pinot3", "ONLINE"); @@ -388,11 +373,9 @@ public void missingEVTest() _helixResourceManager = mock(PinotHelixResourceManager.class); _helixPropertyStore = mock(ZkHelixPropertyStore.class); when(_helixResourceManager.getPropertyStore()).thenReturn(_helixPropertyStore); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(_helixResourceManager.getTableIdealState(tableName)).thenReturn(idealState); - when(_helixResourceManager.getTableExternalView(tableName)).thenReturn(null); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(realtimeTableName)); + when(_helixResourceManager.getTableIdealState(realtimeTableName)).thenReturn(idealState); + when(_helixResourceManager.getTableExternalView(realtimeTableName)).thenReturn(null); } { _config = mock(ControllerConf.class); @@ -416,30 +399,28 @@ public void missingEVTest() _segmentStatusChecker.setTableSizeReader(_tableSizeReader); _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, - ControllerGauge.SEGMENTS_IN_ERROR_STATE), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, + + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, + ControllerGauge.SEGMENTS_IN_ERROR_STATE), 0); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, - ControllerGauge.NUMBER_OF_REPLICAS), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, - ControllerGauge.TABLE_COMPRESSED_SIZE), 0); + assertEquals( + MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, ControllerGauge.NUMBER_OF_REPLICAS), + 0); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, + ControllerGauge.TABLE_COMPRESSED_SIZE), 0); } @Test public void missingIdealTest() throws Exception { - final String tableName = "myTable_REALTIME"; - List allTableNames = new ArrayList<>(); - allTableNames.add(tableName); + String realtimeTableName = "myTable_REALTIME"; { _helixResourceManager = mock(PinotHelixResourceManager.class); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(_helixResourceManager.getTableIdealState(tableName)).thenReturn(null); - when(_helixResourceManager.getTableExternalView(tableName)).thenReturn(null); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(realtimeTableName)); + when(_helixResourceManager.getTableIdealState(realtimeTableName)).thenReturn(null); + when(_helixResourceManager.getTableExternalView(realtimeTableName)).thenReturn(null); } { _config = mock(ControllerConf.class); @@ -463,24 +444,24 @@ public void missingIdealTest() _segmentStatusChecker.setTableSizeReader(_tableSizeReader); _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, tableName, - ControllerGauge.SEGMENTS_IN_ERROR_STATE)); - Assert.assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, tableName, + + assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, realtimeTableName, + ControllerGauge.SEGMENTS_IN_ERROR_STATE)); + assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, realtimeTableName, ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS)); - Assert.assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, tableName, - ControllerGauge.NUMBER_OF_REPLICAS)); - Assert.assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, tableName, - ControllerGauge.PERCENT_OF_REPLICAS)); - Assert.assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, tableName, - ControllerGauge.TABLE_COMPRESSED_SIZE)); + assertFalse( + MetricValueUtils.tableGaugeExists(_controllerMetrics, realtimeTableName, ControllerGauge.NUMBER_OF_REPLICAS)); + assertFalse( + MetricValueUtils.tableGaugeExists(_controllerMetrics, realtimeTableName, ControllerGauge.PERCENT_OF_REPLICAS)); + assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, realtimeTableName, + ControllerGauge.TABLE_COMPRESSED_SIZE)); } @Test public void missingEVPartitionPushTest() throws Exception { String offlineTableName = "myTable_OFFLINE"; - List allTableNames = new ArrayList(); - allTableNames.add(offlineTableName); + IdealState idealState = new IdealState(offlineTableName); idealState.setPartitionState("myTable_0", "pinot1", "ONLINE"); idealState.setPartitionState("myTable_1", "pinot1", "ONLINE"); @@ -527,15 +508,13 @@ public void missingEVPartitionPushTest() _helixResourceManager = mock(PinotHelixResourceManager.class); _helixPropertyStore = mock(ZkHelixPropertyStore.class); when(_helixResourceManager.getPropertyStore()).thenReturn(_helixPropertyStore); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(offlineTableName)); when(_helixResourceManager.getTableIdealState(offlineTableName)).thenReturn(idealState); when(_helixResourceManager.getTableExternalView(offlineTableName)).thenReturn(externalView); - when(_helixResourceManager.getSegmentZKMetadata(offlineTableName, "myTable_0")) - .thenReturn(new SegmentZKMetadata(znrecord)); - when(_helixResourceManager.getSegmentZKMetadata(offlineTableName, "myTable_2")) - .thenReturn(new SegmentZKMetadata(znrecord2)); + when(_helixResourceManager.getSegmentZKMetadata(offlineTableName, "myTable_0")).thenReturn( + new SegmentZKMetadata(znrecord)); + when(_helixResourceManager.getSegmentZKMetadata(offlineTableName, "myTable_2")).thenReturn( + new SegmentZKMetadata(znrecord2)); } { _config = mock(ControllerConf.class); @@ -559,27 +538,27 @@ public void missingEVPartitionPushTest() _segmentStatusChecker.setTableSizeReader(_tableSizeReader); _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.SEGMENTS_IN_ERROR_STATE), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.SEGMENTS_IN_ERROR_STATE), 0); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.NUMBER_OF_REPLICAS), 2); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.PERCENT_OF_REPLICAS), 100); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 100); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.TABLE_COMPRESSED_SIZE), 0); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.NUMBER_OF_REPLICAS), 2); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.PERCENT_OF_REPLICAS), 100); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 100); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.TABLE_COMPRESSED_SIZE), 0); } @Test public void noReplicas() throws Exception { - final String tableName = "myTable_REALTIME"; - List allTableNames = new ArrayList(); - allTableNames.add(tableName); - IdealState idealState = new IdealState(tableName); + String realtimeTableName = "myTable_REALTIME"; + + IdealState idealState = new IdealState(realtimeTableName); idealState.setPartitionState("myTable_0", "pinot1", "OFFLINE"); idealState.setPartitionState("myTable_0", "pinot2", "OFFLINE"); idealState.setPartitionState("myTable_0", "pinot3", "OFFLINE"); @@ -590,11 +569,9 @@ public void noReplicas() _helixResourceManager = mock(PinotHelixResourceManager.class); _helixPropertyStore = mock(ZkHelixPropertyStore.class); when(_helixResourceManager.getPropertyStore()).thenReturn(_helixPropertyStore); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(_helixResourceManager.getTableIdealState(tableName)).thenReturn(idealState); - when(_helixResourceManager.getTableExternalView(tableName)).thenReturn(null); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(realtimeTableName)); + when(_helixResourceManager.getTableIdealState(realtimeTableName)).thenReturn(idealState); + when(_helixResourceManager.getTableExternalView(realtimeTableName)).thenReturn(null); } { _config = mock(ControllerConf.class); @@ -618,26 +595,26 @@ public void noReplicas() _segmentStatusChecker.setTableSizeReader(_tableSizeReader); _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, + + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, ControllerGauge.SEGMENTS_IN_ERROR_STATE), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS), 0); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, - ControllerGauge.NUMBER_OF_REPLICAS), 1); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, - ControllerGauge.PERCENT_OF_REPLICAS), 100); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, - ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 100); + assertEquals( + MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, ControllerGauge.NUMBER_OF_REPLICAS), + 1); + assertEquals( + MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, ControllerGauge.PERCENT_OF_REPLICAS), + 100); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, + ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 100); } @Test - public void disabledTableTest() - throws Exception { + public void disabledTableTest() { + String offlineTableName = "myTable_OFFLINE"; - final String tableName = "myTable_OFFLINE"; - List allTableNames = new ArrayList(); - allTableNames.add(tableName); - IdealState idealState = new IdealState(tableName); + IdealState idealState = new IdealState(offlineTableName); // disable table in idealstate idealState.enable(false); idealState.setPartitionState("myTable_OFFLINE", "pinot1", "OFFLINE"); @@ -648,11 +625,9 @@ public void disabledTableTest() { _helixResourceManager = mock(PinotHelixResourceManager.class); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(_helixResourceManager.getTableIdealState(tableName)).thenReturn(idealState); - when(_helixResourceManager.getTableExternalView(tableName)).thenReturn(null); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(offlineTableName)); + when(_helixResourceManager.getTableIdealState(offlineTableName)).thenReturn(idealState); + when(_helixResourceManager.getTableExternalView(offlineTableName)).thenReturn(null); } { _config = mock(ControllerConf.class); @@ -669,23 +644,21 @@ public void disabledTableTest() _segmentStatusChecker = new SegmentStatusChecker(_helixResourceManager, _leadControllerManager, _config, _controllerMetrics, _executorService); + // verify state before test - Assert.assertEquals( - MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, ControllerGauge.DISABLED_TABLE_COUNT), 0); + assertEquals(MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, ControllerGauge.DISABLED_TABLE_COUNT), 0); + // update metrics _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertEquals( - MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, ControllerGauge.DISABLED_TABLE_COUNT), 1); + assertEquals(MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, ControllerGauge.DISABLED_TABLE_COUNT), 1); } @Test - public void disabledEmptyTableTest() - throws Exception { + public void disabledEmptyTableTest() { + String offlineTableName = "myTable_OFFLINE"; - final String tableName = "myTable_OFFLINE"; - List allTableNames = Lists.newArrayList(tableName); - IdealState idealState = new IdealState(tableName); + IdealState idealState = new IdealState(offlineTableName); // disable table in idealstate idealState.enable(false); idealState.setReplicas("1"); @@ -693,11 +666,9 @@ public void disabledEmptyTableTest() { _helixResourceManager = mock(PinotHelixResourceManager.class); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(_helixResourceManager.getTableIdealState(tableName)).thenReturn(idealState); - when(_helixResourceManager.getTableExternalView(tableName)).thenReturn(null); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(offlineTableName)); + when(_helixResourceManager.getTableIdealState(offlineTableName)).thenReturn(idealState); + when(_helixResourceManager.getTableExternalView(offlineTableName)).thenReturn(null); } { _config = mock(ControllerConf.class); @@ -714,14 +685,14 @@ public void disabledEmptyTableTest() _segmentStatusChecker = new SegmentStatusChecker(_helixResourceManager, _leadControllerManager, _config, _controllerMetrics, _executorService); + // verify state before test - Assert.assertFalse( - MetricValueUtils.globalGaugeExists(_controllerMetrics, ControllerGauge.DISABLED_TABLE_COUNT)); + assertFalse(MetricValueUtils.globalGaugeExists(_controllerMetrics, ControllerGauge.DISABLED_TABLE_COUNT)); + // update metrics _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertEquals( - MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, ControllerGauge.DISABLED_TABLE_COUNT), 1); + assertEquals(MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, ControllerGauge.DISABLED_TABLE_COUNT), 1); } @Test @@ -734,22 +705,20 @@ public void noSegments() @Test public void lessThanOnePercentSegmentsUnavailableTest() - throws Exception { - String tableName = "myTable_OFFLINE"; - int numSegments = 200; - List allTableNames = new ArrayList(); - allTableNames.add(tableName); + throws Exception { + String offlineTableName = "myTable_OFFLINE"; TableConfig tableConfig = - new TableConfigBuilder(TableType.OFFLINE).setTableName(tableName).setNumReplicas(1).build(); + new TableConfigBuilder(TableType.OFFLINE).setTableName(offlineTableName).setNumReplicas(1).build(); - IdealState idealState = new IdealState(tableName); + IdealState idealState = new IdealState(offlineTableName); + int numSegments = 200; for (int i = 0; i < numSegments; i++) { idealState.setPartitionState("myTable_" + i, "pinot1", "ONLINE"); } idealState.setReplicas("1"); idealState.setRebalanceMode(IdealState.RebalanceMode.CUSTOMIZED); - ExternalView externalView = new ExternalView(tableName); + ExternalView externalView = new ExternalView(offlineTableName); externalView.setState("myTable_0", "pinot1", "OFFLINE"); for (int i = 1; i < numSegments; i++) { externalView.setState("myTable_" + i, "pinot1", "ONLINE"); @@ -757,19 +726,17 @@ public void lessThanOnePercentSegmentsUnavailableTest() { _helixResourceManager = mock(PinotHelixResourceManager.class); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(_helixResourceManager.getTableConfig(tableName)).thenReturn(tableConfig); - when(_helixResourceManager.getTableIdealState(tableName)).thenReturn(idealState); - when(_helixResourceManager.getTableExternalView(tableName)).thenReturn(externalView); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(offlineTableName)); + when(_helixResourceManager.getTableConfig(offlineTableName)).thenReturn(tableConfig); + when(_helixResourceManager.getTableIdealState(offlineTableName)).thenReturn(idealState); + when(_helixResourceManager.getTableExternalView(offlineTableName)).thenReturn(externalView); } { _helixPropertyStore = mock(ZkHelixPropertyStore.class); when(_helixResourceManager.getPropertyStore()).thenReturn(_helixPropertyStore); - SegmentLineage segmentLineage = new SegmentLineage(tableName); - when(_helixPropertyStore.get(eq("/SEGMENT_LINEAGE/" + tableName), any(), eq(AccessOption.PERSISTENT))) - .thenReturn(segmentLineage.toZNRecord()); + SegmentLineage segmentLineage = new SegmentLineage(offlineTableName); + when(_helixPropertyStore.get(eq("/SEGMENT_LINEAGE/" + offlineTableName), any(), + eq(AccessOption.PERSISTENT))).thenReturn(segmentLineage.toZNRecord()); } { _config = mock(ControllerConf.class); @@ -788,37 +755,35 @@ public void lessThanOnePercentSegmentsUnavailableTest() _metricsRegistry = PinotMetricUtils.getPinotMetricsRegistry(); _controllerMetrics = new ControllerMetrics(_metricsRegistry); _segmentStatusChecker = - new SegmentStatusChecker(_helixResourceManager, _leadControllerManager, _config, _controllerMetrics, - _executorService); + new SegmentStatusChecker(_helixResourceManager, _leadControllerManager, _config, _controllerMetrics, + _executorService); _segmentStatusChecker.setTableSizeReader(_tableSizeReader); _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), - ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 99); + + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, externalView.getId(), + ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 99); } public void noSegmentsInternal(final int nReplicas) throws Exception { - final String tableName = "myTable_REALTIME"; + String realtimeTableName = "myTable_REALTIME"; + String nReplicasStr = Integer.toString(nReplicas); int nReplicasExpectedValue = nReplicas; if (nReplicas < 0) { nReplicasStr = "abc"; nReplicasExpectedValue = 1; } - List allTableNames = new ArrayList(); - allTableNames.add(tableName); - IdealState idealState = new IdealState(tableName); + IdealState idealState = new IdealState(realtimeTableName); idealState.setReplicas(nReplicasStr); idealState.setRebalanceMode(IdealState.RebalanceMode.CUSTOMIZED); { _helixResourceManager = mock(PinotHelixResourceManager.class); - when(_helixResourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_helixResourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(allTableNames); - when(_helixResourceManager.getTableIdealState(tableName)).thenReturn(idealState); - when(_helixResourceManager.getTableExternalView(tableName)).thenReturn(null); + when(_helixResourceManager.getAllTables()).thenReturn(List.of(realtimeTableName)); + when(_helixResourceManager.getTableIdealState(realtimeTableName)).thenReturn(idealState); + when(_helixResourceManager.getTableExternalView(realtimeTableName)).thenReturn(null); } { _config = mock(ControllerConf.class); @@ -843,15 +808,17 @@ public void noSegmentsInternal(final int nReplicas) _segmentStatusChecker.start(); _segmentStatusChecker.run(); - Assert.assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, tableName, + assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, realtimeTableName, ControllerGauge.SEGMENTS_IN_ERROR_STATE)); - Assert.assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, tableName, + assertFalse(MetricValueUtils.tableGaugeExists(_controllerMetrics, realtimeTableName, ControllerGauge.SEGMENTS_IN_ERROR_STATE)); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, - ControllerGauge.NUMBER_OF_REPLICAS), nReplicasExpectedValue); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, - ControllerGauge.PERCENT_OF_REPLICAS), 100); - Assert.assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, tableName, + assertEquals( + MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, ControllerGauge.NUMBER_OF_REPLICAS), + nReplicasExpectedValue); + assertEquals( + MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, ControllerGauge.PERCENT_OF_REPLICAS), + 100); + assertEquals(MetricValueUtils.getTableGaugeValue(_controllerMetrics, realtimeTableName, ControllerGauge.PERCENT_SEGMENTS_AVAILABLE), 100); } } diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTaskTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTaskTest.java index 15c3cf6d8197..f4e0eb46b14f 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTaskTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTaskTest.java @@ -19,7 +19,6 @@ package org.apache.pinot.controller.helix.core.periodictask; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Properties; import java.util.concurrent.atomic.AtomicBoolean; @@ -35,7 +34,6 @@ import org.testng.annotations.BeforeTest; import org.testng.annotations.Test; -import static org.apache.pinot.spi.utils.CommonConstants.DEFAULT_DATABASE; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -89,9 +87,7 @@ public void processTable(String tableNameWithType) { public void beforeTest() { List tables = new ArrayList<>(_numTables); IntStream.range(0, _numTables).forEach(i -> tables.add("table_" + i + " _OFFLINE")); - when(_resourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(_resourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(tables); + when(_resourceManager.getAllTables()).thenReturn(tables); when(_leadControllerManager.isLeaderForTable(anyString())).thenReturn(true); } @@ -109,7 +105,6 @@ public void testRandomInitialDelay() { _task.getInitialDelayInSeconds() >= ControllerConf.ControllerPeriodicTasksConf.MIN_INITIAL_DELAY_IN_SECONDS); assertTrue( _task.getInitialDelayInSeconds() < ControllerConf.ControllerPeriodicTasksConf.MAX_INITIAL_DELAY_IN_SECONDS); - assertEquals(_task.getIntervalInSeconds(), RUN_FREQUENCY_IN_SECONDS); } @@ -124,7 +119,7 @@ public void testControllerPeriodicTaskCalls() { assertFalse(_stopTaskCalled.get()); assertTrue(_task.isStarted()); assertEquals(MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, TASK_NAME, - ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), 0); + ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), 0); // Run periodic task with leadership resetState(); @@ -133,8 +128,7 @@ public void testControllerPeriodicTaskCalls() { assertTrue(_processTablesCalled.get()); assertEquals(_tablesProcessed.get(), _numTables); assertEquals(MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, TASK_NAME, - ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), - _numTables); + ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), _numTables); assertFalse(_stopTaskCalled.get()); assertTrue(_task.isStarted()); @@ -145,7 +139,7 @@ public void testControllerPeriodicTaskCalls() { assertFalse(_processTablesCalled.get()); assertEquals(_tablesProcessed.get(), 0); assertEquals(MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, TASK_NAME, - ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), 0); + ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), 0); assertTrue(_stopTaskCalled.get()); assertFalse(_task.isStarted()); @@ -156,7 +150,7 @@ public void testControllerPeriodicTaskCalls() { assertFalse(_processTablesCalled.get()); assertEquals(_tablesProcessed.get(), 0); assertEquals(MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, TASK_NAME, - ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), 0); + ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), 0); assertFalse(_stopTaskCalled.get()); assertFalse(_task.isStarted()); @@ -169,7 +163,7 @@ public void testControllerPeriodicTaskCalls() { assertFalse(_stopTaskCalled.get()); assertTrue(_task.isStarted()); assertEquals(MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, TASK_NAME, - ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), 0); + ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), 0); // Run periodic task with leadership resetState(); @@ -178,7 +172,7 @@ public void testControllerPeriodicTaskCalls() { assertTrue(_processTablesCalled.get()); assertEquals(_tablesProcessed.get(), _numTables); assertEquals(MetricValueUtils.getGlobalGaugeValue(_controllerMetrics, TASK_NAME, - ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), _numTables); + ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED), _numTables); assertFalse(_stopTaskCalled.get()); assertTrue(_task.isStarted()); } diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/retention/RetentionManagerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/retention/RetentionManagerTest.java index ce5e31e5ef13..b3e656de9ead 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/retention/RetentionManagerTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/retention/RetentionManagerTest.java @@ -19,7 +19,6 @@ package org.apache.pinot.controller.helix.core.retention; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -43,15 +42,13 @@ import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.builder.TableConfigBuilder; import org.apache.pinot.spi.utils.builder.TableNameBuilder; -import org.mockito.ArgumentMatchers; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; -import org.testng.Assert; import org.testng.annotations.Test; -import static org.apache.pinot.spi.utils.CommonConstants.DEFAULT_DATABASE; +import static org.mockito.ArgumentMatchers.anyList; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.*; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; public class RetentionManagerTest { @@ -60,8 +57,7 @@ public class RetentionManagerTest { private static final String OFFLINE_TABLE_NAME = TableNameBuilder.OFFLINE.tableNameWithType(TEST_TABLE_NAME); private static final String REALTIME_TABLE_NAME = TableNameBuilder.REALTIME.tableNameWithType(TEST_TABLE_NAME); - private void testDifferentTimeUnits(long pastTimeStamp, TimeUnit timeUnit, long dayAfterTomorrowTimeStamp) - throws Exception { + private void testDifferentTimeUnits(long pastTimeStamp, TimeUnit timeUnit, long dayAfterTomorrowTimeStamp) { List segmentsZKMetadata = new ArrayList<>(); // Create metadata for 10 segments really old, that will be removed by the retention manager. final int numOlderSegments = 10; @@ -105,8 +101,7 @@ private void testDifferentTimeUnits(long pastTimeStamp, TimeUnit timeUnit, long } @Test - public void testRetentionWithMinutes() - throws Exception { + public void testRetentionWithMinutes() { final long theDayAfterTomorrowSinceEpoch = System.currentTimeMillis() / 1000 / 60 / 60 / 24 + 2; final long minutesSinceEpochTimeStamp = theDayAfterTomorrowSinceEpoch * 24 * 60; final long pastMinutesSinceEpoch = 22383360L; @@ -114,8 +109,7 @@ public void testRetentionWithMinutes() } @Test - public void testRetentionWithSeconds() - throws Exception { + public void testRetentionWithSeconds() { final long theDayAfterTomorrowSinceEpoch = System.currentTimeMillis() / 1000 / 60 / 60 / 24 + 2; final long secondsSinceEpochTimeStamp = theDayAfterTomorrowSinceEpoch * 24 * 60 * 60; final long pastSecondsSinceEpoch = 1343001600L; @@ -123,8 +117,7 @@ public void testRetentionWithSeconds() } @Test - public void testRetentionWithMillis() - throws Exception { + public void testRetentionWithMillis() { final long theDayAfterTomorrowSinceEpoch = System.currentTimeMillis() / 1000 / 60 / 60 / 24 + 2; final long millisSinceEpochTimeStamp = theDayAfterTomorrowSinceEpoch * 24 * 60 * 60 * 1000; final long pastMillisSinceEpoch = 1343001600000L; @@ -132,8 +125,7 @@ public void testRetentionWithMillis() } @Test - public void testRetentionWithHours() - throws Exception { + public void testRetentionWithHours() { final long theDayAfterTomorrowSinceEpoch = System.currentTimeMillis() / 1000 / 60 / 60 / 24 + 2; final long hoursSinceEpochTimeStamp = theDayAfterTomorrowSinceEpoch * 24; final long pastHoursSinceEpoch = 373056L; @@ -141,8 +133,7 @@ public void testRetentionWithHours() } @Test - public void testRetentionWithDays() - throws Exception { + public void testRetentionWithDays() { final long daysSinceEpochTimeStamp = System.currentTimeMillis() / 1000 / 60 / 60 / 24 + 2; final long pastDaysSinceEpoch = 15544L; testDifferentTimeUnits(pastDaysSinceEpoch, TimeUnit.DAYS, daysSinceEpochTimeStamp); @@ -161,10 +152,8 @@ private TableConfig createRealtimeTableConfig1(int replicaCount) { private void setupPinotHelixResourceManager(TableConfig tableConfig, final List removedSegments, PinotHelixResourceManager resourceManager, LeadControllerManager leadControllerManager) { - final String tableNameWithType = tableConfig.getTableName(); - when(resourceManager.getDatabaseNames()) - .thenReturn(Collections.singletonList(DEFAULT_DATABASE)); - when(resourceManager.getAllTables(DEFAULT_DATABASE)).thenReturn(Collections.singletonList(tableNameWithType)); + String tableNameWithType = tableConfig.getTableName(); + when(resourceManager.getAllTables()).thenReturn(List.of(tableNameWithType)); ZkHelixPropertyStore propertyStore = mock(ZkHelixPropertyStore.class); when(resourceManager.getPropertyStore()).thenReturn(propertyStore); @@ -172,38 +161,27 @@ private void setupPinotHelixResourceManager(TableConfig tableConfig, final List< SegmentDeletionManager deletionManager = mock(SegmentDeletionManager.class); // Ignore the call to SegmentDeletionManager.removeAgedDeletedSegments. we only test that the call is made once per // run of the retention manager - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocationOnMock) - throws Throwable { - return null; - } - }).when(deletionManager).removeAgedDeletedSegments(leadControllerManager); + doAnswer(invocationOnMock -> null).when(deletionManager).removeAgedDeletedSegments(leadControllerManager); when(resourceManager.getSegmentDeletionManager()).thenReturn(deletionManager); // If and when PinotHelixResourceManager.deleteSegments() is invoked, make sure that the segments deleted // are exactly the same as the ones we expect to be deleted. - doAnswer(new Answer() { - @Override - public Object answer(InvocationOnMock invocationOnMock) - throws Throwable { - Object[] args = invocationOnMock.getArguments(); - String tableNameArg = (String) args[0]; - Assert.assertEquals(tableNameArg, tableNameWithType); - List segmentListArg = (List) args[1]; - Assert.assertEquals(segmentListArg.size(), removedSegments.size()); - for (String segmentName : removedSegments) { - Assert.assertTrue(segmentListArg.contains(segmentName)); - } - return null; + doAnswer(invocationOnMock -> { + Object[] args = invocationOnMock.getArguments(); + String tableNameArg = (String) args[0]; + assertEquals(tableNameArg, tableNameWithType); + List segmentListArg = (List) args[1]; + assertEquals(segmentListArg.size(), removedSegments.size()); + for (String segmentName : removedSegments) { + assertTrue(segmentListArg.contains(segmentName)); } - }).when(resourceManager).deleteSegments(anyString(), ArgumentMatchers.anyList()); + return null; + }).when(resourceManager).deleteSegments(anyString(), anyList()); } // This test makes sure that we clean up the segments marked OFFLINE in realtime for more than 7 days @Test - public void testRealtimeLLCCleanup() - throws Exception { + public void testRealtimeLLCCleanup() { final int initialNumSegments = 8; final long now = System.currentTimeMillis(); @@ -237,8 +215,7 @@ public void testRealtimeLLCCleanup() // This test makes sure that we do not clean up last llc completed segments @Test - public void testRealtimeLastLLCCleanup() - throws Exception { + public void testRealtimeLastLLCCleanup() { final long now = System.currentTimeMillis(); final int replicaCount = 1; From c7ef7a9b023de4f8a03d556e160468b3efa48411 Mon Sep 17 00:00:00 2001 From: "Xiaotian (Jackie) Jiang" <17555551+Jackie-Jiang@users.noreply.github.com> Date: Mon, 8 Apr 2024 10:43:58 -0700 Subject: [PATCH 05/72] Make KafkaConsumerFactory method less restrictive to avoid incompatibility (#12815) --- .../tests/LLCRealtimeClusterIntegrationTest.java | 3 ++- .../pinot/plugin/stream/kafka20/KafkaConsumerFactory.java | 8 +++++--- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java index 56d418a08013..0cfbc51dbf18 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java @@ -56,6 +56,7 @@ import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.env.PinotConfiguration; +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.StreamConfigProperties; @@ -464,7 +465,7 @@ public static void init(String helixClusterName, HelixAdmin helixAdmin, String t } @Override - public KafkaPartitionLevelConsumer createPartitionGroupConsumer(String clientId, + public PartitionGroupConsumer createPartitionGroupConsumer(String clientId, PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) { /* * The segment data manager is creating a consumer to consume rows into a segment. diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaConsumerFactory.java b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaConsumerFactory.java index aeae590bdeb6..c117fe551656 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaConsumerFactory.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaConsumerFactory.java @@ -18,24 +18,26 @@ */ package org.apache.pinot.plugin.stream.kafka20; +import org.apache.pinot.spi.stream.PartitionGroupConsumer; import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; import org.apache.pinot.spi.stream.StreamConsumerFactory; +import org.apache.pinot.spi.stream.StreamMetadataProvider; public class KafkaConsumerFactory extends StreamConsumerFactory { @Override - public KafkaStreamMetadataProvider createPartitionMetadataProvider(String clientId, int partition) { + public StreamMetadataProvider createPartitionMetadataProvider(String clientId, int partition) { return new KafkaStreamMetadataProvider(clientId, _streamConfig, partition); } @Override - public KafkaStreamMetadataProvider createStreamMetadataProvider(String clientId) { + public StreamMetadataProvider createStreamMetadataProvider(String clientId) { return new KafkaStreamMetadataProvider(clientId, _streamConfig); } @Override - public KafkaPartitionLevelConsumer createPartitionGroupConsumer(String clientId, + public PartitionGroupConsumer createPartitionGroupConsumer(String clientId, PartitionGroupConsumptionStatus partitionGroupConsumptionStatus) { return new KafkaPartitionLevelConsumer(clientId, _streamConfig, partitionGroupConsumptionStatus.getPartitionGroupId()); From a07cd63b57f0a7991524570a9c6187cdfb006184 Mon Sep 17 00:00:00 2001 From: sullis Date: Mon, 8 Apr 2024 11:39:34 -0700 Subject: [PATCH 06/72] set dependabot open-pull-requests-limit (#12798) --- .github/dependabot.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/dependabot.yml b/.github/dependabot.yml index d135c26b02cc..9719e1a53392 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -24,6 +24,7 @@ updates: directory: "/" schedule: interval: "daily" + open-pull-requests-limit: 20 - package-ecosystem: "npm" directory: "/pinot-controller/src/main/resources" From 650e2584b13f73443c26592d4a7f3f21fd03bec8 Mon Sep 17 00:00:00 2001 From: "Xiaotian (Jackie) Jiang" <17555551+Jackie-Jiang@users.noreply.github.com> Date: Mon, 8 Apr 2024 12:02:38 -0700 Subject: [PATCH 07/72] [Minor] Add Nullable annotation to HttpHeaders in BrokerRequestHandler (#12816) --- .../requesthandler/BaseBrokerRequestHandler.java | 13 +++++++------ .../broker/requesthandler/BrokerRequestHandler.java | 4 ++-- .../BrokerRequestHandlerDelegate.java | 12 ++++++------ 3 files changed, 15 insertions(+), 14 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java index 31bae2215aac..b2d4e24d3f32 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java @@ -257,7 +257,7 @@ public boolean cancelQuery(long requestId, int timeoutMs, Executor executor, Htt @Override public BrokerResponse handleRequest(JsonNode request, @Nullable SqlNodeAndOptions sqlNodeAndOptions, - @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, HttpHeaders httpHeaders) + @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, @Nullable HttpHeaders httpHeaders) throws Exception { requestContext.setRequestArrivalTimeMillis(System.currentTimeMillis()); @@ -307,7 +307,7 @@ public BrokerResponse handleRequest(JsonNode request, @Nullable SqlNodeAndOption protected BrokerResponse handleRequest(long requestId, String query, @Nullable SqlNodeAndOptions sqlNodeAndOptions, JsonNode request, @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, - HttpHeaders httpHeaders) + @Nullable HttpHeaders httpHeaders) throws Exception { LOGGER.debug("SQL query for request {}: {}", requestId, query); @@ -378,8 +378,9 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S boolean ignoreCase = _tableCache.isIgnoreCase(); String tableName; try { - tableName = getActualTableName( - DatabaseUtils.translateTableName(dataSource.getTableName(), httpHeaders, ignoreCase), _tableCache); + tableName = + getActualTableName(DatabaseUtils.translateTableName(dataSource.getTableName(), httpHeaders, ignoreCase), + _tableCache); } catch (DatabaseConflictException e) { LOGGER.info("{}. Request {}: {}", e.getMessage(), requestId, query); _brokerMetrics.addMeteredGlobalValue(BrokerMeter.QUERY_VALIDATION_EXCEPTIONS, 1); @@ -912,7 +913,7 @@ private String getServerTenant(String tableNameWithType) { *

Currently only supports subquery within the filter. */ private void handleSubquery(PinotQuery pinotQuery, long requestId, JsonNode jsonRequest, - @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, HttpHeaders httpHeaders) + @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, @Nullable HttpHeaders httpHeaders) throws Exception { Expression filterExpression = pinotQuery.getFilterExpression(); if (filterExpression != null) { @@ -928,7 +929,7 @@ private void handleSubquery(PinotQuery pinotQuery, long requestId, JsonNode json * IN_ID_SET transform function. */ private void handleSubquery(Expression expression, long requestId, JsonNode jsonRequest, - @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, HttpHeaders httpHeaders) + @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, @Nullable HttpHeaders httpHeaders) throws Exception { Function function = expression.getFunctionCall(); if (function == null) { diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestHandler.java index 99df35a09ffa..4aa7b26eaab4 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestHandler.java @@ -39,11 +39,11 @@ public interface BrokerRequestHandler { void shutDown(); BrokerResponse handleRequest(JsonNode request, @Nullable SqlNodeAndOptions sqlNodeAndOptions, - @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, HttpHeaders httpHeaders) + @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, @Nullable HttpHeaders httpHeaders) throws Exception; default BrokerResponse handleRequest(JsonNode request, @Nullable RequesterIdentity requesterIdentity, - RequestContext requestContext, HttpHeaders httpHeaders) + RequestContext requestContext, @Nullable HttpHeaders httpHeaders) throws Exception { return handleRequest(request, null, requesterIdentity, requestContext, httpHeaders); } diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestHandlerDelegate.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestHandlerDelegate.java index fb87f715c185..0da84e6eadc4 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestHandlerDelegate.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestHandlerDelegate.java @@ -82,7 +82,7 @@ public void shutDown() { @Override public BrokerResponse handleRequest(JsonNode request, @Nullable SqlNodeAndOptions sqlNodeAndOptions, - @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, HttpHeaders httpHeaders) + @Nullable RequesterIdentity requesterIdentity, RequestContext requestContext, @Nullable HttpHeaders httpHeaders) throws Exception { requestContext.setBrokerId(_brokerId); if (sqlNodeAndOptions == null) { @@ -96,13 +96,13 @@ public BrokerResponse handleRequest(JsonNode request, @Nullable SqlNodeAndOption } } if (request.has(CommonConstants.Broker.Request.QUERY_OPTIONS)) { - sqlNodeAndOptions.setExtraOptions(RequestUtils.getOptionsFromJson(request, - CommonConstants.Broker.Request.QUERY_OPTIONS)); + sqlNodeAndOptions.setExtraOptions( + RequestUtils.getOptionsFromJson(request, CommonConstants.Broker.Request.QUERY_OPTIONS)); } - if (_multiStageBrokerRequestHandler != null && Boolean.parseBoolean(sqlNodeAndOptions.getOptions().get( - CommonConstants.Broker.Request.QueryOptionKey.USE_MULTISTAGE_ENGINE))) { - return _multiStageBrokerRequestHandler.handleRequest(request, requesterIdentity, requestContext, httpHeaders); + if (_multiStageBrokerRequestHandler != null && Boolean.parseBoolean( + sqlNodeAndOptions.getOptions().get(CommonConstants.Broker.Request.QueryOptionKey.USE_MULTISTAGE_ENGINE))) { + return _multiStageBrokerRequestHandler.handleRequest(request, requesterIdentity, requestContext, httpHeaders); } else { return _singleStageBrokerRequestHandler.handleRequest(request, sqlNodeAndOptions, requesterIdentity, requestContext, httpHeaders); From 805bc4ac1bbb70348eb3a6aa35a5a551cb8b4e32 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Apr 2024 12:55:58 -0700 Subject: [PATCH 08/72] Bump nl.jqno.equalsverifier:equalsverifier from 3.15.8 to 3.16.1 (#12836) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index ffe795d07e96..29b731ba1931 100644 --- a/pom.xml +++ b/pom.xml @@ -419,7 +419,7 @@ nl.jqno.equalsverifier equalsverifier - 3.15.8 + 3.16.1 test From 98873ebcbfb3d22b3f40cecf57b076d9532e7b14 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Apr 2024 12:56:31 -0700 Subject: [PATCH 09/72] Bump net.sf.jopt-simple:jopt-simple from 4.6 to 5.0.4 (#12835) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 29b731ba1931..1d9a1447a399 100644 --- a/pom.xml +++ b/pom.xml @@ -1379,7 +1379,7 @@ net.sf.jopt-simple jopt-simple - 4.6 + 5.0.4 org.reflections From d001acedf3df6697f457e72d82949049f5a7a847 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Apr 2024 12:57:17 -0700 Subject: [PATCH 10/72] Bump jakarta.annotation:jakarta.annotation-api from 2.1.1 to 3.0.0 (#12834) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 1d9a1447a399..75068cdd84f3 100644 --- a/pom.xml +++ b/pom.xml @@ -195,7 +195,7 @@ 6.0.0 2.0.1 3.0.2 - 2.1.1 + 3.0.0 4.0.2 3.1.0 2.1.3 From 68b730246319e9582a9e16e8d91e99afca6e1cf2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Apr 2024 12:57:42 -0700 Subject: [PATCH 11/72] Bump org.apache.datasketches:datasketches-java from 5.0.1 to 5.0.2 (#12833) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 75068cdd84f3..bbdd6e85ece3 100644 --- a/pom.xml +++ b/pom.xml @@ -1204,7 +1204,7 @@ org.apache.datasketches datasketches-java - 5.0.1 + 5.0.2 com.dynatrace.hash4j From 9b38a1f14cc6b738740a8a07b5f7f448f0d6e8ec Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Apr 2024 12:58:23 -0700 Subject: [PATCH 12/72] Bump io.projectreactor:reactor-core from 3.6.2 to 3.6.4 (#12832) --- pinot-plugins/pinot-file-system/pinot-adls/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml index a34ada8eed3c..fa27ff8582d2 100644 --- a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml +++ b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml @@ -62,7 +62,7 @@ io.projectreactor reactor-core - 3.6.2 + 3.6.4 net.java.dev.jna From 39edec5cb4934cd5171a0430624333b03ea55508 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Apr 2024 12:58:47 -0700 Subject: [PATCH 13/72] Bump com.azure:azure-core-http-netty from 1.14.0 to 1.14.2 (#12831) --- pinot-plugins/pinot-file-system/pinot-adls/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml index fa27ff8582d2..ba35053abc5f 100644 --- a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml +++ b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml @@ -52,7 +52,7 @@ com.azure azure-core-http-netty - 1.14.0 + 1.14.2 com.azure From 0f8630d2a3aac79da881d5a9fda12ce692910d13 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Apr 2024 12:59:20 -0700 Subject: [PATCH 14/72] Bump net.java.dev.jna:jna-platform from 5.6.0 to 5.14.0 (#12829) --- pinot-plugins/pinot-file-system/pinot-adls/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml index ba35053abc5f..1774de3e3356 100644 --- a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml +++ b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml @@ -67,7 +67,7 @@ net.java.dev.jna jna-platform - 5.6.0 + 5.14.0 net.java.dev.jna From eb912865521fa1a817d6155c5cdeeeac49a124ed Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Apr 2024 13:00:16 -0700 Subject: [PATCH 15/72] Bump com.diffplug.spotless:spotless-maven-plugin from 2.28.0 to 2.43.0 (#12827) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index bbdd6e85ece3..1ec79bac03c8 100644 --- a/pom.xml +++ b/pom.xml @@ -1583,7 +1583,7 @@ com.diffplug.spotless spotless-maven-plugin - 2.28.0 + 2.43.0 verify From 2fe29bb052251a85975d7b44fa0510027722fb86 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Apr 2024 13:00:34 -0700 Subject: [PATCH 16/72] Bump com.azure:azure-storage-file-datalake from 12.18.2 to 12.18.3 (#12826) --- pinot-plugins/pinot-file-system/pinot-adls/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml index 1774de3e3356..a0e513d53a60 100644 --- a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml +++ b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml @@ -39,7 +39,7 @@ com.azure azure-storage-file-datalake - 12.18.2 + 12.18.3 com.azure From 5504d0857b2f42c265d875d41a2155ba3ae54529 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Apr 2024 13:00:55 -0700 Subject: [PATCH 17/72] Bump bouncycastle.version from 1.77 to 1.78 (#12825) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 1ec79bac03c8..abe4405307f6 100644 --- a/pom.xml +++ b/pom.xml @@ -237,7 +237,7 @@ 1.5.4 9.4.54.v20240208 9.37.3 - 1.77 + 1.78 0.26 2.11.4 From 569189fe0d3ee2603e1b6fe69f1ac521e936a80e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Apr 2024 13:01:26 -0700 Subject: [PATCH 18/72] Bump actions/checkout from 2 to 4 (#12822) --- .github/workflows/pinot_tests.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/pinot_tests.yml b/.github/workflows/pinot_tests.yml index a0c4b4861e10..e0215d3cf8d3 100644 --- a/.github/workflows/pinot_tests.yml +++ b/.github/workflows/pinot_tests.yml @@ -271,7 +271,7 @@ jobs: ] name: Pinot Compatibility Regression Testing against ${{ matrix.old_commit }} on ${{ matrix.test_suite }} steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Set up JDK 11 uses: actions/setup-java@v4 with: From 364f0000367a52af4a525110fd89f697ad020f21 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Apr 2024 13:01:47 -0700 Subject: [PATCH 19/72] Bump com.google.cloud:libraries-bom from 26.34.0 to 26.37.0 (#12823) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index abe4405307f6..4a0036f95957 100644 --- a/pom.xml +++ b/pom.xml @@ -212,7 +212,7 @@ 2.2 - 26.34.0 + 26.37.0 1.23.0 2.10.1 33.1.0-jre From af44a2048e4a6f7946873843c83d46ce91dd3281 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Apr 2024 13:02:07 -0700 Subject: [PATCH 20/72] Bump actions/setup-node from 3 to 4 (#12821) --- .github/workflows/pinot_compatibility_tests.yml | 2 +- .github/workflows/pinot_tests.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/pinot_compatibility_tests.yml b/.github/workflows/pinot_compatibility_tests.yml index f2ff0f95f7f8..e9d242fede91 100644 --- a/.github/workflows/pinot_compatibility_tests.yml +++ b/.github/workflows/pinot_compatibility_tests.yml @@ -44,7 +44,7 @@ jobs: distribution: 'temurin' cache: 'maven' - name: Setup node - uses: actions/setup-node@v3 + uses: actions/setup-node@v4 with: node-version: v10.16.1 cache: 'npm' diff --git a/.github/workflows/pinot_tests.yml b/.github/workflows/pinot_tests.yml index e0215d3cf8d3..1d90331719c4 100644 --- a/.github/workflows/pinot_tests.yml +++ b/.github/workflows/pinot_tests.yml @@ -279,7 +279,7 @@ jobs: distribution: 'temurin' cache: 'maven' - name: Setup node - uses: actions/setup-node@v3 + uses: actions/setup-node@v4 with: node-version: v10.16.1 cache: 'npm' From 3c57a47a5028aeeffc1656ca81344b1ed04202c3 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Apr 2024 13:02:28 -0700 Subject: [PATCH 21/72] Bump testcontainers.version from 1.17.3 to 1.19.7 (#12820) --- pinot-integration-tests/pom.xml | 2 +- pinot-plugins/pinot-input-format/pinot-protobuf/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pinot-integration-tests/pom.xml b/pinot-integration-tests/pom.xml index 995e423c666d..baa472c1efb4 100644 --- a/pinot-integration-tests/pom.xml +++ b/pinot-integration-tests/pom.xml @@ -35,7 +35,7 @@ ${basedir}/.. 0.2.19 - 1.17.3 + 1.19.7 diff --git a/pinot-plugins/pinot-input-format/pinot-protobuf/pom.xml b/pinot-plugins/pinot-input-format/pinot-protobuf/pom.xml index 2d301bb2fd14..da6832e869e2 100644 --- a/pinot-plugins/pinot-input-format/pinot-protobuf/pom.xml +++ b/pinot-plugins/pinot-input-format/pinot-protobuf/pom.xml @@ -36,7 +36,7 @@ ${basedir}/../../.. 2.8.1 - 1.17.3 + 1.19.7 package From 369c6593d1ba6e04ae531fd903ac0968bc38f377 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Apr 2024 13:02:46 -0700 Subject: [PATCH 22/72] Bump docker/login-action from 2 to 3 (#12819) --- .github/workflows/build-multi-arch-pinot-docker-image.yml | 4 ++-- .github/workflows/build-pinot-base-docker-image.yml | 2 +- .github/workflows/build-pinot-docker-image.yml | 2 +- .github/workflows/build-presto-docker-image.yml | 2 +- .github/workflows/build-superset-docker-image.yml | 2 +- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/build-multi-arch-pinot-docker-image.yml b/.github/workflows/build-multi-arch-pinot-docker-image.yml index 25460a77bf70..74a22ea180b7 100644 --- a/.github/workflows/build-multi-arch-pinot-docker-image.yml +++ b/.github/workflows/build-multi-arch-pinot-docker-image.yml @@ -64,7 +64,7 @@ jobs: needs: [ generate-build-info ] steps: - name: Login to DockerHub - uses: docker/login-action@v2 + uses: docker/login-action@v3 with: username: ${{ secrets.DOCKERHUB_USERNAME }} password: ${{ secrets.DOCKERHUB_TOKEN }} @@ -91,7 +91,7 @@ jobs: needs: [ generate-build-info, build-pinot-docker-image ] steps: - name: Login to DockerHub - uses: docker/login-action@v2 + uses: docker/login-action@v3 with: username: ${{ secrets.DOCKERHUB_USERNAME }} password: ${{ secrets.DOCKERHUB_TOKEN }} diff --git a/.github/workflows/build-pinot-base-docker-image.yml b/.github/workflows/build-pinot-base-docker-image.yml index 36162d49051f..810ea865123e 100644 --- a/.github/workflows/build-pinot-base-docker-image.yml +++ b/.github/workflows/build-pinot-base-docker-image.yml @@ -32,7 +32,7 @@ jobs: openJdkDist: [ "amazoncorretto", "ms-openjdk" ] steps: - name: Login to DockerHub - uses: docker/login-action@v2 + uses: docker/login-action@v3 with: username: ${{ secrets.DOCKERHUB_USERNAME }} password: ${{ secrets.DOCKERHUB_TOKEN }} diff --git a/.github/workflows/build-pinot-docker-image.yml b/.github/workflows/build-pinot-docker-image.yml index c8652f2775c7..8d796dc1c350 100644 --- a/.github/workflows/build-pinot-docker-image.yml +++ b/.github/workflows/build-pinot-docker-image.yml @@ -47,7 +47,7 @@ jobs: runs-on: ubuntu-latest steps: - name: Login to DockerHub - uses: docker/login-action@v2 + uses: docker/login-action@v3 with: username: ${{ secrets.DOCKERHUB_USERNAME }} password: ${{ secrets.DOCKERHUB_TOKEN }} diff --git a/.github/workflows/build-presto-docker-image.yml b/.github/workflows/build-presto-docker-image.yml index 6c76671a3e3f..a30a2bc6ee31 100644 --- a/.github/workflows/build-presto-docker-image.yml +++ b/.github/workflows/build-presto-docker-image.yml @@ -47,7 +47,7 @@ jobs: runs-on: ubuntu-latest steps: - name: Login to DockerHub - uses: docker/login-action@v2 + uses: docker/login-action@v3 with: username: ${{ secrets.DOCKERHUB_USERNAME }} password: ${{ secrets.DOCKERHUB_TOKEN }} diff --git a/.github/workflows/build-superset-docker-image.yml b/.github/workflows/build-superset-docker-image.yml index 19b720905ac3..7ede0e1628f5 100644 --- a/.github/workflows/build-superset-docker-image.yml +++ b/.github/workflows/build-superset-docker-image.yml @@ -43,7 +43,7 @@ jobs: runs-on: ubuntu-latest steps: - name: Login to DockerHub - uses: docker/login-action@v2 + uses: docker/login-action@v3 with: username: ${{ secrets.DOCKERHUB_USERNAME }} password: ${{ secrets.DOCKERHUB_TOKEN }} From 2f5dbf1c70ea357936fca18de1e7617e410d14b2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Apr 2024 13:03:50 -0700 Subject: [PATCH 23/72] Bump docker/setup-buildx-action from 2 to 3 (#12817) --- .github/workflows/build-multi-arch-pinot-docker-image.yml | 4 ++-- .github/workflows/build-pinot-base-docker-image.yml | 2 +- .github/workflows/build-pinot-docker-image.yml | 2 +- .github/workflows/build-presto-docker-image.yml | 2 +- .github/workflows/build-superset-docker-image.yml | 2 +- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/build-multi-arch-pinot-docker-image.yml b/.github/workflows/build-multi-arch-pinot-docker-image.yml index 74a22ea180b7..e3314bec9c12 100644 --- a/.github/workflows/build-multi-arch-pinot-docker-image.yml +++ b/.github/workflows/build-multi-arch-pinot-docker-image.yml @@ -72,7 +72,7 @@ jobs: name: Set up QEMU with: platforms: linux/${{ matrix.arch }} - - uses: docker/setup-buildx-action@v2 + - uses: docker/setup-buildx-action@v3 name: Set up Docker Buildx - uses: actions/checkout@v4 - name: Build and push the Docker image @@ -97,7 +97,7 @@ jobs: password: ${{ secrets.DOCKERHUB_TOKEN }} - uses: docker/setup-qemu-action@v3 name: Set up QEMU - - uses: docker/setup-buildx-action@v2 + - uses: docker/setup-buildx-action@v3 name: Set up Docker Buildx - uses: actions/checkout@v4 - name: Create Multi-Arch Manifest diff --git a/.github/workflows/build-pinot-base-docker-image.yml b/.github/workflows/build-pinot-base-docker-image.yml index 810ea865123e..e079fde1c304 100644 --- a/.github/workflows/build-pinot-base-docker-image.yml +++ b/.github/workflows/build-pinot-base-docker-image.yml @@ -38,7 +38,7 @@ jobs: password: ${{ secrets.DOCKERHUB_TOKEN }} - uses: docker/setup-qemu-action@v3 name: Set up QEMU - - uses: docker/setup-buildx-action@v2 + - uses: docker/setup-buildx-action@v3 name: Set up Docker Buildx - uses: actions/checkout@v4 - name: Build and push the Docker image diff --git a/.github/workflows/build-pinot-docker-image.yml b/.github/workflows/build-pinot-docker-image.yml index 8d796dc1c350..c21e8bbadd6e 100644 --- a/.github/workflows/build-pinot-docker-image.yml +++ b/.github/workflows/build-pinot-docker-image.yml @@ -53,7 +53,7 @@ jobs: password: ${{ secrets.DOCKERHUB_TOKEN }} - uses: docker/setup-qemu-action@v3 name: Set up QEMU - - uses: docker/setup-buildx-action@v2 + - uses: docker/setup-buildx-action@v3 name: Set up Docker Buildx - uses: actions/checkout@v4 - name: Build and push the Docker image diff --git a/.github/workflows/build-presto-docker-image.yml b/.github/workflows/build-presto-docker-image.yml index a30a2bc6ee31..d054074f1551 100644 --- a/.github/workflows/build-presto-docker-image.yml +++ b/.github/workflows/build-presto-docker-image.yml @@ -53,7 +53,7 @@ jobs: password: ${{ secrets.DOCKERHUB_TOKEN }} - uses: docker/setup-qemu-action@v3 name: Set up QEMU - - uses: docker/setup-buildx-action@v2 + - uses: docker/setup-buildx-action@v3 name: Set up Docker Buildx - uses: actions/checkout@v4 - name: Build and push the Docker image diff --git a/.github/workflows/build-superset-docker-image.yml b/.github/workflows/build-superset-docker-image.yml index 7ede0e1628f5..9cc362d7c502 100644 --- a/.github/workflows/build-superset-docker-image.yml +++ b/.github/workflows/build-superset-docker-image.yml @@ -49,7 +49,7 @@ jobs: password: ${{ secrets.DOCKERHUB_TOKEN }} - uses: docker/setup-qemu-action@v3 name: Set up QEMU - - uses: docker/setup-buildx-action@v2 + - uses: docker/setup-buildx-action@v3 name: Set up Docker Buildx - uses: actions/checkout@v4 - name: Build and push the Docker image From 396988de6760f8e455292cf4c8637990c24f0d0f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Apr 2024 13:04:07 -0700 Subject: [PATCH 24/72] Bump github/codeql-action from 2 to 3 (#12818) --- .github/workflows/pinot_vuln_check.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/pinot_vuln_check.yml b/.github/workflows/pinot_vuln_check.yml index 4eebc1fb1074..b4924d3bc848 100644 --- a/.github/workflows/pinot_vuln_check.yml +++ b/.github/workflows/pinot_vuln_check.yml @@ -60,6 +60,6 @@ jobs: severity: 'CRITICAL,HIGH' timeout: '10m' - name: Upload Trivy scan results to GitHub Security tab - uses: github/codeql-action/upload-sarif@v2 + uses: github/codeql-action/upload-sarif@v3 with: sarif_file: 'trivy-results.sarif' From 156a3e425f0c20e5c0908cc2cc61afe8f83c8cf9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Apr 2024 15:25:10 -0700 Subject: [PATCH 25/72] Bump org.apache.maven.plugins:maven-gpg-plugin from 3.2.0 to 3.2.2 (#12728) --- pinot-connectors/pinot-spark-2-connector/pom.xml | 2 +- pinot-connectors/pinot-spark-3-connector/pom.xml | 2 +- pinot-connectors/pinot-spark-common/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pinot-connectors/pinot-spark-2-connector/pom.xml b/pinot-connectors/pinot-spark-2-connector/pom.xml index a51582fcac05..1463cff56fc7 100644 --- a/pinot-connectors/pinot-spark-2-connector/pom.xml +++ b/pinot-connectors/pinot-spark-2-connector/pom.xml @@ -152,7 +152,7 @@ Thus, explicitly adding this plugin to a new profile to sign the files at the end all at once. --> org.apache.maven.plugins maven-gpg-plugin - 3.2.0 + 3.2.2 diff --git a/pinot-connectors/pinot-spark-3-connector/pom.xml b/pinot-connectors/pinot-spark-3-connector/pom.xml index 6e53637cb270..96efe295f345 100644 --- a/pinot-connectors/pinot-spark-3-connector/pom.xml +++ b/pinot-connectors/pinot-spark-3-connector/pom.xml @@ -148,7 +148,7 @@ Thus, explicitly adding this plugin to a new profile to sign the files at the end all at once. --> org.apache.maven.plugins maven-gpg-plugin - 3.2.0 + 3.2.2 diff --git a/pinot-connectors/pinot-spark-common/pom.xml b/pinot-connectors/pinot-spark-common/pom.xml index bf60a2f6adfa..e130a295b732 100644 --- a/pinot-connectors/pinot-spark-common/pom.xml +++ b/pinot-connectors/pinot-spark-common/pom.xml @@ -163,7 +163,7 @@ Thus, explicitly adding this plugin to a new profile to sign the files at the end all at once. --> org.apache.maven.plugins maven-gpg-plugin - 3.2.0 + 3.2.2 From 620c46acd3fe7a0c084bed9474081601a89e2bcd Mon Sep 17 00:00:00 2001 From: "Xiaotian (Jackie) Jiang" <17555551+Jackie-Jiang@users.noreply.github.com> Date: Mon, 8 Apr 2024 16:53:37 -0700 Subject: [PATCH 26/72] Move package org.apache.calcite to org.apache.pinot.calcite (#12837) --- .../MultiStageBrokerRequestHandler.java | 2 +- .../api/resources/PinotQueryResource.java | 2 +- .../calcite/jdbc/CalciteSchemaBuilder.java | 15 +------ .../prepare/PinotCalciteCatalogReader.java | 19 ++++---- .../calcite/rel/hint/PinotHintOptions.java | 6 ++- .../rel/hint/PinotHintStrategyTable.java | 18 ++++---- .../rel/logical/PinotLogicalExchange.java | 23 ++++------ .../rel/logical/PinotLogicalSortExchange.java | 45 +++++++------------ .../rel/logical/PinotRelExchangeType.java | 2 +- .../rules/ImmutableSortExchangeCopyRule.java | 15 ++++--- .../PinotAggregateExchangeNodeInsertRule.java | 14 +++--- .../PinotAggregateLiteralAttachmentRule.java | 6 +-- .../rules/PinotAggregateToSemiJoinRule.java | 3 +- .../rel/rules/PinotEvaluateLiteralRule.java | 2 +- .../rules/PinotExchangeEliminationRule.java | 4 +- .../rules/PinotFilterExpandSearchRule.java | 2 +- .../PinotJoinExchangeNodeInsertRule.java | 4 +- .../PinotJoinToDynamicBroadcastRule.java | 10 ++--- .../calcite/rel/rules/PinotQueryRuleSets.java | 4 +- .../rules/PinotRelDistributionTraitRule.java | 8 ++-- .../calcite/rel/rules/PinotRuleUtils.java | 4 +- .../PinotSetOpExchangeNodeInsertRule.java | 4 +- .../PinotSingleValueAggregateRemoveRule.java | 2 +- .../rel/rules/PinotSortExchangeCopyRule.java | 4 +- .../PinotSortExchangeNodeInsertRule.java | 4 +- .../PinotWindowExchangeNodeInsertRule.java | 6 +-- .../calcite/sql/PinotSqlAggFunction.java | 6 ++- .../sql/PinotSqlTransformFunction.java | 5 ++- .../calcite/sql/fun/PinotOperatorTable.java | 23 +++++----- .../sql/fun/PinotSqlCoalesceFunction.java | 3 +- .../util/PinotChainedSqlOperatorTable.java | 21 +++++---- .../calcite/sql2rel/PinotConvertletTable.java | 6 ++- .../apache/pinot/query/QueryEnvironment.java | 14 +++--- .../pinot/query/catalog/PinotCatalog.java | 5 ++- .../logical/PinotLogicalQueryPlanner.java | 2 +- .../query/planner/logical/PlanFragmenter.java | 2 +- .../logical/RelToPlanNodeConverter.java | 6 +-- .../planner/logical/SubPlanFragmenter.java | 2 +- .../physical/DispatchablePlanVisitor.java | 2 +- .../query/planner/plannode/AggregateNode.java | 4 +- .../query/planner/plannode/ExchangeNode.java | 2 +- .../planner/plannode/MailboxReceiveNode.java | 2 +- .../planner/plannode/MailboxSendNode.java | 2 +- .../pinot/query/routing/WorkerManager.java | 2 +- .../rules/PinotSortExchangeCopyRuleTest.java | 35 +++++++-------- .../pinot/query/QueryEnvironmentTestBase.java | 2 +- .../runtime/operator/AggregateOperator.java | 2 +- .../runtime/operator/HashJoinOperator.java | 2 +- .../operator/MultistageGroupByExecutor.java | 2 +- .../plan/pipeline/PipelineBreakerVisitor.java | 2 +- .../operator/AggregateOperatorTest.java | 2 +- .../operator/HashJoinOperatorTest.java | 2 +- .../pipeline/PipelineBreakerExecutorTest.java | 2 +- 53 files changed, 193 insertions(+), 195 deletions(-) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/jdbc/CalciteSchemaBuilder.java (71%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/prepare/PinotCalciteCatalogReader.java (96%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/hint/PinotHintOptions.java (95%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/hint/PinotHintStrategyTable.java (90%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/logical/PinotLogicalExchange.java (81%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/logical/PinotLogicalSortExchange.java (78%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/logical/PinotRelExchangeType.java (97%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/rules/ImmutableSortExchangeCopyRule.java (98%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java (97%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/rules/PinotAggregateLiteralAttachmentRule.java (96%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/rules/PinotAggregateToSemiJoinRule.java (98%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/rules/PinotEvaluateLiteralRule.java (99%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/rules/PinotExchangeEliminationRule.java (94%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/rules/PinotFilterExpandSearchRule.java (98%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/rules/PinotJoinExchangeNodeInsertRule.java (96%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/rules/PinotJoinToDynamicBroadcastRule.java (96%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/rules/PinotQueryRuleSets.java (97%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/rules/PinotRelDistributionTraitRule.java (97%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/rules/PinotRuleUtils.java (97%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/rules/PinotSetOpExchangeNodeInsertRule.java (96%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/rules/PinotSingleValueAggregateRemoveRule.java (98%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/rules/PinotSortExchangeCopyRule.java (97%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/rules/PinotSortExchangeNodeInsertRule.java (96%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java (98%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/sql/PinotSqlAggFunction.java (91%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/sql/PinotSqlTransformFunction.java (90%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/sql/fun/PinotOperatorTable.java (90%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/sql/fun/PinotSqlCoalesceFunction.java (92%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/sql/util/PinotChainedSqlOperatorTable.java (83%) rename pinot-query-planner/src/main/java/org/apache/{ => pinot}/calcite/sql2rel/PinotConvertletTable.java (92%) rename pinot-query-planner/src/test/java/org/apache/{ => pinot}/calcite/rel/rules/PinotSortExchangeCopyRuleTest.java (92%) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java index ee9f6cf19a74..01e4884d6a08 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java @@ -31,7 +31,6 @@ import javax.ws.rs.WebApplicationException; import javax.ws.rs.core.HttpHeaders; import javax.ws.rs.core.Response; -import org.apache.calcite.jdbc.CalciteSchemaBuilder; import org.apache.commons.lang3.tuple.Pair; import org.apache.pinot.broker.api.AccessControl; import org.apache.pinot.broker.api.RequesterIdentity; @@ -39,6 +38,7 @@ import org.apache.pinot.broker.querylog.QueryLogger; import org.apache.pinot.broker.queryquota.QueryQuotaManager; import org.apache.pinot.broker.routing.BrokerRoutingManager; +import org.apache.pinot.calcite.jdbc.CalciteSchemaBuilder; import org.apache.pinot.common.config.provider.TableCache; import org.apache.pinot.common.exception.QueryException; import org.apache.pinot.common.metrics.BrokerMeter; diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java index dbd1df943827..aa40ec40a8d4 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java @@ -49,11 +49,11 @@ import javax.ws.rs.core.Context; import javax.ws.rs.core.HttpHeaders; import javax.ws.rs.core.Response; -import org.apache.calcite.jdbc.CalciteSchemaBuilder; import org.apache.calcite.sql.SqlNode; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.helix.model.InstanceConfig; +import org.apache.pinot.calcite.jdbc.CalciteSchemaBuilder; import org.apache.pinot.common.Utils; import org.apache.pinot.common.exception.QueryException; import org.apache.pinot.common.response.ProcessingException; diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/jdbc/CalciteSchemaBuilder.java similarity index 71% rename from pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/jdbc/CalciteSchemaBuilder.java index bab3ac045d91..1da5e2d7f1a0 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/jdbc/CalciteSchemaBuilder.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/jdbc/CalciteSchemaBuilder.java @@ -16,10 +16,11 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.jdbc; +package org.apache.pinot.calcite.jdbc; import java.util.List; import java.util.Map; +import org.apache.calcite.jdbc.CalciteSchema; import org.apache.calcite.schema.Function; import org.apache.calcite.schema.Schema; import org.apache.calcite.schema.SchemaPlus; @@ -28,26 +29,14 @@ /** * This class is used to create a {@link CalciteSchema} with a given {@link Schema} as the root. - * - *

This class resides in calcite.jdbc namespace because there's no complex logic we have in terms of catalog-based - * schema construct. We instead create a {@link SimpleCalciteSchema} that's package protected. */ public class CalciteSchemaBuilder { - private CalciteSchemaBuilder() { - // do not instantiate. } /** * Creates a {@link CalciteSchema} with a given {@link Schema} as the root. * - *

Calcite creates two layer of abstraction, the {@link CalciteSchema} is use internally for planner and - * {@link Schema} is user-facing with overrides. In our case we don't have a complex internal wrapper extension - * so we only reuse the package protected {@link SimpleCalciteSchema}. - * - *

If there's need to extend this feature for planner functionalities we should create our own extension to the - * {@link CalciteSchema}. - * * @param root schema to use as a root schema * @return calcite schema with given schema as the root */ diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/prepare/PinotCalciteCatalogReader.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/prepare/PinotCalciteCatalogReader.java similarity index 96% rename from pinot-query-planner/src/main/java/org/apache/calcite/prepare/PinotCalciteCatalogReader.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/prepare/PinotCalciteCatalogReader.java index 64d1e54e4db0..c345ca216cb1 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/prepare/PinotCalciteCatalogReader.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/prepare/PinotCalciteCatalogReader.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.prepare; +package org.apache.pinot.calcite.prepare; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; @@ -35,6 +35,9 @@ import org.apache.calcite.linq4j.function.Hints; import org.apache.calcite.model.ModelHandler; import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.prepare.CalciteCatalogReader; +import org.apache.calcite.prepare.Prepare; +import org.apache.calcite.prepare.RelOptTableImpl; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rel.type.RelDataTypeFactoryImpl; @@ -78,16 +81,15 @@ /** - * ============================================================================ - * THIS CLASS IS COPIED FROM Calcite's {@link org.apache.calcite.prepare.CalciteCatalogReader} and modified the - * case sensitivity of Function lookup. which is ALWAYS case-insensitive regardless of conventions on - * column/table identifier. - * ============================================================================ + * ============================================================================================================== + * THIS CLASS IS COPIED FROM Calcite's {@link CalciteCatalogReader} and modified the case sensitivity of Function + * lookup, which is ALWAYS case-insensitive regardless of conventions on column/table identifier. + * ============================================================================================================== * - * Pinot's implementation of {@link org.apache.calcite.prepare.Prepare.CatalogReader} - * and also {@link org.apache.calcite.sql.SqlOperatorTable} based on tables and + * Pinot's implementation of {@link Prepare.CatalogReader} and also {@link SqlOperatorTable} based on tables and * functions defined schemas. */ +//@formatter:off public class PinotCalciteCatalogReader implements Prepare.CatalogReader { protected final CalciteSchema _rootSchema; protected final RelDataTypeFactory _typeFactory; @@ -463,3 +465,4 @@ private static RelDataType toSql(RelDataTypeFactory typeFactory, return null; } } +//@formatter:on diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintOptions.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java similarity index 95% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintOptions.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java index 0a7ec1b10a70..a45e92aba66f 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintOptions.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java @@ -16,15 +16,17 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.hint; +package org.apache.pinot.calcite.rel.hint; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.hint.RelHint; import org.apache.pinot.query.planner.logical.LiteralHintUtils; /** * {@code PinotHintOptions} specified the supported hint options by Pinot based a particular type of relation node. * - *

for each {@link org.apache.calcite.rel.RelNode} type we support an option hint name.

+ *

for each {@link RelNode} type we support an option hint name.

*

for each option hint name there's a corresponding {@link RelHint} that supported only key-value option stored * in {@link RelHint#kvOptions}

*/ diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintStrategyTable.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintStrategyTable.java similarity index 90% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintStrategyTable.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintStrategyTable.java index c49a4b975403..524bffc75daf 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/hint/PinotHintStrategyTable.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintStrategyTable.java @@ -16,13 +16,16 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.hint; +package org.apache.pinot.calcite.rel.hint; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import javax.annotation.Nullable; +import org.apache.calcite.rel.hint.HintPredicates; +import org.apache.calcite.rel.hint.HintStrategyTable; +import org.apache.calcite.rel.hint.RelHint; import org.apache.pinot.spi.utils.BooleanUtils; @@ -30,17 +33,14 @@ * Default hint strategy set for Pinot query. */ public class PinotHintStrategyTable { - private PinotHintStrategyTable() { - // do not instantiate. } - public static final HintStrategyTable PINOT_HINT_STRATEGY_TABLE = HintStrategyTable.builder() - .hintStrategy(PinotHintOptions.INTERNAL_AGG_OPTIONS, HintPredicates.AGGREGATE) - .hintStrategy(PinotHintOptions.AGGREGATE_HINT_OPTIONS, HintPredicates.AGGREGATE) - .hintStrategy(PinotHintOptions.JOIN_HINT_OPTIONS, HintPredicates.JOIN) - .hintStrategy(PinotHintOptions.TABLE_HINT_OPTIONS, HintPredicates.TABLE_SCAN) - .build(); + public static final HintStrategyTable PINOT_HINT_STRATEGY_TABLE = + HintStrategyTable.builder().hintStrategy(PinotHintOptions.INTERNAL_AGG_OPTIONS, HintPredicates.AGGREGATE) + .hintStrategy(PinotHintOptions.AGGREGATE_HINT_OPTIONS, HintPredicates.AGGREGATE) + .hintStrategy(PinotHintOptions.JOIN_HINT_OPTIONS, HintPredicates.JOIN) + .hintStrategy(PinotHintOptions.TABLE_HINT_OPTIONS, HintPredicates.TABLE_SCAN).build(); /** * Check if a hint-able {@link org.apache.calcite.rel.RelNode} contains a specific {@link RelHint} by name. diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/logical/PinotLogicalExchange.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalExchange.java similarity index 81% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/logical/PinotLogicalExchange.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalExchange.java index c08b7ae8ade0..375a00500b78 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/logical/PinotLogicalExchange.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalExchange.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.logical; +package org.apache.pinot.calcite.rel.logical; import org.apache.calcite.plan.Convention; import org.apache.calcite.plan.RelOptCluster; @@ -36,16 +36,14 @@ public class PinotLogicalExchange extends Exchange { private final PinotRelExchangeType _exchangeType; - private PinotLogicalExchange(RelOptCluster cluster, RelTraitSet traitSet, - RelNode input, RelDistribution distribution, PinotRelExchangeType exchangeType) { + private PinotLogicalExchange(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, RelDistribution distribution, + PinotRelExchangeType exchangeType) { super(cluster, traitSet, input, distribution); _exchangeType = exchangeType; assert traitSet.containsIfApplicable(Convention.NONE); } - - public static PinotLogicalExchange create(RelNode input, - RelDistribution distribution) { + public static PinotLogicalExchange create(RelNode input, RelDistribution distribution) { return create(input, distribution, PinotRelExchangeType.getDefaultExchangeType()); } @@ -56,22 +54,19 @@ public static PinotLogicalExchange create(RelNode input, * @param distribution Distribution specification * @param exchangeType RelExchangeType specification */ - public static PinotLogicalExchange create(RelNode input, - RelDistribution distribution, PinotRelExchangeType exchangeType) { + public static PinotLogicalExchange create(RelNode input, RelDistribution distribution, + PinotRelExchangeType exchangeType) { RelOptCluster cluster = input.getCluster(); distribution = RelDistributionTraitDef.INSTANCE.canonize(distribution); - RelTraitSet traitSet = - input.getTraitSet().replace(Convention.NONE).replace(distribution); + RelTraitSet traitSet = input.getTraitSet().replace(Convention.NONE).replace(distribution); return new PinotLogicalExchange(cluster, traitSet, input, distribution, exchangeType); } //~ Methods ---------------------------------------------------------------- @Override - public Exchange copy(RelTraitSet traitSet, RelNode newInput, - RelDistribution newDistribution) { - return new PinotLogicalExchange(getCluster(), traitSet, newInput, - newDistribution, _exchangeType); + public Exchange copy(RelTraitSet traitSet, RelNode newInput, RelDistribution newDistribution) { + return new PinotLogicalExchange(getCluster(), traitSet, newInput, newDistribution, _exchangeType); } @Override diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/logical/PinotLogicalSortExchange.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalSortExchange.java similarity index 78% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/logical/PinotLogicalSortExchange.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalSortExchange.java index ace06a6c43f1..141b20d422f7 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/logical/PinotLogicalSortExchange.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalSortExchange.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.logical; +package org.apache.pinot.calcite.rel.logical; import org.apache.calcite.plan.Convention; import org.apache.calcite.plan.RelOptCluster; @@ -46,9 +46,9 @@ public class PinotLogicalSortExchange extends SortExchange { protected final boolean _isSortOnReceiver; protected final PinotRelExchangeType _exchangeType; - private PinotLogicalSortExchange(RelOptCluster cluster, RelTraitSet traitSet, - RelNode input, RelDistribution distribution, PinotRelExchangeType exchangeType, RelCollation collation, - boolean isSortOnSender, boolean isSortOnReceiver) { + private PinotLogicalSortExchange(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, + RelDistribution distribution, PinotRelExchangeType exchangeType, RelCollation collation, boolean isSortOnSender, + boolean isSortOnReceiver) { super(cluster, traitSet, input, distribution, collation); _exchangeType = exchangeType; _isSortOnSender = isSortOnSender; @@ -65,12 +65,8 @@ public PinotLogicalSortExchange(RelInput input) { _isSortOnReceiver = true; } - public static PinotLogicalSortExchange create( - RelNode input, - RelDistribution distribution, - RelCollation collation, - boolean isSortOnSender, - boolean isSortOnReceiver) { + public static PinotLogicalSortExchange create(RelNode input, RelDistribution distribution, RelCollation collation, + boolean isSortOnSender, boolean isSortOnReceiver) { return create(input, distribution, PinotRelExchangeType.getDefaultExchangeType(), collation, isSortOnSender, isSortOnReceiver); } @@ -85,36 +81,29 @@ public static PinotLogicalSortExchange create( * @param isSortOnSender whether to sort on the sender * @param isSortOnReceiver whether to sort on receiver */ - public static PinotLogicalSortExchange create( - RelNode input, - RelDistribution distribution, - PinotRelExchangeType exchangeType, - RelCollation collation, - boolean isSortOnSender, - boolean isSortOnReceiver) { + public static PinotLogicalSortExchange create(RelNode input, RelDistribution distribution, + PinotRelExchangeType exchangeType, RelCollation collation, boolean isSortOnSender, boolean isSortOnReceiver) { RelOptCluster cluster = input.getCluster(); collation = RelCollationTraitDef.INSTANCE.canonize(collation); distribution = RelDistributionTraitDef.INSTANCE.canonize(distribution); - RelTraitSet traitSet = - input.getTraitSet().replace(Convention.NONE).replace(distribution).replace(collation); - return new PinotLogicalSortExchange(cluster, traitSet, input, distribution, exchangeType, - collation, isSortOnSender, isSortOnReceiver); + RelTraitSet traitSet = input.getTraitSet().replace(Convention.NONE).replace(distribution).replace(collation); + return new PinotLogicalSortExchange(cluster, traitSet, input, distribution, exchangeType, collation, isSortOnSender, + isSortOnReceiver); } //~ Methods ---------------------------------------------------------------- @Override - public SortExchange copy(RelTraitSet traitSet, RelNode newInput, - RelDistribution newDistribution, RelCollation newCollation) { - return new PinotLogicalSortExchange(this.getCluster(), traitSet, newInput, - newDistribution, _exchangeType, newCollation, _isSortOnSender, _isSortOnReceiver); + public SortExchange copy(RelTraitSet traitSet, RelNode newInput, RelDistribution newDistribution, + RelCollation newCollation) { + return new PinotLogicalSortExchange(this.getCluster(), traitSet, newInput, newDistribution, _exchangeType, + newCollation, _isSortOnSender, _isSortOnReceiver); } @Override public RelWriter explainTerms(RelWriter pw) { - RelWriter relWriter = super.explainTerms(pw) - .item("isSortOnSender", _isSortOnSender) - .item("isSortOnReceiver", _isSortOnReceiver); + RelWriter relWriter = + super.explainTerms(pw).item("isSortOnSender", _isSortOnSender).item("isSortOnReceiver", _isSortOnReceiver); if (_exchangeType != PinotRelExchangeType.getDefaultExchangeType()) { relWriter.item("relExchangeType", _exchangeType); } diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/logical/PinotRelExchangeType.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotRelExchangeType.java similarity index 97% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/logical/PinotRelExchangeType.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotRelExchangeType.java index 4a06e1d02562..49d5cff72839 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/logical/PinotRelExchangeType.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotRelExchangeType.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.logical; +package org.apache.pinot.calcite.rel.logical; /** Type of exchange. */ public enum PinotRelExchangeType { diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/ImmutableSortExchangeCopyRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/ImmutableSortExchangeCopyRule.java similarity index 98% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/ImmutableSortExchangeCopyRule.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/ImmutableSortExchangeCopyRule.java index eca7fc61104a..7a7ae16c4da3 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/ImmutableSortExchangeCopyRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/ImmutableSortExchangeCopyRule.java @@ -16,13 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; - -// NOTE: this file was generated using Calcite's code generator, but instead of pulling in all -// the dependencies for codegen we just manually generate it and check it in. If active development -// on this needs to happen, re-generate it using Calcite's generator. - -// CHECKSTYLE:OFF +package org.apache.pinot.calcite.rel.rules; import com.google.common.base.MoreObjects; import java.util.ArrayList; @@ -35,10 +29,16 @@ /** + * NOTE: this file was generated using Calcite's code generator, but instead of pulling in all + * the dependencies for codegen we just manually generate it and check it in. If active development + * on this needs to happen, re-generate it using Calcite's generator. + * * {@code ImmutableSortExchangeCopyRule} contains immutable implementation classes generated from * abstract value types defined as nested inside {@link SortExchangeCopyRule}. * @see ImmutableSortExchangeCopyRule.Config */ +// CHECKSTYLE:OFF +//@formatter:off @SuppressWarnings({"all"}) final class ImmutableSortExchangeCopyRule { private ImmutableSortExchangeCopyRule() { @@ -412,3 +412,4 @@ public ImmutableSortExchangeCopyRule.Config build() { } } } +//@formatter:on diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java similarity index 97% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java index df904123d201..b7f0038145cb 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; +package org.apache.pinot.calcite.rel.rules; import com.google.common.collect.ImmutableList; import java.util.ArrayList; @@ -33,14 +33,12 @@ import org.apache.calcite.rel.core.Aggregate; import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.rel.core.Project; -import org.apache.calcite.rel.hint.PinotHintOptions; -import org.apache.calcite.rel.hint.PinotHintStrategyTable; import org.apache.calcite.rel.hint.RelHint; import org.apache.calcite.rel.logical.LogicalAggregate; -import org.apache.calcite.rel.logical.PinotLogicalExchange; +import org.apache.calcite.rel.rules.AggregateExtractProjectRule; +import org.apache.calcite.rel.rules.AggregateReduceFunctionsRule; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; -import org.apache.calcite.sql.PinotSqlAggFunction; import org.apache.calcite.sql.SqlAggFunction; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.type.OperandTypes; @@ -53,6 +51,10 @@ import org.apache.calcite.util.mapping.Mapping; import org.apache.calcite.util.mapping.MappingType; import org.apache.calcite.util.mapping.Mappings; +import org.apache.pinot.calcite.rel.hint.PinotHintOptions; +import org.apache.pinot.calcite.rel.hint.PinotHintStrategyTable; +import org.apache.pinot.calcite.rel.logical.PinotLogicalExchange; +import org.apache.pinot.calcite.sql.PinotSqlAggFunction; import org.apache.pinot.query.planner.plannode.AggregateNode.AggType; import org.apache.pinot.segment.spi.AggregationFunctionType; @@ -62,7 +64,7 @@ * (1) non-data-locale Pinot server agg stage, and (2) the data-locale Pinot intermediate agg stage. * * Pinot uses special intermediate data representation for partially aggregated results, thus we can't use - * {@link org.apache.calcite.rel.rules.AggregateReduceFunctionsRule} to reduce complex aggregation. + * {@link AggregateReduceFunctionsRule} to reduce complex aggregation. * * This rule is here to introduces Pinot-special aggregation splits. In-general there are several options: *
    diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotAggregateLiteralAttachmentRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotAggregateLiteralAttachmentRule.java similarity index 96% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotAggregateLiteralAttachmentRule.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotAggregateLiteralAttachmentRule.java index abad94fabf5e..74af35b47a3d 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotAggregateLiteralAttachmentRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotAggregateLiteralAttachmentRule.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; +package org.apache.pinot.calcite.rel.rules; import com.google.common.collect.ImmutableList; import java.util.HashMap; @@ -28,14 +28,14 @@ import org.apache.calcite.rel.core.Aggregate; import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.rel.core.Project; -import org.apache.calcite.rel.hint.PinotHintOptions; -import org.apache.calcite.rel.hint.PinotHintStrategyTable; import org.apache.calcite.rel.hint.RelHint; import org.apache.calcite.rel.logical.LogicalAggregate; import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; import org.apache.calcite.tools.RelBuilderFactory; import org.apache.calcite.util.Pair; +import org.apache.pinot.calcite.rel.hint.PinotHintOptions; +import org.apache.pinot.calcite.rel.hint.PinotHintStrategyTable; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; import org.apache.pinot.query.planner.logical.LiteralHintUtils; import org.apache.pinot.query.planner.logical.RexExpression; diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotAggregateToSemiJoinRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotAggregateToSemiJoinRule.java similarity index 98% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotAggregateToSemiJoinRule.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotAggregateToSemiJoinRule.java index 95267d3595ed..ebf61df801a6 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotAggregateToSemiJoinRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotAggregateToSemiJoinRule.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; +package org.apache.pinot.calcite.rel.rules; import java.util.ArrayList; import java.util.List; @@ -29,6 +29,7 @@ import org.apache.calcite.rel.core.Join; import org.apache.calcite.rel.core.JoinInfo; import org.apache.calcite.rel.logical.LogicalAggregate; +import org.apache.calcite.rel.rules.CoreRules; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; import org.apache.calcite.tools.RelBuilder; diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotEvaluateLiteralRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotEvaluateLiteralRule.java similarity index 99% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotEvaluateLiteralRule.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotEvaluateLiteralRule.java index 9cbaed67734a..5ea790b4084e 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotEvaluateLiteralRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotEvaluateLiteralRule.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; +package org.apache.pinot.calcite.rel.rules; import java.math.BigDecimal; import java.sql.Timestamp; diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotExchangeEliminationRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotExchangeEliminationRule.java similarity index 94% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotExchangeEliminationRule.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotExchangeEliminationRule.java index 308eb83e62a1..e6ba7b9c518f 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotExchangeEliminationRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotExchangeEliminationRule.java @@ -16,15 +16,15 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; +package org.apache.pinot.calcite.rel.rules; import java.util.Collections; import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.plan.RelOptRuleCall; import org.apache.calcite.rel.RelDistribution; import org.apache.calcite.rel.RelNode; -import org.apache.calcite.rel.logical.PinotLogicalExchange; import org.apache.calcite.tools.RelBuilderFactory; +import org.apache.pinot.calcite.rel.logical.PinotLogicalExchange; /** diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotFilterExpandSearchRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotFilterExpandSearchRule.java similarity index 98% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotFilterExpandSearchRule.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotFilterExpandSearchRule.java index 37c2aabc74af..1a7e00f6bbfe 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotFilterExpandSearchRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotFilterExpandSearchRule.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; +package org.apache.pinot.calcite.rel.rules; import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.plan.RelOptRuleCall; diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotJoinExchangeNodeInsertRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotJoinExchangeNodeInsertRule.java similarity index 96% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotJoinExchangeNodeInsertRule.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotJoinExchangeNodeInsertRule.java index a4b64b980040..1b485551f875 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotJoinExchangeNodeInsertRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotJoinExchangeNodeInsertRule.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; +package org.apache.pinot.calcite.rel.rules; import com.google.common.collect.ImmutableList; import org.apache.calcite.plan.RelOptRule; @@ -26,8 +26,8 @@ import org.apache.calcite.rel.core.Join; import org.apache.calcite.rel.core.JoinInfo; import org.apache.calcite.rel.logical.LogicalJoin; -import org.apache.calcite.rel.logical.PinotLogicalExchange; import org.apache.calcite.tools.RelBuilderFactory; +import org.apache.pinot.calcite.rel.logical.PinotLogicalExchange; /** diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotJoinToDynamicBroadcastRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotJoinToDynamicBroadcastRule.java similarity index 96% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotJoinToDynamicBroadcastRule.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotJoinToDynamicBroadcastRule.java index bdbaec23b5e9..dd1cff07cfca 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotJoinToDynamicBroadcastRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotJoinToDynamicBroadcastRule.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; +package org.apache.pinot.calcite.rel.rules; import com.google.common.collect.ImmutableList; import java.util.Collections; @@ -30,12 +30,12 @@ import org.apache.calcite.rel.core.Join; import org.apache.calcite.rel.core.JoinInfo; import org.apache.calcite.rel.core.JoinRelType; -import org.apache.calcite.rel.hint.PinotHintOptions; -import org.apache.calcite.rel.hint.PinotHintStrategyTable; import org.apache.calcite.rel.logical.LogicalJoin; -import org.apache.calcite.rel.logical.PinotLogicalExchange; -import org.apache.calcite.rel.logical.PinotRelExchangeType; import org.apache.calcite.tools.RelBuilderFactory; +import org.apache.pinot.calcite.rel.hint.PinotHintOptions; +import org.apache.pinot.calcite.rel.hint.PinotHintStrategyTable; +import org.apache.pinot.calcite.rel.logical.PinotLogicalExchange; +import org.apache.pinot.calcite.rel.logical.PinotRelExchangeType; import org.apache.zookeeper.common.StringUtils; diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotQueryRuleSets.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotQueryRuleSets.java similarity index 97% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotQueryRuleSets.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotQueryRuleSets.java index df81c67fefd8..cbac4de9e318 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotQueryRuleSets.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotQueryRuleSets.java @@ -16,13 +16,15 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; +package org.apache.pinot.calcite.rel.rules; import com.google.common.collect.ImmutableList; import java.util.Arrays; import java.util.Collection; import org.apache.calcite.adapter.enumerable.EnumerableRules; import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.rel.rules.CoreRules; +import org.apache.calcite.rel.rules.PruneEmptyRules; /** diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotRelDistributionTraitRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotRelDistributionTraitRule.java similarity index 97% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotRelDistributionTraitRule.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotRelDistributionTraitRule.java index 71008e4fd950..529f974be74b 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotRelDistributionTraitRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotRelDistributionTraitRule.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; +package org.apache.pinot.calcite.rel.rules; import com.google.common.collect.ImmutableList; import java.util.ArrayList; @@ -29,16 +29,16 @@ import org.apache.calcite.rel.RelDistributions; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.core.Exchange; -import org.apache.calcite.rel.hint.PinotHintOptions; -import org.apache.calcite.rel.hint.PinotHintStrategyTable; import org.apache.calcite.rel.logical.LogicalAggregate; import org.apache.calcite.rel.logical.LogicalFilter; import org.apache.calcite.rel.logical.LogicalJoin; import org.apache.calcite.rel.logical.LogicalProject; import org.apache.calcite.rel.logical.LogicalTableScan; -import org.apache.calcite.rel.logical.PinotLogicalExchange; import org.apache.calcite.tools.RelBuilderFactory; import org.apache.calcite.util.mapping.Mappings; +import org.apache.pinot.calcite.rel.hint.PinotHintOptions; +import org.apache.pinot.calcite.rel.hint.PinotHintStrategyTable; +import org.apache.pinot.calcite.rel.logical.PinotLogicalExchange; import org.apache.pinot.query.planner.plannode.AggregateNode; diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotRuleUtils.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotRuleUtils.java similarity index 97% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotRuleUtils.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotRuleUtils.java index 264613579726..d134c8af903a 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotRuleUtils.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotRuleUtils.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; +package org.apache.pinot.calcite.rel.rules; import org.apache.calcite.plan.Contexts; import org.apache.calcite.plan.hep.HepRelVertex; @@ -28,12 +28,12 @@ import org.apache.calcite.rel.core.Project; import org.apache.calcite.rel.core.RelFactories; import org.apache.calcite.rel.core.TableScan; -import org.apache.calcite.rel.hint.PinotHintStrategyTable; import org.apache.calcite.rex.RexCall; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql2rel.SqlToRelConverter; import org.apache.calcite.tools.RelBuilder; import org.apache.calcite.tools.RelBuilderFactory; +import org.apache.pinot.calcite.rel.hint.PinotHintStrategyTable; public class PinotRuleUtils { diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotSetOpExchangeNodeInsertRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotSetOpExchangeNodeInsertRule.java similarity index 96% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotSetOpExchangeNodeInsertRule.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotSetOpExchangeNodeInsertRule.java index 9fe933aa4b35..425fe09335cb 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotSetOpExchangeNodeInsertRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotSetOpExchangeNodeInsertRule.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; +package org.apache.pinot.calcite.rel.rules; import java.util.ArrayList; import java.util.List; @@ -30,8 +30,8 @@ import org.apache.calcite.rel.logical.LogicalIntersect; import org.apache.calcite.rel.logical.LogicalMinus; import org.apache.calcite.rel.logical.LogicalUnion; -import org.apache.calcite.rel.logical.PinotLogicalExchange; import org.apache.calcite.tools.RelBuilderFactory; +import org.apache.pinot.calcite.rel.logical.PinotLogicalExchange; /** diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotSingleValueAggregateRemoveRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotSingleValueAggregateRemoveRule.java similarity index 98% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotSingleValueAggregateRemoveRule.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotSingleValueAggregateRemoveRule.java index 0fe1cb10b4b2..29ef71713696 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotSingleValueAggregateRemoveRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotSingleValueAggregateRemoveRule.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; +package org.apache.pinot.calcite.rel.rules; import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.plan.RelOptRuleCall; diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotSortExchangeCopyRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotSortExchangeCopyRule.java similarity index 97% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotSortExchangeCopyRule.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotSortExchangeCopyRule.java index 86310fc7e0c6..9ced39e91a96 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotSortExchangeCopyRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotSortExchangeCopyRule.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; +package org.apache.pinot.calcite.rel.rules; import com.google.common.base.Preconditions; import org.apache.calcite.plan.RelOptRuleCall; @@ -26,13 +26,13 @@ import org.apache.calcite.rel.core.Sort; import org.apache.calcite.rel.core.SortExchange; import org.apache.calcite.rel.logical.LogicalSort; -import org.apache.calcite.rel.logical.PinotLogicalSortExchange; import org.apache.calcite.rel.metadata.RelMdUtil; import org.apache.calcite.rel.metadata.RelMetadataQuery; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.pinot.calcite.rel.logical.PinotLogicalSortExchange; import org.apache.pinot.query.planner.logical.RexExpressionUtils; import org.apache.pinot.query.type.TypeFactory; import org.apache.pinot.query.type.TypeSystem; diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotSortExchangeNodeInsertRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotSortExchangeNodeInsertRule.java similarity index 96% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotSortExchangeNodeInsertRule.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotSortExchangeNodeInsertRule.java index 56d08fd9ed62..2536b0ed01a2 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotSortExchangeNodeInsertRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotSortExchangeNodeInsertRule.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; +package org.apache.pinot.calcite.rel.rules; import java.util.Collections; import org.apache.calcite.plan.RelOptRule; @@ -24,8 +24,8 @@ import org.apache.calcite.rel.RelDistributions; import org.apache.calcite.rel.core.Sort; import org.apache.calcite.rel.logical.LogicalSort; -import org.apache.calcite.rel.logical.PinotLogicalSortExchange; import org.apache.calcite.tools.RelBuilderFactory; +import org.apache.pinot.calcite.rel.logical.PinotLogicalSortExchange; /** diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java similarity index 98% rename from pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java index b3f1176705a5..317a406332ae 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; +package org.apache.pinot.calcite.rel.rules; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; @@ -36,8 +36,6 @@ import org.apache.calcite.rel.core.Window; import org.apache.calcite.rel.logical.LogicalProject; import org.apache.calcite.rel.logical.LogicalWindow; -import org.apache.calcite.rel.logical.PinotLogicalExchange; -import org.apache.calcite.rel.logical.PinotLogicalSortExchange; import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.rex.RexBuilder; @@ -46,6 +44,8 @@ import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.tools.RelBuilderFactory; +import org.apache.pinot.calcite.rel.logical.PinotLogicalExchange; +import org.apache.pinot.calcite.rel.logical.PinotLogicalSortExchange; /** diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/sql/PinotSqlAggFunction.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/PinotSqlAggFunction.java similarity index 91% rename from pinot-query-planner/src/main/java/org/apache/calcite/sql/PinotSqlAggFunction.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/PinotSqlAggFunction.java index 0d4146f4317d..56a6cb7f0e04 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/sql/PinotSqlAggFunction.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/PinotSqlAggFunction.java @@ -16,8 +16,12 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.sql; +package org.apache.pinot.calcite.sql; +import org.apache.calcite.sql.SqlAggFunction; +import org.apache.calcite.sql.SqlFunctionCategory; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.type.SqlOperandTypeChecker; import org.apache.calcite.sql.type.SqlOperandTypeInference; import org.apache.calcite.sql.type.SqlReturnTypeInference; diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/sql/PinotSqlTransformFunction.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/PinotSqlTransformFunction.java similarity index 90% rename from pinot-query-planner/src/main/java/org/apache/calcite/sql/PinotSqlTransformFunction.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/PinotSqlTransformFunction.java index 827c9f37337b..7c97fbf7ae33 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/sql/PinotSqlTransformFunction.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/PinotSqlTransformFunction.java @@ -16,8 +16,11 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.sql; +package org.apache.pinot.calcite.sql; +import org.apache.calcite.sql.SqlFunction; +import org.apache.calcite.sql.SqlFunctionCategory; +import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.type.SqlOperandTypeChecker; import org.apache.calcite.sql.type.SqlOperandTypeInference; import org.apache.calcite.sql.type.SqlReturnTypeInference; diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/sql/fun/PinotOperatorTable.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/fun/PinotOperatorTable.java similarity index 90% rename from pinot-query-planner/src/main/java/org/apache/calcite/sql/fun/PinotOperatorTable.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/fun/PinotOperatorTable.java index 3617a7c06270..1eb1890304a4 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/sql/fun/PinotOperatorTable.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/fun/PinotOperatorTable.java @@ -16,18 +16,19 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.sql.fun; +package org.apache.pinot.calcite.sql.fun; import java.lang.reflect.Field; import java.util.ArrayList; import java.util.List; import java.util.Locale; -import org.apache.calcite.sql.PinotSqlAggFunction; -import org.apache.calcite.sql.PinotSqlTransformFunction; import org.apache.calcite.sql.SqlFunction; import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.validate.SqlNameMatchers; import org.apache.calcite.util.Util; +import org.apache.pinot.calcite.sql.PinotSqlAggFunction; +import org.apache.pinot.calcite.sql.PinotSqlTransformFunction; import org.apache.pinot.common.function.TransformFunctionType; import org.apache.pinot.segment.spi.AggregationFunctionType; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; @@ -87,8 +88,7 @@ public final void initNoDuplicate() { if (op != null && notRegistered(op)) { register(op); } - } else if ( - SqlOperator.class.isAssignableFrom(field.getType())) { + } else if (SqlOperator.class.isAssignableFrom(field.getType())) { SqlOperator op = (SqlOperator) field.get(this); if (op != null && notRegistered(op)) { register(op); @@ -133,9 +133,10 @@ public final void initNoDuplicate() { private void registerAggregateFunction(String functionName, AggregationFunctionType functionType) { // register function behavior that's different from Calcite if (functionType.getOperandTypeChecker() != null && functionType.getReturnTypeInference() != null) { - PinotSqlAggFunction sqlAggFunction = new PinotSqlAggFunction(functionName.toUpperCase(Locale.ROOT), null, - functionType.getSqlKind(), functionType.getReturnTypeInference(), null, - functionType.getOperandTypeChecker(), functionType.getSqlFunctionCategory()); + PinotSqlAggFunction sqlAggFunction = + new PinotSqlAggFunction(functionName.toUpperCase(Locale.ROOT), null, functionType.getSqlKind(), + functionType.getReturnTypeInference(), null, functionType.getOperandTypeChecker(), + functionType.getSqlFunctionCategory()); if (notRegistered(sqlAggFunction)) { register(sqlAggFunction); } @@ -146,9 +147,9 @@ private void registerTransformFunction(String functionName, TransformFunctionTyp // register function behavior that's different from Calcite if (functionType.getOperandTypeChecker() != null && functionType.getReturnTypeInference() != null) { PinotSqlTransformFunction sqlTransformFunction = - new PinotSqlTransformFunction(functionName.toUpperCase(Locale.ROOT), - functionType.getSqlKind(), functionType.getReturnTypeInference(), null, - functionType.getOperandTypeChecker(), functionType.getSqlFunctionCategory()); + new PinotSqlTransformFunction(functionName.toUpperCase(Locale.ROOT), functionType.getSqlKind(), + functionType.getReturnTypeInference(), null, functionType.getOperandTypeChecker(), + functionType.getSqlFunctionCategory()); if (notRegistered(sqlTransformFunction)) { register(sqlTransformFunction); } diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/sql/fun/PinotSqlCoalesceFunction.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/fun/PinotSqlCoalesceFunction.java similarity index 92% rename from pinot-query-planner/src/main/java/org/apache/calcite/sql/fun/PinotSqlCoalesceFunction.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/fun/PinotSqlCoalesceFunction.java index 92ef85857f9a..086c42a57247 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/sql/fun/PinotSqlCoalesceFunction.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/fun/PinotSqlCoalesceFunction.java @@ -16,10 +16,11 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.sql.fun; +package org.apache.pinot.calcite.sql.fun; import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.fun.SqlCoalesceFunction; import org.apache.calcite.sql.validate.SqlValidator; diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/sql/util/PinotChainedSqlOperatorTable.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/util/PinotChainedSqlOperatorTable.java similarity index 83% rename from pinot-query-planner/src/main/java/org/apache/calcite/sql/util/PinotChainedSqlOperatorTable.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/util/PinotChainedSqlOperatorTable.java index 8e09d232d818..8d4db390b712 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/sql/util/PinotChainedSqlOperatorTable.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/util/PinotChainedSqlOperatorTable.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.sql.util; +package org.apache.pinot.calcite.sql.util; import com.google.common.collect.ImmutableList; import java.util.ArrayList; @@ -26,19 +26,23 @@ import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.SqlOperatorTable; import org.apache.calcite.sql.SqlSyntax; +import org.apache.calcite.sql.util.ChainedSqlOperatorTable; +import org.apache.calcite.sql.util.SqlOperatorTables; import org.apache.calcite.sql.validate.SqlNameMatcher; import org.checkerframework.checker.nullness.qual.Nullable; + /** - * ============================================================================ - * THIS CLASS IS COPIED FROM Calcite's {@link org.apache.calcite.sql.util.ChainedSqlOperatorTable} and modified the - * function lookup to terminate early once found from ordered SqlOperatorTable list. This is to avoid some - * hard-coded casting assuming all Sql identifier looked-up are of the same SqlOperator type. - * ============================================================================ + * ================================================================================================================= + * THIS CLASS IS COPIED FROM Calcite's {@link ChainedSqlOperatorTable} and modified the function lookup to terminate + * early once found from ordered SqlOperatorTable list. This is to avoid some hard-coded casting assuming all Sql + * identifier looked-up are of the same SqlOperator type. + * ================================================================================================================= * - * PinotChainedSqlOperatorTable implements the {@link SqlOperatorTable} interface by - * chaining together any number of underlying operator table instances. + * PinotChainedSqlOperatorTable implements the {@link SqlOperatorTable} interface by chaining together any number of + * underlying operator table instances. */ +//@formatter:off public class PinotChainedSqlOperatorTable implements SqlOperatorTable { //~ Instance fields -------------------------------------------------------- @@ -90,3 +94,4 @@ public void add(SqlOperatorTable table) { return list; } } +//@formatter:on diff --git a/pinot-query-planner/src/main/java/org/apache/calcite/sql2rel/PinotConvertletTable.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql2rel/PinotConvertletTable.java similarity index 92% rename from pinot-query-planner/src/main/java/org/apache/calcite/sql2rel/PinotConvertletTable.java rename to pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql2rel/PinotConvertletTable.java index 827810d04f3b..5633d58a6972 100644 --- a/pinot-query-planner/src/main/java/org/apache/calcite/sql2rel/PinotConvertletTable.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql2rel/PinotConvertletTable.java @@ -16,13 +16,17 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.sql2rel; +package org.apache.pinot.calcite.sql2rel; import java.util.List; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.sql2rel.SqlRexContext; +import org.apache.calcite.sql2rel.SqlRexConvertlet; +import org.apache.calcite.sql2rel.SqlRexConvertletTable; +import org.apache.calcite.sql2rel.StandardConvertletTable; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java index 329b2e42d28c..059faac2d43d 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java @@ -35,13 +35,9 @@ import org.apache.calcite.plan.hep.HepMatchOrder; import org.apache.calcite.plan.hep.HepProgram; import org.apache.calcite.plan.hep.HepProgramBuilder; -import org.apache.calcite.prepare.PinotCalciteCatalogReader; import org.apache.calcite.prepare.Prepare; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.RelRoot; -import org.apache.calcite.rel.rules.PinotQueryRuleSets; -import org.apache.calcite.rel.rules.PinotRelDistributionTraitRule; -import org.apache.calcite.rel.rules.PinotRuleUtils; import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.runtime.CalciteContextException; @@ -50,14 +46,18 @@ import org.apache.calcite.sql.SqlExplainLevel; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.fun.PinotOperatorTable; -import org.apache.calcite.sql.util.PinotChainedSqlOperatorTable; -import org.apache.calcite.sql2rel.PinotConvertletTable; import org.apache.calcite.sql2rel.RelDecorrelator; import org.apache.calcite.sql2rel.SqlToRelConverter; import org.apache.calcite.tools.FrameworkConfig; import org.apache.calcite.tools.Frameworks; import org.apache.calcite.tools.RelBuilder; +import org.apache.pinot.calcite.prepare.PinotCalciteCatalogReader; +import org.apache.pinot.calcite.rel.rules.PinotQueryRuleSets; +import org.apache.pinot.calcite.rel.rules.PinotRelDistributionTraitRule; +import org.apache.pinot.calcite.rel.rules.PinotRuleUtils; +import org.apache.pinot.calcite.sql.fun.PinotOperatorTable; +import org.apache.pinot.calcite.sql.util.PinotChainedSqlOperatorTable; +import org.apache.pinot.calcite.sql2rel.PinotConvertletTable; import org.apache.pinot.common.config.provider.TableCache; import org.apache.pinot.query.context.PlannerContext; import org.apache.pinot.query.planner.PlannerUtils; diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java index 7becc2a562d9..cb9630e7244d 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java @@ -32,6 +32,7 @@ import org.apache.calcite.schema.SchemaVersion; import org.apache.calcite.schema.Schemas; import org.apache.calcite.schema.Table; +import org.apache.pinot.calcite.jdbc.CalciteSchemaBuilder; import org.apache.pinot.common.config.provider.TableCache; import org.apache.pinot.common.utils.DatabaseUtils; import org.apache.pinot.spi.utils.builder.TableNameBuilder; @@ -103,7 +104,7 @@ public Set getTypeNames() { /** * {@code PinotCatalog} doesn't need to return function collections b/c they are already registered. - * see: {@link org.apache.calcite.jdbc.CalciteSchemaBuilder#asRootSchema(Schema, String)} + * see: {@link CalciteSchemaBuilder#asRootSchema(Schema, String)} */ @Override public Collection getFunctions(String name) { @@ -112,7 +113,7 @@ public Collection getFunctions(String name) { /** * {@code PinotCatalog} doesn't need to return function name set b/c they are already registered. - * see: {@link org.apache.calcite.jdbc.CalciteSchemaBuilder#asRootSchema(Schema, String)} + * see: {@link CalciteSchemaBuilder#asRootSchema(Schema, String)} */ @Override public Set getFunctionNames() { diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/PinotLogicalQueryPlanner.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/PinotLogicalQueryPlanner.java index 486b78da94e7..4e03059ac1ec 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/PinotLogicalQueryPlanner.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/PinotLogicalQueryPlanner.java @@ -30,7 +30,7 @@ import org.apache.calcite.rel.RelDistribution; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.RelRoot; -import org.apache.calcite.rel.logical.PinotRelExchangeType; +import org.apache.pinot.calcite.rel.logical.PinotRelExchangeType; import org.apache.pinot.query.planner.PlanFragment; import org.apache.pinot.query.planner.QueryPlanMetadata; import org.apache.pinot.query.planner.SubPlan; diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/PlanFragmenter.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/PlanFragmenter.java index 136e1ca73f90..118673aecf1b 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/PlanFragmenter.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/PlanFragmenter.java @@ -24,7 +24,7 @@ import java.util.ArrayList; import java.util.List; import org.apache.calcite.rel.RelDistribution; -import org.apache.calcite.rel.logical.PinotRelExchangeType; +import org.apache.pinot.calcite.rel.logical.PinotRelExchangeType; import org.apache.pinot.query.planner.PlanFragment; import org.apache.pinot.query.planner.SubPlan; import org.apache.pinot.query.planner.plannode.AggregateNode; diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java index 450c2ccc4549..a505ab3102a7 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java @@ -40,13 +40,13 @@ import org.apache.calcite.rel.logical.LogicalTableScan; import org.apache.calcite.rel.logical.LogicalValues; import org.apache.calcite.rel.logical.LogicalWindow; -import org.apache.calcite.rel.logical.PinotLogicalExchange; -import org.apache.calcite.rel.logical.PinotLogicalSortExchange; -import org.apache.calcite.rel.logical.PinotRelExchangeType; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.rel.type.RelRecordType; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.pinot.calcite.rel.logical.PinotLogicalExchange; +import org.apache.pinot.calcite.rel.logical.PinotLogicalSortExchange; +import org.apache.pinot.calcite.rel.logical.PinotRelExchangeType; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; import org.apache.pinot.common.utils.DatabaseUtils; diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/SubPlanFragmenter.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/SubPlanFragmenter.java index 989b52e971c1..e5e17cf05e84 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/SubPlanFragmenter.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/SubPlanFragmenter.java @@ -22,8 +22,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.calcite.rel.logical.PinotRelExchangeType; import org.apache.calcite.runtime.ImmutablePairList; +import org.apache.pinot.calcite.rel.logical.PinotRelExchangeType; import org.apache.pinot.query.planner.SubPlanMetadata; import org.apache.pinot.query.planner.plannode.AggregateNode; import org.apache.pinot.query.planner.plannode.ExchangeNode; diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/physical/DispatchablePlanVisitor.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/physical/DispatchablePlanVisitor.java index 5aebbd1c8e47..d54611ae89c3 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/physical/DispatchablePlanVisitor.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/physical/DispatchablePlanVisitor.java @@ -18,7 +18,7 @@ */ package org.apache.pinot.query.planner.physical; -import org.apache.calcite.rel.hint.PinotHintOptions; +import org.apache.pinot.calcite.rel.hint.PinotHintOptions; import org.apache.pinot.query.planner.plannode.AggregateNode; import org.apache.pinot.query.planner.plannode.ExchangeNode; import org.apache.pinot.query.planner.plannode.FilterNode; diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/AggregateNode.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/AggregateNode.java index 4b31b53d1cfa..e0be21a68b28 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/AggregateNode.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/AggregateNode.java @@ -22,9 +22,9 @@ import java.util.List; import java.util.stream.Collectors; import org.apache.calcite.rel.core.AggregateCall; -import org.apache.calcite.rel.hint.PinotHintOptions; -import org.apache.calcite.rel.hint.PinotHintStrategyTable; import org.apache.calcite.rel.hint.RelHint; +import org.apache.pinot.calcite.rel.hint.PinotHintOptions; +import org.apache.pinot.calcite.rel.hint.PinotHintStrategyTable; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.query.planner.logical.RexExpression; import org.apache.pinot.query.planner.logical.RexExpressionUtils; diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/ExchangeNode.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/ExchangeNode.java index ea98b2fbb354..c6d9024f5ecb 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/ExchangeNode.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/ExchangeNode.java @@ -22,7 +22,7 @@ import java.util.Set; import org.apache.calcite.rel.RelDistribution; import org.apache.calcite.rel.RelFieldCollation; -import org.apache.calcite.rel.logical.PinotRelExchangeType; +import org.apache.pinot.calcite.rel.logical.PinotRelExchangeType; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.query.planner.serde.ProtoProperties; diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/MailboxReceiveNode.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/MailboxReceiveNode.java index e517eee8e555..825a32f9a4e2 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/MailboxReceiveNode.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/MailboxReceiveNode.java @@ -27,8 +27,8 @@ import org.apache.calcite.rel.RelFieldCollation; import org.apache.calcite.rel.RelFieldCollation.Direction; import org.apache.calcite.rel.RelFieldCollation.NullDirection; -import org.apache.calcite.rel.logical.PinotRelExchangeType; import org.apache.commons.collections.CollectionUtils; +import org.apache.pinot.calcite.rel.logical.PinotRelExchangeType; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.query.planner.logical.RexExpression; import org.apache.pinot.query.planner.serde.ProtoProperties; diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/MailboxSendNode.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/MailboxSendNode.java index 534e82effcbb..efc4f20e852f 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/MailboxSendNode.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/MailboxSendNode.java @@ -25,8 +25,8 @@ import javax.annotation.Nullable; import org.apache.calcite.rel.RelDistribution; import org.apache.calcite.rel.RelFieldCollation; -import org.apache.calcite.rel.logical.PinotRelExchangeType; import org.apache.commons.collections.CollectionUtils; +import org.apache.pinot.calcite.rel.logical.PinotRelExchangeType; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.query.planner.logical.RexExpression; import org.apache.pinot.query.planner.serde.ProtoProperties; diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerManager.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerManager.java index 0ecb86ce4b56..d256a19a0098 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerManager.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerManager.java @@ -29,8 +29,8 @@ import java.util.Random; import java.util.Set; import javax.annotation.Nullable; -import org.apache.calcite.rel.hint.PinotHintOptions; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pinot.calcite.rel.hint.PinotHintOptions; import org.apache.pinot.core.routing.RoutingManager; import org.apache.pinot.core.routing.RoutingTable; import org.apache.pinot.core.routing.TablePartitionInfo; diff --git a/pinot-query-planner/src/test/java/org/apache/calcite/rel/rules/PinotSortExchangeCopyRuleTest.java b/pinot-query-planner/src/test/java/org/apache/pinot/calcite/rel/rules/PinotSortExchangeCopyRuleTest.java similarity index 92% rename from pinot-query-planner/src/test/java/org/apache/calcite/rel/rules/PinotSortExchangeCopyRuleTest.java rename to pinot-query-planner/src/test/java/org/apache/pinot/calcite/rel/rules/PinotSortExchangeCopyRuleTest.java index 241b46160553..d470f7a4251c 100644 --- a/pinot-query-planner/src/test/java/org/apache/calcite/rel/rules/PinotSortExchangeCopyRuleTest.java +++ b/pinot-query-planner/src/test/java/org/apache/pinot/calcite/rel/rules/PinotSortExchangeCopyRuleTest.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.calcite.rel.rules; +package org.apache.pinot.calcite.rel.rules; import com.google.common.collect.ImmutableList; import org.apache.calcite.plan.RelOptCluster; @@ -29,11 +29,11 @@ import org.apache.calcite.rel.core.Sort; import org.apache.calcite.rel.core.SortExchange; import org.apache.calcite.rel.logical.LogicalSort; -import org.apache.calcite.rel.logical.PinotLogicalSortExchange; import org.apache.calcite.rel.metadata.RelMetadataQuery; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.pinot.calcite.rel.logical.PinotLogicalSortExchange; import org.apache.pinot.query.type.TypeFactory; import org.apache.pinot.query.type.TypeSystem; import org.mockito.ArgumentCaptor; @@ -81,8 +81,8 @@ public void tearDown() @Test public void shouldMatchLimitNoOffsetNoSort() { // Given: - SortExchange exchange = PinotLogicalSortExchange.create(_input, RelDistributions.SINGLETON, RelCollations.EMPTY, - false, false); + SortExchange exchange = + PinotLogicalSortExchange.create(_input, RelDistributions.SINGLETON, RelCollations.EMPTY, false, false); Sort sort = LogicalSort.create(exchange, RelCollations.EMPTY, null, literal(1)); Mockito.when(_call.rel(0)).thenReturn(sort); Mockito.when(_call.rel(1)).thenReturn(exchange); @@ -109,8 +109,8 @@ public void shouldMatchLimitNoOffsetNoSort() { public void shouldMatchLimitNoOffsetYesSortNoSortEnabled() { // Given: RelCollation collation = RelCollations.of(1); - SortExchange exchange = PinotLogicalSortExchange.create(_input, RelDistributions.SINGLETON, collation, - false, false); + SortExchange exchange = + PinotLogicalSortExchange.create(_input, RelDistributions.SINGLETON, collation, false, false); Sort sort = LogicalSort.create(exchange, collation, null, literal(1)); Mockito.when(_call.rel(0)).thenReturn(sort); Mockito.when(_call.rel(1)).thenReturn(exchange); @@ -137,8 +137,7 @@ public void shouldMatchLimitNoOffsetYesSortNoSortEnabled() { public void shouldMatchLimitNoOffsetYesSortOnSender() { // Given: RelCollation collation = RelCollations.of(1); - SortExchange exchange = PinotLogicalSortExchange.create(_input, RelDistributions.SINGLETON, collation, - true, false); + SortExchange exchange = PinotLogicalSortExchange.create(_input, RelDistributions.SINGLETON, collation, true, false); Sort sort = LogicalSort.create(exchange, collation, null, literal(1)); Mockito.when(_call.rel(0)).thenReturn(sort); Mockito.when(_call.rel(1)).thenReturn(exchange); @@ -165,8 +164,7 @@ public void shouldMatchLimitNoOffsetYesSortOnSender() { public void shouldMatchLimitNoOffsetYesSort() { // Given: RelCollation collation = RelCollations.of(1); - SortExchange exchange = PinotLogicalSortExchange.create(_input, RelDistributions.SINGLETON, collation, false, - true); + SortExchange exchange = PinotLogicalSortExchange.create(_input, RelDistributions.SINGLETON, collation, false, true); Sort sort = LogicalSort.create(exchange, collation, null, literal(1)); Mockito.when(_call.rel(0)).thenReturn(sort); Mockito.when(_call.rel(1)).thenReturn(exchange); @@ -192,8 +190,8 @@ public void shouldMatchLimitNoOffsetYesSort() { @Test public void shouldMatchNoSortAndPushDownLimitPlusOffset() { // Given: - SortExchange exchange = PinotLogicalSortExchange.create(_input, RelDistributions.SINGLETON, RelCollations.EMPTY, - false, true); + SortExchange exchange = + PinotLogicalSortExchange.create(_input, RelDistributions.SINGLETON, RelCollations.EMPTY, false, true); Sort sort = LogicalSort.create(exchange, RelCollations.EMPTY, literal(2), literal(1)); Mockito.when(_call.rel(0)).thenReturn(sort); Mockito.when(_call.rel(1)).thenReturn(exchange); @@ -220,8 +218,7 @@ public void shouldMatchNoSortAndPushDownLimitPlusOffset() { public void shouldMatchSortOnly() { // Given: RelCollation collation = RelCollations.of(1); - SortExchange exchange = PinotLogicalSortExchange.create(_input, RelDistributions.SINGLETON, collation, false, - true); + SortExchange exchange = PinotLogicalSortExchange.create(_input, RelDistributions.SINGLETON, collation, false, true); Sort sort = LogicalSort.create(exchange, collation, null, null); Mockito.when(_call.rel(0)).thenReturn(sort); Mockito.when(_call.rel(1)).thenReturn(exchange); @@ -238,8 +235,7 @@ public void shouldMatchSortOnly() { public void shouldMatchLimitOffsetAndSort() { // Given: RelCollation collation = RelCollations.of(1); - SortExchange exchange = PinotLogicalSortExchange.create(_input, RelDistributions.SINGLETON, collation, false, - true); + SortExchange exchange = PinotLogicalSortExchange.create(_input, RelDistributions.SINGLETON, collation, false, true); Sort sort = LogicalSort.create(exchange, collation, literal(1), literal(2)); Mockito.when(_call.rel(0)).thenReturn(sort); Mockito.when(_call.rel(1)).thenReturn(exchange); @@ -266,8 +262,7 @@ public void shouldMatchLimitOffsetAndSort() { public void shouldNotMatchOnlySortAlreadySorted() { // Given: RelCollation collation = RelCollations.of(1); - SortExchange exchange = PinotLogicalSortExchange.create(_input, RelDistributions.SINGLETON, collation, false, - true); + SortExchange exchange = PinotLogicalSortExchange.create(_input, RelDistributions.SINGLETON, collation, false, true); Sort sort = LogicalSort.create(exchange, collation, null, null); Mockito.when(_call.rel(0)).thenReturn(sort); Mockito.when(_call.rel(1)).thenReturn(exchange); @@ -283,8 +278,8 @@ public void shouldNotMatchOnlySortAlreadySorted() { @Test public void shouldNotMatchOffsetNoLimitNoSort() { // Given: - SortExchange exchange = PinotLogicalSortExchange.create(_input, RelDistributions.SINGLETON, RelCollations.EMPTY, - false, true); + SortExchange exchange = + PinotLogicalSortExchange.create(_input, RelDistributions.SINGLETON, RelCollations.EMPTY, false, true); Sort sort = LogicalSort.create(exchange, RelCollations.EMPTY, literal(1), null); Mockito.when(_call.rel(0)).thenReturn(sort); Mockito.when(_call.rel(1)).thenReturn(exchange); diff --git a/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryEnvironmentTestBase.java b/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryEnvironmentTestBase.java index 615f9bace302..9a97e75b887e 100644 --- a/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryEnvironmentTestBase.java +++ b/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryEnvironmentTestBase.java @@ -29,9 +29,9 @@ import java.util.Map; import java.util.Random; import javax.annotation.Nullable; -import org.apache.calcite.jdbc.CalciteSchemaBuilder; import org.apache.commons.collections.MapUtils; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pinot.calcite.jdbc.CalciteSchemaBuilder; import org.apache.pinot.common.config.provider.TableCache; import org.apache.pinot.core.routing.RoutingManager; import org.apache.pinot.core.routing.TablePartitionInfo; diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java index 39577ba40873..6b8bc3bcc40c 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java @@ -26,8 +26,8 @@ import java.util.List; import java.util.Map; import javax.annotation.Nullable; -import org.apache.calcite.rel.hint.PinotHintOptions; import org.apache.calcite.sql.SqlKind; +import org.apache.pinot.calcite.rel.hint.PinotHintOptions; import org.apache.pinot.common.datablock.DataBlock; import org.apache.pinot.common.datatable.DataTable; import org.apache.pinot.common.request.Literal; diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/HashJoinOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/HashJoinOperator.java index 96645c888275..92d50dc54e49 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/HashJoinOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/HashJoinOperator.java @@ -29,7 +29,7 @@ import java.util.Set; import javax.annotation.Nullable; import org.apache.calcite.rel.core.JoinRelType; -import org.apache.calcite.rel.hint.PinotHintOptions; +import org.apache.pinot.calcite.rel.hint.PinotHintOptions; import org.apache.pinot.common.datablock.DataBlock; import org.apache.pinot.common.datatable.DataTable; import org.apache.pinot.common.exception.QueryException; diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java index a89125d048d1..c199a21276a2 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java @@ -24,7 +24,7 @@ import java.util.List; import java.util.Map; import javax.annotation.Nullable; -import org.apache.calcite.rel.hint.PinotHintOptions; +import org.apache.pinot.calcite.rel.hint.PinotHintOptions; import org.apache.pinot.common.datablock.DataBlock; import org.apache.pinot.common.request.context.ExpressionContext; import org.apache.pinot.common.utils.DataSchema; diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerVisitor.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerVisitor.java index efa97d663818..7c192004ae17 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerVisitor.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerVisitor.java @@ -18,7 +18,7 @@ */ package org.apache.pinot.query.runtime.plan.pipeline; -import org.apache.calcite.rel.logical.PinotRelExchangeType; +import org.apache.pinot.calcite.rel.logical.PinotRelExchangeType; import org.apache.pinot.query.planner.plannode.DefaultPostOrderTraversalVisitor; import org.apache.pinot.query.planner.plannode.MailboxReceiveNode; import org.apache.pinot.query.planner.plannode.PlanNode; diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/AggregateOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/AggregateOperatorTest.java index c1e5255f8576..c207118aaa19 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/AggregateOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/AggregateOperatorTest.java @@ -24,9 +24,9 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import org.apache.calcite.rel.hint.PinotHintOptions; import org.apache.calcite.rel.hint.RelHint; import org.apache.calcite.sql.SqlKind; +import org.apache.pinot.calcite.rel.hint.PinotHintOptions; import org.apache.pinot.common.datatable.DataTable; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/HashJoinOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/HashJoinOperatorTest.java index fa99ce0c01b8..48dbe8ef3e72 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/HashJoinOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/HashJoinOperatorTest.java @@ -27,9 +27,9 @@ import java.util.List; import java.util.Map; import org.apache.calcite.rel.core.JoinRelType; -import org.apache.calcite.rel.hint.PinotHintOptions; import org.apache.calcite.rel.hint.RelHint; import org.apache.calcite.sql.SqlKind; +import org.apache.pinot.calcite.rel.hint.PinotHintOptions; import org.apache.pinot.common.datatable.DataTable; import org.apache.pinot.common.exception.QueryException; import org.apache.pinot.common.utils.DataSchema; diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutorTest.java index 58dcd2106ef7..c3953bef93f2 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutorTest.java @@ -27,7 +27,7 @@ import java.util.concurrent.Executors; import org.apache.calcite.rel.RelDistribution; import org.apache.calcite.rel.core.JoinRelType; -import org.apache.calcite.rel.logical.PinotRelExchangeType; +import org.apache.pinot.calcite.rel.logical.PinotRelExchangeType; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.query.mailbox.MailboxService; import org.apache.pinot.query.mailbox.ReceivingMailbox; From 101af7c5833ecae5b276318c03705714d027ad2a Mon Sep 17 00:00:00 2001 From: Xiaobing <61892277+klsince@users.noreply.github.com> Date: Tue, 9 Apr 2024 12:51:16 -0700 Subject: [PATCH 27/72] add SegmentContext to collect validDocIds bitmaps for many segments together (#12694) * add SegmentContext to collect validDocIds bitmaps for many segments together --- .../data/manager/BaseTableDataManager.java | 10 +++ .../realtime/RealtimeTableDataManager.java | 14 ++++ .../operator/InstanceResponseOperator.java | 12 ++-- .../StreamingInstanceResponseOperator.java | 9 +-- .../AcquireReleaseColumnsSegmentPlanNode.java | 10 +-- .../pinot/core/plan/AggregationPlanNode.java | 13 ++-- .../pinot/core/plan/DistinctPlanNode.java | 9 ++- .../pinot/core/plan/DocIdSetPlanNode.java | 11 ++-- .../pinot/core/plan/FilterPlanNode.java | 34 ++++------ .../pinot/core/plan/GroupByPlanNode.java | 13 ++-- .../core/plan/InstanceResponsePlanNode.java | 10 +-- .../pinot/core/plan/ProjectPlanNode.java | 15 +++-- .../pinot/core/plan/SelectionPlanNode.java | 20 +++--- .../StreamingInstanceResponsePlanNode.java | 8 +-- .../core/plan/StreamingSelectionPlanNode.java | 9 ++- .../plan/maker/InstancePlanMakerImplV2.java | 50 +++++++-------- .../pinot/core/plan/maker/PlanMaker.java | 12 ++-- .../function/AggregationFunctionUtils.java | 23 ++++--- .../executor/ServerQueryExecutorV1Impl.java | 45 +++++++------ .../combine/SelectionCombineOperatorTest.java | 3 +- .../query/SelectionOrderByOperatorTest.java | 6 +- .../StreamingSelectionOnlyOperatorTest.java | 6 +- .../pinot/core/plan/FilterPlanNodeTest.java | 13 ++-- .../org/apache/pinot/core/plan/TestUtils.java | 43 +++++++++++++ ...AndDictionaryAggregationPlanMakerTest.java | 9 ++- .../DictionaryBasedGroupKeyGeneratorTest.java | 5 +- .../aggregation/groupby/GroupByTrimTest.java | 4 +- .../NoDictionaryGroupKeyGeneratorTest.java | 5 +- .../core/startree/v2/BaseStarTreeV2Test.java | 7 +- .../apache/pinot/queries/BaseQueriesTest.java | 19 ++++-- .../pinot/queries/BooleanAggQueriesTest.java | 3 +- .../BenchmarkNativeAndLuceneBasedLike.java | 3 +- .../BenchmarkNativeVsLuceneTextIndex.java | 3 +- .../MockInstanceDataManagerFactory.java | 6 ++ .../local/data/manager/TableDataManager.java | 9 +++ .../BasePartitionUpsertMetadataManager.java | 64 +++++++++++++++++++ ...rentMapPartitionUpsertMetadataManager.java | 34 +--------- ...ncurrentMapTableUpsertMetadataManager.java | 8 +++ .../upsert/TableUpsertMetadataManager.java | 5 ++ .../pinot/segment/spi/SegmentContext.java | 43 +++++++++++++ 40 files changed, 421 insertions(+), 204 deletions(-) create mode 100644 pinot-core/src/test/java/org/apache/pinot/core/plan/TestUtils.java create mode 100644 pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/SegmentContext.java diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index d4b770222f17..c46a85690dc5 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -63,6 +63,8 @@ import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.segment.local.segment.index.loader.LoaderUtils; import org.apache.pinot.segment.spi.ImmutableSegment; +import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.SegmentMetadata; import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; import org.apache.pinot.segment.spi.loader.SegmentDirectoryLoader; @@ -433,6 +435,14 @@ public Map getSegmentErrors() { } } + @Override + public List getSegmentContexts(List selectedSegments, + Map queryOptions) { + List segmentContexts = new ArrayList<>(selectedSegments.size()); + selectedSegments.forEach(s -> segmentContexts.add(new SegmentContext(s))); + return segmentContexts; + } + @Override public void reloadSegment(String segmentName, IndexLoadingConfig indexLoadingConfig, SegmentZKMetadata zkMetadata, SegmentMetadata localMetadata, @Nullable Schema schema, boolean forceDownload) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java index 0c62ab9b4d73..8e50049028e9 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java @@ -23,6 +23,7 @@ import java.io.File; import java.io.IOException; import java.net.URI; +import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; import java.util.List; @@ -46,6 +47,7 @@ import org.apache.pinot.common.utils.LLCSegmentName; import org.apache.pinot.common.utils.SegmentUtils; import org.apache.pinot.common.utils.TarGzCompressionUtils; +import org.apache.pinot.common.utils.config.QueryOptionsUtils; import org.apache.pinot.common.utils.fetcher.SegmentFetcherFactory; import org.apache.pinot.core.data.manager.BaseTableDataManager; import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager; @@ -67,6 +69,7 @@ import org.apache.pinot.segment.local.utils.tablestate.TableStateUtils; import org.apache.pinot.segment.spi.ImmutableSegment; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.spi.config.instance.InstanceDataManagerConfig; import org.apache.pinot.spi.config.table.DedupConfig; import org.apache.pinot.spi.config.table.IndexingConfig; @@ -300,6 +303,17 @@ public void onConsumingToOnline(String segmentNameStr) { _ingestionDelayTracker.markPartitionForVerification(segmentName.getPartitionGroupId()); } + @Override + public List getSegmentContexts(List selectedSegments, + Map queryOptions) { + List segmentContexts = new ArrayList<>(selectedSegments.size()); + selectedSegments.forEach(s -> segmentContexts.add(new SegmentContext(s))); + if (isUpsertEnabled() && !QueryOptionsUtils.isSkipUpsert(queryOptions)) { + _tableUpsertMetadataManager.setSegmentContexts(segmentContexts); + } + return segmentContexts; + } + /** * Returns all partitionGroupIds for the partitions hosted by this server for current table. * @apiNote this involves Zookeeper read and should not be used frequently due to efficiency concerns. diff --git a/pinot-core/src/main/java/org/apache/pinot/core/operator/InstanceResponseOperator.java b/pinot-core/src/main/java/org/apache/pinot/core/operator/InstanceResponseOperator.java index b803526882ad..0ef9bb22071d 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/operator/InstanceResponseOperator.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/operator/InstanceResponseOperator.java @@ -29,7 +29,7 @@ import org.apache.pinot.core.operator.combine.BaseCombineOperator; import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.segment.spi.FetchContext; -import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.spi.accounting.ThreadResourceUsageProvider; import org.apache.pinot.spi.exception.EarlyTerminationException; import org.apache.pinot.spi.exception.QueryCancelledException; @@ -40,15 +40,15 @@ public class InstanceResponseOperator extends BaseOperator _combineOperator; - protected final List _indexSegments; + protected final List _segmentContexts; protected final List _fetchContexts; protected final int _fetchContextSize; protected final QueryContext _queryContext; - public InstanceResponseOperator(BaseCombineOperator combineOperator, List indexSegments, + public InstanceResponseOperator(BaseCombineOperator combineOperator, List segmentContexts, List fetchContexts, QueryContext queryContext) { _combineOperator = combineOperator; - _indexSegments = indexSegments; + _segmentContexts = segmentContexts; _fetchContexts = fetchContexts; _fetchContextSize = fetchContexts.size(); _queryContext = queryContext; @@ -128,13 +128,13 @@ private BaseResultsBlock getCombinedResults() { public void prefetchAll() { for (int i = 0; i < _fetchContextSize; i++) { - _indexSegments.get(i).prefetch(_fetchContexts.get(i)); + _segmentContexts.get(i).getIndexSegment().prefetch(_fetchContexts.get(i)); } } public void releaseAll() { for (int i = 0; i < _fetchContextSize; i++) { - _indexSegments.get(i).release(_fetchContexts.get(i)); + _segmentContexts.get(i).getIndexSegment().release(_fetchContexts.get(i)); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/operator/streaming/StreamingInstanceResponseOperator.java b/pinot-core/src/main/java/org/apache/pinot/core/operator/streaming/StreamingInstanceResponseOperator.java index ef503ae7fff1..61a5b8cc83d9 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/operator/streaming/StreamingInstanceResponseOperator.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/operator/streaming/StreamingInstanceResponseOperator.java @@ -30,7 +30,7 @@ import org.apache.pinot.core.query.executor.ResultsBlockStreamer; import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.segment.spi.FetchContext; -import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.spi.exception.EarlyTerminationException; import org.apache.pinot.spi.exception.QueryCancelledException; import org.apache.pinot.spi.trace.Tracing; @@ -42,9 +42,10 @@ public class StreamingInstanceResponseOperator extends InstanceResponseOperator private final BaseStreamingCombineOperator _streamingCombineOperator; private final ResultsBlockStreamer _streamer; - public StreamingInstanceResponseOperator(BaseCombineOperator combinedOperator, List indexSegments, - List fetchContexts, ResultsBlockStreamer streamer, QueryContext queryContext) { - super(combinedOperator, indexSegments, fetchContexts, queryContext); + public StreamingInstanceResponseOperator(BaseCombineOperator combinedOperator, + List segmentContexts, List fetchContexts, ResultsBlockStreamer streamer, + QueryContext queryContext) { + super(combinedOperator, segmentContexts, fetchContexts, queryContext); _streamingCombineOperator = combinedOperator instanceof BaseStreamingCombineOperator ? (BaseStreamingCombineOperator) combinedOperator : null; diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/AcquireReleaseColumnsSegmentPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/AcquireReleaseColumnsSegmentPlanNode.java index 73406c20fa60..0d9ed602f8ab 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/AcquireReleaseColumnsSegmentPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/AcquireReleaseColumnsSegmentPlanNode.java @@ -20,7 +20,7 @@ import org.apache.pinot.core.operator.AcquireReleaseColumnsSegmentOperator; import org.apache.pinot.segment.spi.FetchContext; -import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; /** @@ -36,13 +36,13 @@ public class AcquireReleaseColumnsSegmentPlanNode implements PlanNode { private final PlanNode _childPlanNode; - private final IndexSegment _indexSegment; + private final SegmentContext _segmentContext; private final FetchContext _fetchContext; - public AcquireReleaseColumnsSegmentPlanNode(PlanNode childPlanNode, IndexSegment indexSegment, + public AcquireReleaseColumnsSegmentPlanNode(PlanNode childPlanNode, SegmentContext segmentContext, FetchContext fetchContext) { _childPlanNode = childPlanNode; - _indexSegment = indexSegment; + _segmentContext = segmentContext; _fetchContext = fetchContext; } @@ -52,6 +52,6 @@ public AcquireReleaseColumnsSegmentPlanNode(PlanNode childPlanNode, IndexSegment */ @Override public AcquireReleaseColumnsSegmentOperator run() { - return new AcquireReleaseColumnsSegmentOperator(_childPlanNode, _indexSegment, _fetchContext); + return new AcquireReleaseColumnsSegmentOperator(_childPlanNode, _segmentContext.getIndexSegment(), _fetchContext); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/AggregationPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/AggregationPlanNode.java index d09e465cf048..2a1321f1b978 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/AggregationPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/AggregationPlanNode.java @@ -34,6 +34,7 @@ import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.segment.spi.AggregationFunctionType; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.datasource.DataSource; import static org.apache.pinot.segment.spi.AggregationFunctionType.*; @@ -56,10 +57,12 @@ public class AggregationPlanNode implements PlanNode { EnumSet.of(COUNT, MIN, MINMV, MAX, MAXMV, MINMAXRANGE, MINMAXRANGEMV); private final IndexSegment _indexSegment; + private final SegmentContext _segmentContext; private final QueryContext _queryContext; - public AggregationPlanNode(IndexSegment indexSegment, QueryContext queryContext) { - _indexSegment = indexSegment; + public AggregationPlanNode(SegmentContext segmentContext, QueryContext queryContext) { + _indexSegment = segmentContext.getIndexSegment(); + _segmentContext = segmentContext; _queryContext = queryContext; } @@ -74,7 +77,7 @@ public Operator run() { */ private FilteredAggregationOperator buildFilteredAggOperator() { return new FilteredAggregationOperator(_queryContext, - AggregationFunctionUtils.buildFilteredAggregationInfos(_indexSegment, _queryContext), + AggregationFunctionUtils.buildFilteredAggregationInfos(_segmentContext, _queryContext), _indexSegment.getSegmentMetadata().getTotalDocs()); } @@ -88,7 +91,7 @@ public Operator buildNonFilteredAggOperator() { assert aggregationFunctions != null; int numTotalDocs = _indexSegment.getSegmentMetadata().getTotalDocs(); - FilterPlanNode filterPlanNode = new FilterPlanNode(_indexSegment, _queryContext); + FilterPlanNode filterPlanNode = new FilterPlanNode(_segmentContext, _queryContext); BaseFilterOperator filterOperator = filterPlanNode.run(); if (!_queryContext.isNullHandlingEnabled()) { @@ -110,7 +113,7 @@ public Operator buildNonFilteredAggOperator() { } AggregationInfo aggregationInfo = - AggregationFunctionUtils.buildAggregationInfo(_indexSegment, _queryContext, aggregationFunctions, + AggregationFunctionUtils.buildAggregationInfo(_segmentContext, _queryContext, aggregationFunctions, _queryContext.getFilter(), filterOperator, filterPlanNode.getPredicateEvaluators()); return new AggregationOperator(_queryContext, aggregationInfo, numTotalDocs); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/DistinctPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/DistinctPlanNode.java index 0ae7e3f31dc6..44ffcd98aa70 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/DistinctPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/DistinctPlanNode.java @@ -27,6 +27,7 @@ import org.apache.pinot.core.operator.query.DistinctOperator; import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.datasource.DataSource; import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader; @@ -36,10 +37,12 @@ */ public class DistinctPlanNode implements PlanNode { private final IndexSegment _indexSegment; + private final SegmentContext _segmentContext; private final QueryContext _queryContext; - public DistinctPlanNode(IndexSegment indexSegment, QueryContext queryContext) { - _indexSegment = indexSegment; + public DistinctPlanNode(SegmentContext segmentContext, QueryContext queryContext) { + _indexSegment = segmentContext.getIndexSegment(); + _segmentContext = segmentContext; _queryContext = queryContext; } @@ -70,7 +73,7 @@ public Operator run() { } BaseProjectOperator projectOperator = - new ProjectPlanNode(_indexSegment, _queryContext, expressions, DocIdSetPlanNode.MAX_DOC_PER_CALL).run(); + new ProjectPlanNode(_segmentContext, _queryContext, expressions, DocIdSetPlanNode.MAX_DOC_PER_CALL).run(); return new DistinctOperator(_indexSegment, _queryContext, projectOperator); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/DocIdSetPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/DocIdSetPlanNode.java index 766d80e74dba..0e3558014b58 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/DocIdSetPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/DocIdSetPlanNode.java @@ -22,22 +22,21 @@ import org.apache.pinot.core.operator.DocIdSetOperator; import org.apache.pinot.core.operator.filter.BaseFilterOperator; import org.apache.pinot.core.query.request.context.QueryContext; -import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; public class DocIdSetPlanNode implements PlanNode { public static final int MAX_DOC_PER_CALL = 10_000; - - private final IndexSegment _indexSegment; + private final SegmentContext _segmentContext; private final QueryContext _queryContext; private final int _maxDocPerCall; private final BaseFilterOperator _filterOperator; - public DocIdSetPlanNode(IndexSegment indexSegment, QueryContext queryContext, int maxDocPerCall, + public DocIdSetPlanNode(SegmentContext segmentContext, QueryContext queryContext, int maxDocPerCall, @Nullable BaseFilterOperator filterOperator) { assert maxDocPerCall > 0 && maxDocPerCall <= MAX_DOC_PER_CALL; - _indexSegment = indexSegment; + _segmentContext = segmentContext; _queryContext = queryContext; _maxDocPerCall = maxDocPerCall; _filterOperator = filterOperator; @@ -46,7 +45,7 @@ public DocIdSetPlanNode(IndexSegment indexSegment, QueryContext queryContext, in @Override public DocIdSetOperator run() { return new DocIdSetOperator( - _filterOperator != null ? _filterOperator : new FilterPlanNode(_indexSegment, _queryContext).run(), + _filterOperator != null ? _filterOperator : new FilterPlanNode(_segmentContext, _queryContext).run(), _maxDocPerCall); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/FilterPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/FilterPlanNode.java index 9a6d17cc9f30..156624922093 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/FilterPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/FilterPlanNode.java @@ -53,8 +53,8 @@ import org.apache.pinot.segment.local.realtime.impl.invertedindex.NativeMutableTextIndex; import org.apache.pinot.segment.local.segment.index.readers.text.NativeTextIndexReader; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.datasource.DataSource; -import org.apache.pinot.segment.spi.index.mutable.ThreadSafeMutableRoaringBitmap; import org.apache.pinot.segment.spi.index.reader.JsonIndexReader; import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader; import org.apache.pinot.segment.spi.index.reader.TextIndexReader; @@ -66,50 +66,40 @@ public class FilterPlanNode implements PlanNode { private final IndexSegment _indexSegment; + private final SegmentContext _segmentContext; private final QueryContext _queryContext; private final FilterContext _filter; // Cache the predicate evaluators private final List> _predicateEvaluators = new ArrayList<>(4); - public FilterPlanNode(IndexSegment indexSegment, QueryContext queryContext) { - this(indexSegment, queryContext, null); + public FilterPlanNode(SegmentContext segmentContext, QueryContext queryContext) { + this(segmentContext, queryContext, null); } - public FilterPlanNode(IndexSegment indexSegment, QueryContext queryContext, @Nullable FilterContext filter) { - _indexSegment = indexSegment; + public FilterPlanNode(SegmentContext segmentContext, QueryContext queryContext, @Nullable FilterContext filter) { + _indexSegment = segmentContext.getIndexSegment(); + _segmentContext = segmentContext; _queryContext = queryContext; _filter = filter != null ? filter : _queryContext.getFilter(); } @Override public BaseFilterOperator run() { - // NOTE: Snapshot the queryableDocIds before reading the numDocs to prevent the latest updates getting lost - MutableRoaringBitmap queryableDocIdSnapshot = null; - if (!_queryContext.isSkipUpsert()) { - ThreadSafeMutableRoaringBitmap queryableDocIds = _indexSegment.getQueryableDocIds(); - if (queryableDocIds != null) { - queryableDocIdSnapshot = queryableDocIds.getMutableRoaringBitmap(); - } else { - ThreadSafeMutableRoaringBitmap validDocIds = _indexSegment.getValidDocIds(); - if (validDocIds != null) { - queryableDocIdSnapshot = validDocIds.getMutableRoaringBitmap(); - } - } - } + MutableRoaringBitmap queryableDocIdsSnapshot = _segmentContext.getQueryableDocIdsSnapshot(); int numDocs = _indexSegment.getSegmentMetadata().getTotalDocs(); if (_filter != null) { BaseFilterOperator filterOperator = constructPhysicalOperator(_filter, numDocs); - if (queryableDocIdSnapshot != null) { - BaseFilterOperator validDocFilter = new BitmapBasedFilterOperator(queryableDocIdSnapshot, false, numDocs); + if (queryableDocIdsSnapshot != null) { + BaseFilterOperator validDocFilter = new BitmapBasedFilterOperator(queryableDocIdsSnapshot, false, numDocs); return FilterOperatorUtils.getAndFilterOperator(_queryContext, Arrays.asList(filterOperator, validDocFilter), numDocs); } else { return filterOperator; } - } else if (queryableDocIdSnapshot != null) { - return new BitmapBasedFilterOperator(queryableDocIdSnapshot, false, numDocs); + } else if (queryableDocIdsSnapshot != null) { + return new BitmapBasedFilterOperator(queryableDocIdsSnapshot, false, numDocs); } else { return new MatchAllFilterOperator(numDocs); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/GroupByPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/GroupByPlanNode.java index 89b1afa5524d..b44b8dc8ee4f 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/GroupByPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/GroupByPlanNode.java @@ -26,6 +26,7 @@ import org.apache.pinot.core.query.aggregation.function.AggregationFunctionUtils; import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; /** @@ -33,10 +34,12 @@ */ public class GroupByPlanNode implements PlanNode { private final IndexSegment _indexSegment; + private final SegmentContext _segmentContext; private final QueryContext _queryContext; - public GroupByPlanNode(IndexSegment indexSegment, QueryContext queryContext) { - _indexSegment = indexSegment; + public GroupByPlanNode(SegmentContext segmentContext, QueryContext queryContext) { + _indexSegment = segmentContext.getIndexSegment(); + _segmentContext = segmentContext; _queryContext = queryContext; } @@ -48,15 +51,15 @@ public Operator run() { private FilteredGroupByOperator buildFilteredGroupByPlan() { return new FilteredGroupByOperator(_queryContext, - AggregationFunctionUtils.buildFilteredAggregationInfos(_indexSegment, _queryContext), + AggregationFunctionUtils.buildFilteredAggregationInfos(_segmentContext, _queryContext), _indexSegment.getSegmentMetadata().getTotalDocs()); } private GroupByOperator buildNonFilteredGroupByPlan() { - FilterPlanNode filterPlanNode = new FilterPlanNode(_indexSegment, _queryContext); + FilterPlanNode filterPlanNode = new FilterPlanNode(_segmentContext, _queryContext); BaseFilterOperator filterOperator = filterPlanNode.run(); AggregationFunctionUtils.AggregationInfo aggregationInfo = - AggregationFunctionUtils.buildAggregationInfo(_indexSegment, _queryContext, + AggregationFunctionUtils.buildAggregationInfo(_segmentContext, _queryContext, _queryContext.getAggregationFunctions(), _queryContext.getFilter(), filterOperator, filterPlanNode.getPredicateEvaluators()); return new GroupByOperator(_queryContext, aggregationInfo, _indexSegment.getSegmentMetadata().getTotalDocs()); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/InstanceResponsePlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/InstanceResponsePlanNode.java index 57b0c18f9e9c..bb600d0c40d9 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/InstanceResponsePlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/InstanceResponsePlanNode.java @@ -22,25 +22,25 @@ import org.apache.pinot.core.operator.InstanceResponseOperator; import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.segment.spi.FetchContext; -import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; public class InstanceResponsePlanNode implements PlanNode { protected final CombinePlanNode _combinePlanNode; - protected final List _indexSegments; + protected final List _segmentContexts; protected final List _fetchContexts; protected final QueryContext _queryContext; - public InstanceResponsePlanNode(CombinePlanNode combinePlanNode, List indexSegments, + public InstanceResponsePlanNode(CombinePlanNode combinePlanNode, List segmentContexts, List fetchContexts, QueryContext queryContext) { _combinePlanNode = combinePlanNode; - _indexSegments = indexSegments; + _segmentContexts = segmentContexts; _fetchContexts = fetchContexts; _queryContext = queryContext; } @Override public InstanceResponseOperator run() { - return new InstanceResponseOperator(_combinePlanNode.run(), _indexSegments, _fetchContexts, _queryContext); + return new InstanceResponseOperator(_combinePlanNode.run(), _segmentContexts, _fetchContexts, _queryContext); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/ProjectPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/ProjectPlanNode.java index 1654a47b1006..6bc763009084 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/ProjectPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/ProjectPlanNode.java @@ -34,6 +34,7 @@ import org.apache.pinot.core.operator.transform.TransformOperator; import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.datasource.DataSource; @@ -42,23 +43,25 @@ */ public class ProjectPlanNode implements PlanNode { private final IndexSegment _indexSegment; + private final SegmentContext _segmentContext; private final QueryContext _queryContext; private final Collection _expressions; private final int _maxDocsPerCall; private final BaseFilterOperator _filterOperator; - public ProjectPlanNode(IndexSegment indexSegment, QueryContext queryContext, + public ProjectPlanNode(SegmentContext segmentContext, QueryContext queryContext, Collection expressions, int maxDocsPerCall, @Nullable BaseFilterOperator filterOperator) { - _indexSegment = indexSegment; + _indexSegment = segmentContext.getIndexSegment(); + _segmentContext = segmentContext; _queryContext = queryContext; _expressions = expressions; _maxDocsPerCall = maxDocsPerCall; _filterOperator = filterOperator; } - public ProjectPlanNode(IndexSegment indexSegment, QueryContext queryContext, + public ProjectPlanNode(SegmentContext segmentContext, QueryContext queryContext, Collection expressions, int maxDocsPerCall) { - this(indexSegment, queryContext, expressions, maxDocsPerCall, null); + this(segmentContext, queryContext, expressions, maxDocsPerCall, null); } @Override @@ -75,8 +78,8 @@ public BaseProjectOperator run() { projectionColumns.forEach(column -> dataSourceMap.put(column, _indexSegment.getDataSource(column))); // NOTE: Skip creating DocIdSetOperator when maxDocsPerCall is 0 (for selection query with LIMIT 0) DocIdSetOperator docIdSetOperator = - _maxDocsPerCall > 0 ? new DocIdSetPlanNode(_indexSegment, _queryContext, _maxDocsPerCall, _filterOperator).run() - : null; + _maxDocsPerCall > 0 ? new DocIdSetPlanNode(_segmentContext, _queryContext, _maxDocsPerCall, + _filterOperator).run() : null; ProjectionOperator projectionOperator = ProjectionOperatorUtils.getProjectionOperator(dataSourceMap, docIdSetOperator); return hasNonIdentifierExpression ? new TransformOperator(_queryContext, projectionOperator, _expressions) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/SelectionPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/SelectionPlanNode.java index 395b4f2ffaf6..e936cd694996 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/SelectionPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/SelectionPlanNode.java @@ -35,6 +35,7 @@ import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.core.query.selection.SelectionOperatorUtils; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.datasource.DataSource; import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader; @@ -44,10 +45,12 @@ */ public class SelectionPlanNode implements PlanNode { private final IndexSegment _indexSegment; + private final SegmentContext _segmentContext; private final QueryContext _queryContext; - public SelectionPlanNode(IndexSegment indexSegment, QueryContext queryContext) { - _indexSegment = indexSegment; + public SelectionPlanNode(SegmentContext segmentContext, QueryContext queryContext) { + _indexSegment = segmentContext.getIndexSegment(); + _segmentContext = segmentContext; _queryContext = queryContext; } @@ -58,7 +61,8 @@ public Operator run() { if (limit == 0) { // Empty selection (LIMIT 0) - BaseProjectOperator projectOperator = new ProjectPlanNode(_indexSegment, _queryContext, expressions, 0).run(); + BaseProjectOperator projectOperator = + new ProjectPlanNode(_segmentContext, _queryContext, expressions, 0).run(); return new EmptySelectionOperator(_indexSegment, _queryContext, expressions, projectOperator); } @@ -68,7 +72,7 @@ public Operator run() { // ie: SELECT ... FROM Table WHERE ... LIMIT 10 int maxDocsPerCall = Math.min(limit, DocIdSetPlanNode.MAX_DOC_PER_CALL); BaseProjectOperator projectOperator = - new ProjectPlanNode(_indexSegment, _queryContext, expressions, maxDocsPerCall).run(); + new ProjectPlanNode(_segmentContext, _queryContext, expressions, maxDocsPerCall).run(); return new SelectionOnlyOperator(_indexSegment, _queryContext, expressions, projectOperator); } int numOrderByExpressions = orderByExpressions.size(); @@ -87,12 +91,12 @@ public Operator run() { maxDocsPerCall = Math.min(limit + _queryContext.getOffset(), DocIdSetPlanNode.MAX_DOC_PER_CALL); } BaseProjectOperator projectOperator = - new ProjectPlanNode(_indexSegment, _queryContext, expressions, maxDocsPerCall).run(); + new ProjectPlanNode(_segmentContext, _queryContext, expressions, maxDocsPerCall).run(); return new SelectionPartiallyOrderedByAscOperator(_indexSegment, _queryContext, expressions, projectOperator, sortedColumnsPrefixSize); } else { BaseProjectOperator projectOperator = - new ProjectPlanNode(_indexSegment, _queryContext, expressions, maxDocsPerCall).run(); + new ProjectPlanNode(_segmentContext, _queryContext, expressions, maxDocsPerCall).run(); return new SelectionPartiallyOrderedByDescOperation(_indexSegment, _queryContext, expressions, projectOperator, sortedColumnsPrefixSize); } @@ -101,7 +105,7 @@ public Operator run() { // All output expressions are ordered // ie: SELECT not_sorted1, not_sorted2 FROM Table WHERE ... ORDER BY not_sorted1, not_sorted2 LIMIT 10 OFFSET 5 BaseProjectOperator projectOperator = - new ProjectPlanNode(_indexSegment, _queryContext, expressions, DocIdSetPlanNode.MAX_DOC_PER_CALL).run(); + new ProjectPlanNode(_segmentContext, _queryContext, expressions, DocIdSetPlanNode.MAX_DOC_PER_CALL).run(); return new SelectionOrderByOperator(_indexSegment, _queryContext, expressions, projectOperator); } // Not all output expressions are ordered, only fetch the order-by expressions and docId to avoid the @@ -111,7 +115,7 @@ public Operator run() { for (OrderByExpressionContext orderByExpression : orderByExpressions) { expressionsToTransform.add(orderByExpression.getExpression()); } - BaseProjectOperator projectOperator = new ProjectPlanNode(_indexSegment, _queryContext, expressionsToTransform, + BaseProjectOperator projectOperator = new ProjectPlanNode(_segmentContext, _queryContext, expressionsToTransform, DocIdSetPlanNode.MAX_DOC_PER_CALL).run(); return new SelectionOrderByOperator(_indexSegment, _queryContext, expressions, projectOperator); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/StreamingInstanceResponsePlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/StreamingInstanceResponsePlanNode.java index 274ca45faf15..52276fa09719 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/StreamingInstanceResponsePlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/StreamingInstanceResponsePlanNode.java @@ -24,21 +24,21 @@ import org.apache.pinot.core.query.executor.ResultsBlockStreamer; import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.segment.spi.FetchContext; -import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; public class StreamingInstanceResponsePlanNode extends InstanceResponsePlanNode { private final ResultsBlockStreamer _streamer; - public StreamingInstanceResponsePlanNode(CombinePlanNode combinePlanNode, List indexSegments, + public StreamingInstanceResponsePlanNode(CombinePlanNode combinePlanNode, List segmentContexts, List fetchContexts, QueryContext queryContext, ResultsBlockStreamer streamer) { - super(combinePlanNode, indexSegments, fetchContexts, queryContext); + super(combinePlanNode, segmentContexts, fetchContexts, queryContext); _streamer = streamer; } @Override public InstanceResponseOperator run() { - return new StreamingInstanceResponseOperator(_combinePlanNode.run(), _indexSegments, _fetchContexts, _streamer, + return new StreamingInstanceResponseOperator(_combinePlanNode.run(), _segmentContexts, _fetchContexts, _streamer, _queryContext); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/StreamingSelectionPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/StreamingSelectionPlanNode.java index 94ab5eef1504..386dff2591ae 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/StreamingSelectionPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/StreamingSelectionPlanNode.java @@ -26,6 +26,7 @@ import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.core.query.selection.SelectionOperatorUtils; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; /** @@ -35,19 +36,21 @@ */ public class StreamingSelectionPlanNode implements PlanNode { private final IndexSegment _indexSegment; + private final SegmentContext _segmentContext; private final QueryContext _queryContext; - public StreamingSelectionPlanNode(IndexSegment indexSegment, QueryContext queryContext) { + public StreamingSelectionPlanNode(SegmentContext segmentContext, QueryContext queryContext) { Preconditions.checkState(queryContext.getOrderByExpressions() == null, "Selection order-by is not supported for streaming"); - _indexSegment = indexSegment; + _indexSegment = segmentContext.getIndexSegment(); + _segmentContext = segmentContext; _queryContext = queryContext; } @Override public StreamingSelectionOnlyOperator run() { List expressions = SelectionOperatorUtils.extractExpressions(_queryContext, _indexSegment); - BaseProjectOperator projectOperator = new ProjectPlanNode(_indexSegment, _queryContext, expressions, + BaseProjectOperator projectOperator = new ProjectPlanNode(_segmentContext, _queryContext, expressions, Math.min(_queryContext.getLimit(), DocIdSetPlanNode.MAX_DOC_PER_CALL)).run(); return new StreamingSelectionOnlyOperator(_indexSegment, _queryContext, expressions, projectOperator); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/InstancePlanMakerImplV2.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/InstancePlanMakerImplV2.java index bf565e68e6f6..54cb034377a5 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/InstancePlanMakerImplV2.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/InstancePlanMakerImplV2.java @@ -53,6 +53,7 @@ import org.apache.pinot.core.util.GroupByUtils; import org.apache.pinot.segment.spi.FetchContext; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.spi.env.PinotConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -133,34 +134,33 @@ public void init(PinotConfiguration queryExecutorConfig) { _minServerGroupTrimSize, _groupByTrimThreshold); } - @Override - public Plan makeInstancePlan(List indexSegments, QueryContext queryContext, + public Plan makeInstancePlan(List segmentContexts, QueryContext queryContext, ExecutorService executorService, ServerMetrics serverMetrics) { applyQueryOptions(queryContext); - int numSegments = indexSegments.size(); + int numSegments = segmentContexts.size(); List planNodes = new ArrayList<>(numSegments); List fetchContexts; - if (queryContext.isEnablePrefetch()) { fetchContexts = new ArrayList<>(numSegments); - for (IndexSegment indexSegment : indexSegments) { - FetchContext fetchContext = _fetchPlanner.planFetchForProcessing(indexSegment, queryContext); + for (SegmentContext segmentContext : segmentContexts) { + FetchContext fetchContext = + _fetchPlanner.planFetchForProcessing(segmentContext.getIndexSegment(), queryContext); fetchContexts.add(fetchContext); planNodes.add( - new AcquireReleaseColumnsSegmentPlanNode(makeSegmentPlanNode(indexSegment, queryContext), indexSegment, + new AcquireReleaseColumnsSegmentPlanNode(makeSegmentPlanNode(segmentContext, queryContext), segmentContext, fetchContext)); } } else { fetchContexts = Collections.emptyList(); - for (IndexSegment indexSegment : indexSegments) { - planNodes.add(makeSegmentPlanNode(indexSegment, queryContext)); + for (SegmentContext segmentContext : segmentContexts) { + planNodes.add(makeSegmentPlanNode(segmentContext, queryContext)); } } CombinePlanNode combinePlanNode = new CombinePlanNode(planNodes, queryContext, executorService, null); return new GlobalPlanImplV0( - new InstanceResponsePlanNode(combinePlanNode, indexSegments, fetchContexts, queryContext)); + new InstanceResponsePlanNode(combinePlanNode, segmentContexts, fetchContexts, queryContext)); } private void applyQueryOptions(QueryContext queryContext) { @@ -231,47 +231,45 @@ private void applyQueryOptions(QueryContext queryContext) { } @Override - public PlanNode makeSegmentPlanNode(IndexSegment indexSegment, QueryContext queryContext) { - rewriteQueryContextWithHints(queryContext, indexSegment); + public PlanNode makeSegmentPlanNode(SegmentContext segmentContext, QueryContext queryContext) { + rewriteQueryContextWithHints(queryContext, segmentContext.getIndexSegment()); if (QueryContextUtils.isAggregationQuery(queryContext)) { List groupByExpressions = queryContext.getGroupByExpressions(); if (groupByExpressions != null) { // Group-by query - return new GroupByPlanNode(indexSegment, queryContext); + return new GroupByPlanNode(segmentContext, queryContext); } else { // Aggregation query - return new AggregationPlanNode(indexSegment, queryContext); + return new AggregationPlanNode(segmentContext, queryContext); } } else if (QueryContextUtils.isSelectionQuery(queryContext)) { - return new SelectionPlanNode(indexSegment, queryContext); + return new SelectionPlanNode(segmentContext, queryContext); } else { assert QueryContextUtils.isDistinctQuery(queryContext); - return new DistinctPlanNode(indexSegment, queryContext); + return new DistinctPlanNode(segmentContext, queryContext); } } - @Override - public Plan makeStreamingInstancePlan(List indexSegments, QueryContext queryContext, + public Plan makeStreamingInstancePlan(List segmentContexts, QueryContext queryContext, ExecutorService executorService, ResultsBlockStreamer streamer, ServerMetrics serverMetrics) { applyQueryOptions(queryContext); - - List planNodes = new ArrayList<>(indexSegments.size()); - for (IndexSegment indexSegment : indexSegments) { - planNodes.add(makeStreamingSegmentPlanNode(indexSegment, queryContext)); + List planNodes = new ArrayList<>(segmentContexts.size()); + for (SegmentContext segmentContext : segmentContexts) { + planNodes.add(makeStreamingSegmentPlanNode(segmentContext, queryContext)); } CombinePlanNode combinePlanNode = new CombinePlanNode(planNodes, queryContext, executorService, streamer); return new GlobalPlanImplV0( - new StreamingInstanceResponsePlanNode(combinePlanNode, indexSegments, Collections.emptyList(), queryContext, + new StreamingInstanceResponsePlanNode(combinePlanNode, segmentContexts, Collections.emptyList(), queryContext, streamer)); } @Override - public PlanNode makeStreamingSegmentPlanNode(IndexSegment indexSegment, QueryContext queryContext) { + public PlanNode makeStreamingSegmentPlanNode(SegmentContext segmentContext, QueryContext queryContext) { if (QueryContextUtils.isSelectionOnlyQuery(queryContext) && queryContext.getLimit() != 0) { // Use streaming operator only for non-empty selection-only query - return new StreamingSelectionPlanNode(indexSegment, queryContext); + return new StreamingSelectionPlanNode(segmentContext, queryContext); } else { - return makeSegmentPlanNode(indexSegment, queryContext); + return makeSegmentPlanNode(segmentContext, queryContext); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/PlanMaker.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/PlanMaker.java index b5f8da91c335..36e62bd86735 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/PlanMaker.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/PlanMaker.java @@ -25,7 +25,7 @@ import org.apache.pinot.core.plan.PlanNode; import org.apache.pinot.core.query.executor.ResultsBlockStreamer; import org.apache.pinot.core.query.request.context.QueryContext; -import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.spi.annotations.InterfaceAudience; import org.apache.pinot.spi.env.PinotConfiguration; @@ -44,24 +44,24 @@ public interface PlanMaker { /** * Returns an instance level {@link Plan} which contains the logical execution plan for multiple segments. */ - Plan makeInstancePlan(List indexSegments, QueryContext queryContext, ExecutorService executorService, - ServerMetrics serverMetrics); + Plan makeInstancePlan(List segmentContexts, QueryContext queryContext, + ExecutorService executorService, ServerMetrics serverMetrics); /** * Returns a segment level {@link PlanNode} which contains the logical execution plan for one segment. */ - PlanNode makeSegmentPlanNode(IndexSegment indexSegment, QueryContext queryContext); + PlanNode makeSegmentPlanNode(SegmentContext segmentContext, QueryContext queryContext); /** * Returns an instance level {@link Plan} for a streaming query which contains the logical execution plan for multiple * segments. */ - Plan makeStreamingInstancePlan(List indexSegments, QueryContext queryContext, + Plan makeStreamingInstancePlan(List segmentContexts, QueryContext queryContext, ExecutorService executorService, ResultsBlockStreamer streamer, ServerMetrics serverMetrics); /** * Returns a segment level {@link PlanNode} for a streaming query which contains the logical execution plan for one * segment. */ - PlanNode makeStreamingSegmentPlanNode(IndexSegment indexSegment, QueryContext queryContext); + PlanNode makeStreamingSegmentPlanNode(SegmentContext segmentContext, QueryContext queryContext); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionUtils.java b/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionUtils.java index 48e5d4784cf3..8d6cbf4aac6f 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionUtils.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionUtils.java @@ -47,7 +47,7 @@ import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.core.startree.StarTreeUtils; import org.apache.pinot.segment.spi.AggregationFunctionType; -import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.index.startree.AggregationFunctionColumnPair; @@ -243,16 +243,15 @@ public boolean isUseStarTree() { /** * Builds {@link AggregationInfo} for aggregations. */ - public static AggregationInfo buildAggregationInfo(IndexSegment indexSegment, QueryContext queryContext, + public static AggregationInfo buildAggregationInfo(SegmentContext segmentContext, QueryContext queryContext, AggregationFunction[] aggregationFunctions, @Nullable FilterContext filter, BaseFilterOperator filterOperator, List> predicateEvaluators) { BaseProjectOperator projectOperator = null; // TODO: Create a short-circuit ProjectOperator when filter result is empty if (!filterOperator.isResultEmpty()) { - projectOperator = - StarTreeUtils.createStarTreeBasedProjectOperator(indexSegment, queryContext, aggregationFunctions, filter, - predicateEvaluators); + projectOperator = StarTreeUtils.createStarTreeBasedProjectOperator(segmentContext.getIndexSegment(), queryContext, + aggregationFunctions, filter, predicateEvaluators); } if (projectOperator != null) { @@ -262,7 +261,7 @@ public static AggregationInfo buildAggregationInfo(IndexSegment indexSegment, Qu AggregationFunctionUtils.collectExpressionsToTransform(aggregationFunctions, queryContext.getGroupByExpressions()); projectOperator = - new ProjectPlanNode(indexSegment, queryContext, expressionsToTransform, DocIdSetPlanNode.MAX_DOC_PER_CALL, + new ProjectPlanNode(segmentContext, queryContext, expressionsToTransform, DocIdSetPlanNode.MAX_DOC_PER_CALL, filterOperator).run(); return new AggregationInfo(aggregationFunctions, projectOperator, false); } @@ -271,11 +270,11 @@ public static AggregationInfo buildAggregationInfo(IndexSegment indexSegment, Qu /** * Builds swim-lanes (list of {@link AggregationInfo}) for filtered aggregations. */ - public static List buildFilteredAggregationInfos(IndexSegment indexSegment, + public static List buildFilteredAggregationInfos(SegmentContext segmentContext, QueryContext queryContext) { assert queryContext.getAggregationFunctions() != null && queryContext.getFilteredAggregationFunctions() != null; - FilterPlanNode mainFilterPlan = new FilterPlanNode(indexSegment, queryContext); + FilterPlanNode mainFilterPlan = new FilterPlanNode(segmentContext, queryContext); BaseFilterOperator mainFilterOperator = mainFilterPlan.run(); List> mainPredicateEvaluators = mainFilterPlan.getPredicateEvaluators(); @@ -285,7 +284,7 @@ public static List buildFilteredAggregationInfos(IndexSegment i Set expressions = collectExpressionsToTransform(aggregationFunctions, queryContext.getGroupByExpressions()); BaseProjectOperator projectOperator = - new ProjectPlanNode(indexSegment, queryContext, expressions, DocIdSetPlanNode.MAX_DOC_PER_CALL, + new ProjectPlanNode(segmentContext, queryContext, expressions, DocIdSetPlanNode.MAX_DOC_PER_CALL, mainFilterOperator).run(); return Collections.singletonList(new AggregationInfo(aggregationFunctions, projectOperator, false)); } @@ -307,7 +306,7 @@ public static List buildFilteredAggregationInfos(IndexSegment i combinedFilter = FilterContext.forAnd(List.of(mainFilter, filter)); } - FilterPlanNode subFilterPlan = new FilterPlanNode(indexSegment, queryContext, filter); + FilterPlanNode subFilterPlan = new FilterPlanNode(segmentContext, queryContext, filter); BaseFilterOperator subFilterOperator = subFilterPlan.run(); BaseFilterOperator combinedFilterOperator; if (mainFilterOperator.isResultMatchingAll() || subFilterOperator.isResultEmpty()) { @@ -342,7 +341,7 @@ public static List buildFilteredAggregationInfos(IndexSegment i AggregationFunction[] aggregationFunctions = filteredAggregationContext._aggregationFunctions.toArray(new AggregationFunction[0]); aggregationInfos.add( - buildAggregationInfo(indexSegment, queryContext, aggregationFunctions, filteredAggregationContext._filter, + buildAggregationInfo(segmentContext, queryContext, aggregationFunctions, filteredAggregationContext._filter, filteredAggregationContext._filterOperator, filteredAggregationContext._predicateEvaluators)); } } @@ -350,7 +349,7 @@ public static List buildFilteredAggregationInfos(IndexSegment i if (!nonFilteredFunctions.isEmpty()) { AggregationFunction[] aggregationFunctions = nonFilteredFunctions.toArray(new AggregationFunction[0]); aggregationInfos.add( - buildAggregationInfo(indexSegment, queryContext, aggregationFunctions, mainFilter, mainFilterOperator, + buildAggregationInfo(segmentContext, queryContext, aggregationFunctions, mainFilter, mainFilterOperator, mainPredicateEvaluators)); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java b/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java index b8c5383bbb69..82664cd57ffe 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java @@ -70,6 +70,7 @@ import org.apache.pinot.segment.spi.ImmutableSegment; import org.apache.pinot.segment.spi.IndexSegment; import org.apache.pinot.segment.spi.MutableSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.SegmentMetadata; import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.env.PinotConfiguration; @@ -256,8 +257,9 @@ private InstanceResponseBlock executeInternal(ServerQueryRequest queryRequest, E InstanceResponseBlock instanceResponse = null; try { - instanceResponse = executeInternal(indexSegments, queryContext, timerContext, executorService, streamer, - queryRequest.isEnableStreaming()); + instanceResponse = + executeInternal(tableDataManager, indexSegments, queryContext, timerContext, executorService, streamer, + queryRequest.isEnableStreaming()); } catch (Exception e) { _serverMetrics.addMeteredTableValue(tableNameWithType, ServerMeter.QUERY_EXECUTION_EXCEPTIONS, 1); instanceResponse = new InstanceResponseBlock(); @@ -344,11 +346,11 @@ private InstanceResponseBlock executeInternal(ServerQueryRequest queryRequest, E } // NOTE: This method might change indexSegments. Do not use it after calling this method. - private InstanceResponseBlock executeInternal(List indexSegments, QueryContext queryContext, - TimerContext timerContext, ExecutorService executorService, @Nullable ResultsBlockStreamer streamer, - boolean enableStreaming) + private InstanceResponseBlock executeInternal(TableDataManager tableDataManager, List indexSegments, + QueryContext queryContext, TimerContext timerContext, ExecutorService executorService, + @Nullable ResultsBlockStreamer streamer, boolean enableStreaming) throws Exception { - handleSubquery(queryContext, indexSegments, timerContext, executorService); + handleSubquery(queryContext, tableDataManager, indexSegments, timerContext, executorService); // Compute total docs for the table before pruning the segments long numTotalDocs = 0; @@ -379,10 +381,12 @@ private InstanceResponseBlock executeInternal(List indexSegments, } } else { TimerContext.Timer planBuildTimer = timerContext.startNewPhaseTimer(ServerQueryPhase.BUILD_QUERY_PLAN); + List selectedSegmentContexts = + tableDataManager.getSegmentContexts(selectedSegments, queryContext.getQueryOptions()); Plan queryPlan = - enableStreaming ? _planMaker.makeStreamingInstancePlan(selectedSegments, queryContext, executorService, + enableStreaming ? _planMaker.makeStreamingInstancePlan(selectedSegmentContexts, queryContext, executorService, streamer, _serverMetrics) - : _planMaker.makeInstancePlan(selectedSegments, queryContext, executorService, _serverMetrics); + : _planMaker.makeInstancePlan(selectedSegmentContexts, queryContext, executorService, _serverMetrics); planBuildTimer.stopAndRecord(); TimerContext.Timer planExecTimer = timerContext.startNewPhaseTimer(ServerQueryPhase.QUERY_PLAN_EXECUTION); @@ -523,12 +527,13 @@ public static InstanceResponseBlock executeExplainQuery(Plan queryPlan, QueryCon * Handles the subquery in the given query. *

    Currently only supports subquery within the filter. */ - private void handleSubquery(QueryContext queryContext, List indexSegments, TimerContext timerContext, - ExecutorService executorService) + private void handleSubquery(QueryContext queryContext, TableDataManager tableDataManager, + List indexSegments, TimerContext timerContext, ExecutorService executorService) throws Exception { FilterContext filter = queryContext.getFilter(); if (filter != null && !filter.isConstant()) { - handleSubquery(filter, indexSegments, timerContext, executorService, queryContext.getEndTimeMs()); + handleSubquery(filter, tableDataManager, indexSegments, timerContext, executorService, + queryContext.getEndTimeMs()); } } @@ -536,16 +541,17 @@ private void handleSubquery(QueryContext queryContext, List indexS * Handles the subquery in the given filter. *

    Currently only supports subquery within the lhs of the predicate. */ - private void handleSubquery(FilterContext filter, List indexSegments, TimerContext timerContext, - ExecutorService executorService, long endTimeMs) + private void handleSubquery(FilterContext filter, TableDataManager tableDataManager, List indexSegments, + TimerContext timerContext, ExecutorService executorService, long endTimeMs) throws Exception { List children = filter.getChildren(); if (children != null) { for (FilterContext child : children) { - handleSubquery(child, indexSegments, timerContext, executorService, endTimeMs); + handleSubquery(child, tableDataManager, indexSegments, timerContext, executorService, endTimeMs); } } else { - handleSubquery(filter.getPredicate().getLhs(), indexSegments, timerContext, executorService, endTimeMs); + handleSubquery(filter.getPredicate().getLhs(), tableDataManager, indexSegments, timerContext, executorService, + endTimeMs); } } @@ -556,8 +562,8 @@ private void handleSubquery(FilterContext filter, List indexSegmen *

    Currently only supports ID_SET subquery within the IN_PARTITIONED_SUBQUERY transform function, which will be * rewritten to an IN_ID_SET transform function. */ - private void handleSubquery(ExpressionContext expression, List indexSegments, TimerContext timerContext, - ExecutorService executorService, long endTimeMs) + private void handleSubquery(ExpressionContext expression, TableDataManager tableDataManager, + List indexSegments, TimerContext timerContext, ExecutorService executorService, long endTimeMs) throws Exception { FunctionContext function = expression.getFunction(); if (function == null) { @@ -584,7 +590,8 @@ private void handleSubquery(ExpressionContext expression, List ind subquery.setEndTimeMs(endTimeMs); // Make a clone of indexSegments because the method might modify the list InstanceResponseBlock instanceResponse = - executeInternal(new ArrayList<>(indexSegments), subquery, timerContext, executorService, null, false); + executeInternal(tableDataManager, new ArrayList<>(indexSegments), subquery, timerContext, executorService, + null, false); BaseResultsBlock resultsBlock = instanceResponse.getResultsBlock(); Preconditions.checkState(resultsBlock instanceof AggregationResultsBlock, "Got unexpected results block type: %s, expecting aggregation results", @@ -598,7 +605,7 @@ private void handleSubquery(ExpressionContext expression, List ind ExpressionContext.forLiteralContext(FieldSpec.DataType.STRING, ((IdSet) result).toBase64String())); } else { for (ExpressionContext argument : arguments) { - handleSubquery(argument, indexSegments, timerContext, executorService, endTimeMs); + handleSubquery(argument, tableDataManager, indexSegments, timerContext, executorService, endTimeMs); } } } diff --git a/pinot-core/src/test/java/org/apache/pinot/core/operator/combine/SelectionCombineOperatorTest.java b/pinot-core/src/test/java/org/apache/pinot/core/operator/combine/SelectionCombineOperatorTest.java index 0b94197b6ade..ef3c0e1e540b 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/operator/combine/SelectionCombineOperatorTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/operator/combine/SelectionCombineOperatorTest.java @@ -38,6 +38,7 @@ import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; @@ -240,7 +241,7 @@ private SelectionResultsBlock getCombineResult(String query) { QueryContext queryContext = QueryContextConverterUtils.getQueryContext(query); List planNodes = new ArrayList<>(NUM_SEGMENTS); for (IndexSegment indexSegment : _indexSegments) { - planNodes.add(PLAN_MAKER.makeSegmentPlanNode(indexSegment, queryContext)); + planNodes.add(PLAN_MAKER.makeSegmentPlanNode(new SegmentContext(indexSegment), queryContext)); } queryContext.setEndTimeMs(System.currentTimeMillis() + Server.DEFAULT_QUERY_EXECUTOR_TIMEOUT_MS); CombinePlanNode combinePlanNode = new CombinePlanNode(planNodes, queryContext, EXECUTOR, null); diff --git a/pinot-core/src/test/java/org/apache/pinot/core/operator/query/SelectionOrderByOperatorTest.java b/pinot-core/src/test/java/org/apache/pinot/core/operator/query/SelectionOrderByOperatorTest.java index 7dddf57bc90d..a99c0412a3b2 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/operator/query/SelectionOrderByOperatorTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/operator/query/SelectionOrderByOperatorTest.java @@ -36,6 +36,7 @@ import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; @@ -131,8 +132,9 @@ public void testPartialSortNullWithoutNullHandling() { private List executeQuery(QueryContext queryContext) { List expressions = SelectionOperatorUtils.extractExpressions(queryContext, _segmentWithNullValues); - BaseProjectOperator projectOperator = new ProjectPlanNode(_segmentWithNullValues, queryContext, expressions, - DocIdSetPlanNode.MAX_DOC_PER_CALL).run(); + BaseProjectOperator projectOperator = + new ProjectPlanNode(new SegmentContext(_segmentWithNullValues), queryContext, expressions, + DocIdSetPlanNode.MAX_DOC_PER_CALL).run(); SelectionOrderByOperator operator = new SelectionOrderByOperator( _segmentWithNullValues, diff --git a/pinot-core/src/test/java/org/apache/pinot/core/operator/streaming/StreamingSelectionOnlyOperatorTest.java b/pinot-core/src/test/java/org/apache/pinot/core/operator/streaming/StreamingSelectionOnlyOperatorTest.java index fb583146ca9b..d21978b1ee4e 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/operator/streaming/StreamingSelectionOnlyOperatorTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/operator/streaming/StreamingSelectionOnlyOperatorTest.java @@ -35,6 +35,7 @@ import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; @@ -76,8 +77,9 @@ public void testNullHandling() { queryContext.setNullHandlingEnabled(true); List expressions = SelectionOperatorUtils.extractExpressions(queryContext, _segmentWithNullValues); - BaseProjectOperator projectOperator = new ProjectPlanNode(_segmentWithNullValues, queryContext, expressions, - DocIdSetPlanNode.MAX_DOC_PER_CALL).run(); + BaseProjectOperator projectOperator = + new ProjectPlanNode(new SegmentContext(_segmentWithNullValues), queryContext, expressions, + DocIdSetPlanNode.MAX_DOC_PER_CALL).run(); StreamingSelectionOnlyOperator operator = new StreamingSelectionOnlyOperator( _segmentWithNullValues, diff --git a/pinot-core/src/test/java/org/apache/pinot/core/plan/FilterPlanNodeTest.java b/pinot-core/src/test/java/org/apache/pinot/core/plan/FilterPlanNodeTest.java index b123f2a807a3..2fe671fa293b 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/plan/FilterPlanNodeTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/plan/FilterPlanNodeTest.java @@ -26,6 +26,7 @@ import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.segment.spi.Constants; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.SegmentMetadata; import org.apache.pinot.segment.spi.index.mutable.ThreadSafeMutableRoaringBitmap; import org.mockito.stubbing.Answer; @@ -48,8 +49,8 @@ public void testConsistentSnapshot() when(segment.getValidDocIds()).thenReturn(bitmap); AtomicInteger numDocs = new AtomicInteger(0); when(meta.getTotalDocs()).then((Answer) invocationOnMock -> numDocs.get()); - QueryContext ctx = mock(QueryContext.class); - when(ctx.getFilter()).thenReturn(null); + QueryContext queryContext = mock(QueryContext.class); + when(queryContext.getFilter()).thenReturn(null); numDocs.set(3); bitmap.add(0); @@ -69,14 +70,16 @@ public void testConsistentSnapshot() // Result should be invariant - always exactly 3 docs for (int i = 0; i < 10_000; i++) { - assertEquals(getNumberOfFilteredDocs(segment, ctx), 3); + SegmentContext segmentContext = new SegmentContext(segment); + segmentContext.setQueryableDocIdsSnapshot(TestUtils.getQueryableDocIdsSnapshotFromSegment(segment)); + assertEquals(getNumberOfFilteredDocs(segmentContext, queryContext), 3); } updater.join(); } - private int getNumberOfFilteredDocs(IndexSegment segment, QueryContext ctx) { - FilterPlanNode node = new FilterPlanNode(segment, ctx); + private int getNumberOfFilteredDocs(SegmentContext segmentContext, QueryContext queryContext) { + FilterPlanNode node = new FilterPlanNode(segmentContext, queryContext); BaseFilterOperator op = node.run(); int numDocsFiltered = 0; FilterBlock block = op.nextBlock(); diff --git a/pinot-core/src/test/java/org/apache/pinot/core/plan/TestUtils.java b/pinot-core/src/test/java/org/apache/pinot/core/plan/TestUtils.java new file mode 100644 index 000000000000..171489e5edf4 --- /dev/null +++ b/pinot-core/src/test/java/org/apache/pinot/core/plan/TestUtils.java @@ -0,0 +1,43 @@ +/** + * 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.core.plan; + +import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.index.mutable.ThreadSafeMutableRoaringBitmap; +import org.roaringbitmap.buffer.MutableRoaringBitmap; + + +public class TestUtils { + private TestUtils() { + } + + public static MutableRoaringBitmap getQueryableDocIdsSnapshotFromSegment(IndexSegment segment) { + MutableRoaringBitmap queryableDocIdsSnapshot = null; + ThreadSafeMutableRoaringBitmap queryableDocIds = segment.getQueryableDocIds(); + if (queryableDocIds != null) { + queryableDocIdsSnapshot = queryableDocIds.getMutableRoaringBitmap(); + } else { + ThreadSafeMutableRoaringBitmap validDocIds = segment.getValidDocIds(); + if (validDocIds != null) { + queryableDocIdsSnapshot = validDocIds.getMutableRoaringBitmap(); + } + } + return queryableDocIdsSnapshot; + } +} diff --git a/pinot-core/src/test/java/org/apache/pinot/core/plan/maker/MetadataAndDictionaryAggregationPlanMakerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/plan/maker/MetadataAndDictionaryAggregationPlanMakerTest.java index cf3e33f9388d..f066b1618b45 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/plan/maker/MetadataAndDictionaryAggregationPlanMakerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/plan/maker/MetadataAndDictionaryAggregationPlanMakerTest.java @@ -32,6 +32,7 @@ import org.apache.pinot.core.operator.query.GroupByOperator; import org.apache.pinot.core.operator.query.NonScanBasedAggregationOperator; import org.apache.pinot.core.operator.query.SelectionOnlyOperator; +import org.apache.pinot.core.plan.TestUtils; import org.apache.pinot.core.query.request.context.QueryContext; import org.apache.pinot.core.query.request.context.utils.QueryContextConverterUtils; import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentImpl; @@ -40,6 +41,7 @@ import org.apache.pinot.segment.local.upsert.ConcurrentMapPartitionUpsertMetadataManager; import org.apache.pinot.segment.local.upsert.UpsertContext; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver; import org.apache.pinot.segment.spi.index.StandardIndexes; @@ -153,9 +155,12 @@ public void deleteSegment() { public void testPlanMaker(String query, Class> operatorClass, Class> upsertOperatorClass) { QueryContext queryContext = QueryContextConverterUtils.getQueryContext(query); - Operator operator = PLAN_MAKER.makeSegmentPlanNode(_indexSegment, queryContext).run(); + Operator operator = PLAN_MAKER.makeSegmentPlanNode(new SegmentContext(_indexSegment), queryContext).run(); assertTrue(operatorClass.isInstance(operator)); - Operator upsertOperator = PLAN_MAKER.makeSegmentPlanNode(_upsertIndexSegment, queryContext).run(); + + SegmentContext segmentContext = new SegmentContext(_upsertIndexSegment); + segmentContext.setQueryableDocIdsSnapshot(TestUtils.getQueryableDocIdsSnapshotFromSegment(_upsertIndexSegment)); + Operator upsertOperator = PLAN_MAKER.makeSegmentPlanNode(segmentContext, queryContext).run(); assertTrue(upsertOperatorClass.isInstance(upsertOperator)); } diff --git a/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/DictionaryBasedGroupKeyGeneratorTest.java b/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/DictionaryBasedGroupKeyGeneratorTest.java index 5e2ae8af3413..79a60bde987a 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/DictionaryBasedGroupKeyGeneratorTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/DictionaryBasedGroupKeyGeneratorTest.java @@ -42,6 +42,7 @@ import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; @@ -151,8 +152,8 @@ private void setup() for (String column : MV_COLUMNS) { expressions.add(ExpressionContext.forIdentifier(column)); } - ProjectPlanNode projectPlanNode = - new ProjectPlanNode(indexSegment, queryContext, expressions, DocIdSetPlanNode.MAX_DOC_PER_CALL); + ProjectPlanNode projectPlanNode = new ProjectPlanNode(new SegmentContext(indexSegment), queryContext, expressions, + DocIdSetPlanNode.MAX_DOC_PER_CALL); _projectOperator = projectPlanNode.run(); _valueBlock = _projectOperator.nextBlock(); } diff --git a/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/GroupByTrimTest.java b/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/GroupByTrimTest.java index dba3faefe6c2..1cebefe52a11 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/GroupByTrimTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/GroupByTrimTest.java @@ -41,6 +41,7 @@ import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.data.FieldSpec; @@ -119,7 +120,8 @@ void testGroupByTrim(QueryContext queryContext, int minSegmentGroupTrimSize, int queryContext.setMinServerGroupTrimSize(minServerGroupTrimSize); // Create a query operator - Operator groupByOperator = new GroupByPlanNode(_indexSegment, queryContext).run(); + Operator groupByOperator = + new GroupByPlanNode(new SegmentContext(_indexSegment), queryContext).run(); GroupByCombineOperator combineOperator = new GroupByCombineOperator(Collections.singletonList(groupByOperator), queryContext, _executorService); diff --git a/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/NoDictionaryGroupKeyGeneratorTest.java b/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/NoDictionaryGroupKeyGeneratorTest.java index 790f062dc703..fa2882c192b2 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/NoDictionaryGroupKeyGeneratorTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/groupby/NoDictionaryGroupKeyGeneratorTest.java @@ -42,6 +42,7 @@ import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; @@ -154,8 +155,8 @@ public void setUp() for (String column : COLUMNS) { expressions.add(ExpressionContext.forIdentifier(column)); } - ProjectPlanNode projectPlanNode = - new ProjectPlanNode(_indexSegment, queryContext, expressions, DocIdSetPlanNode.MAX_DOC_PER_CALL); + ProjectPlanNode projectPlanNode = new ProjectPlanNode(new SegmentContext(_indexSegment), queryContext, expressions, + DocIdSetPlanNode.MAX_DOC_PER_CALL); _projectOperator = projectPlanNode.run(); _valueBlock = _projectOperator.nextBlock(); } diff --git a/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/BaseStarTreeV2Test.java b/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/BaseStarTreeV2Test.java index 14c148bbb271..d4e6d3da4694 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/BaseStarTreeV2Test.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/BaseStarTreeV2Test.java @@ -47,6 +47,7 @@ import org.apache.pinot.segment.spi.AggregationFunctionType; import org.apache.pinot.segment.spi.Constants; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.segment.spi.datasource.DataSource; import org.apache.pinot.segment.spi.index.reader.Dictionary; @@ -230,7 +231,7 @@ public void tearDown() private void testUnsupportedFilter(String query) { QueryContext queryContext = QueryContextConverterUtils.getQueryContext(query); - FilterPlanNode filterPlanNode = new FilterPlanNode(_indexSegment, queryContext); + FilterPlanNode filterPlanNode = new FilterPlanNode(new SegmentContext(_indexSegment), queryContext); filterPlanNode.run(); Map> predicateEvaluatorsMap = StarTreeUtils.extractPredicateEvaluatorsMap(_indexSegment, queryContext.getFilter(), @@ -262,7 +263,7 @@ private void testQuery(String query) List groupByColumns = new ArrayList<>(groupByColumnSet); // Filter - FilterPlanNode filterPlanNode = new FilterPlanNode(_indexSegment, queryContext); + FilterPlanNode filterPlanNode = new FilterPlanNode(new SegmentContext(_indexSegment), queryContext); filterPlanNode.run(); Map> predicateEvaluatorsMap = StarTreeUtils.extractPredicateEvaluatorsMap(_indexSegment, queryContext.getFilter(), @@ -285,7 +286,7 @@ private void testQuery(String query) computeStarTreeResult(starTreeFilterPlanNode, starTreeAggregationColumnReaders, starTreeGroupByColumnReaders); // Extract values without star-tree - FilterPlanNode nonStarTreeFilterPlanNode = new FilterPlanNode(_indexSegment, queryContext); + FilterPlanNode nonStarTreeFilterPlanNode = new FilterPlanNode(new SegmentContext(_indexSegment), queryContext); List nonStarTreeAggregationColumnReaders = new ArrayList<>(numAggregations); List nonStarTreeAggregationColumnDictionaries = new ArrayList<>(numAggregations); for (AggregationFunctionColumnPair aggregationFunctionColumnPair : aggregationFunctionColumnPairs) { diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java index fa5a5068dfae..2746eeb8a1a8 100644 --- a/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java @@ -19,6 +19,7 @@ package org.apache.pinot.queries; import java.io.File; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -49,6 +50,7 @@ import org.apache.pinot.segment.local.segment.index.loader.SegmentPreProcessor; import org.apache.pinot.segment.spi.ImmutableSegment; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.loader.SegmentDirectoryLoaderContext; import org.apache.pinot.segment.spi.loader.SegmentDirectoryLoaderRegistry; import org.apache.pinot.segment.spi.store.SegmentDirectory; @@ -91,7 +93,7 @@ protected T getOperator(String query) { PinotQuery pinotQuery = CalciteSqlParser.compileToPinotQuery(query); PinotQuery serverPinotQuery = GapfillUtils.stripGapfill(pinotQuery); QueryContext queryContext = QueryContextConverterUtils.getQueryContext(serverPinotQuery); - return (T) PLAN_MAKER.makeSegmentPlanNode(getIndexSegment(), queryContext).run(); + return (T) PLAN_MAKER.makeSegmentPlanNode(new SegmentContext(getIndexSegment()), queryContext).run(); } /** @@ -206,7 +208,8 @@ private BrokerResponseNative getBrokerResponse(PinotQuery pinotQuery, PlanMaker // Server side serverQueryContext.setEndTimeMs(System.currentTimeMillis() + Server.DEFAULT_QUERY_EXECUTOR_TIMEOUT_MS); - Plan plan = planMaker.makeInstancePlan(getIndexSegments(), serverQueryContext, EXECUTOR_SERVICE, null); + Plan plan = + planMaker.makeInstancePlan(getSegmentContexts(getIndexSegments()), serverQueryContext, EXECUTOR_SERVICE, null); InstanceResponseBlock instanceResponse; try { instanceResponse = @@ -241,6 +244,12 @@ private BrokerResponseNative getBrokerResponse(PinotQuery pinotQuery, PlanMaker return brokerResponse; } + private static List getSegmentContexts(List indexSegments) { + List segmentContexts = new ArrayList<>(indexSegments.size()); + indexSegments.forEach(s -> segmentContexts.add(new SegmentContext(s))); + return segmentContexts; + } + /** * Run optimized query on multiple index segments. *

    Use this to test the whole flow from server to broker. @@ -297,8 +306,10 @@ private BrokerResponseNative getBrokerResponseDistinctInstances(PinotQuery pinot List> instances = getDistinctInstances(); // Server side serverQueryContext.setEndTimeMs(System.currentTimeMillis() + Server.DEFAULT_QUERY_EXECUTOR_TIMEOUT_MS); - Plan plan1 = planMaker.makeInstancePlan(instances.get(0), serverQueryContext, EXECUTOR_SERVICE, null); - Plan plan2 = planMaker.makeInstancePlan(instances.get(1), serverQueryContext, EXECUTOR_SERVICE, null); + Plan plan1 = + planMaker.makeInstancePlan(getSegmentContexts(instances.get(0)), serverQueryContext, EXECUTOR_SERVICE, null); + Plan plan2 = + planMaker.makeInstancePlan(getSegmentContexts(instances.get(1)), serverQueryContext, EXECUTOR_SERVICE, null); InstanceResponseBlock instanceResponse1; try { diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/BooleanAggQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/BooleanAggQueriesTest.java index 3e87d7446b83..e8913e0a1b56 100644 --- a/pinot-core/src/test/java/org/apache/pinot/queries/BooleanAggQueriesTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/queries/BooleanAggQueriesTest.java @@ -42,6 +42,7 @@ import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.ImmutableSegment; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; @@ -165,7 +166,7 @@ protected T getOperator(String query, boolean enableNullHan PinotQuery serverPinotQuery = GapfillUtils.stripGapfill(pinotQuery); QueryContext queryContext = QueryContextConverterUtils.getQueryContext(serverPinotQuery); - return (T) PLAN_MAKER.makeSegmentPlanNode(getIndexSegment(), queryContext).run(); + return (T) PLAN_MAKER.makeSegmentPlanNode(new SegmentContext(getIndexSegment()), queryContext).run(); } @Test(dataProvider = "nullHandling") diff --git a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeAndLuceneBasedLike.java b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeAndLuceneBasedLike.java index de591f60cf10..4ed28c746f56 100644 --- a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeAndLuceneBasedLike.java +++ b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeAndLuceneBasedLike.java @@ -36,6 +36,7 @@ import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.FSTType; import org.apache.pinot.spi.config.table.FieldConfig; @@ -171,7 +172,7 @@ private void buildSegment(FSTType fstType) @Benchmark @CompilerControl(CompilerControl.Mode.DONT_INLINE) public void query(Blackhole bh) { - Operator operator = _planMaker.makeSegmentPlanNode(_indexSegment, _queryContext).run(); + Operator operator = _planMaker.makeSegmentPlanNode(new SegmentContext(_indexSegment), _queryContext).run(); bh.consume(operator); for (int i = 0; i < _numBlocks; i++) { bh.consume(operator.nextBlock()); diff --git a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeVsLuceneTextIndex.java b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeVsLuceneTextIndex.java index 2b28b6320367..649cc8ebb416 100644 --- a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeVsLuceneTextIndex.java +++ b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeVsLuceneTextIndex.java @@ -39,6 +39,7 @@ import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.ImmutableSegment; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.FSTType; import org.apache.pinot.spi.config.table.FieldConfig; @@ -204,7 +205,7 @@ private ImmutableSegment loadSegment(FSTType fstType) @Benchmark @CompilerControl(CompilerControl.Mode.DONT_INLINE) public void query(Blackhole bh) { - Operator operator = _planMaker.makeSegmentPlanNode(_indexSegment, _queryContext).run(); + Operator operator = _planMaker.makeSegmentPlanNode(new SegmentContext(_indexSegment), _queryContext).run(); for (int i = 0; i < _numBlocks; i++) { bh.consume(operator.nextBlock()); } diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/testutils/MockInstanceDataManagerFactory.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/testutils/MockInstanceDataManagerFactory.java index 6bd3d617b35b..a503c6f96618 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/testutils/MockInstanceDataManagerFactory.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/testutils/MockInstanceDataManagerFactory.java @@ -35,6 +35,7 @@ import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; import org.apache.pinot.segment.spi.ImmutableSegment; import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; @@ -46,6 +47,7 @@ import org.apache.pinot.spi.utils.builder.TableNameBuilder; import static org.mockito.ArgumentMatchers.anyList; +import static org.mockito.ArgumentMatchers.anyMap; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -156,6 +158,10 @@ private TableDataManager mockTableDataManager(List segmentList List segments = invocation.getArgument(0); return segments.stream().map(segmentDataManagerMap::get).collect(Collectors.toList()); }); + when(tableDataManager.getSegmentContexts(anyList(), anyMap())).thenAnswer(invocation -> { + List segments = invocation.getArgument(0); + return segments.stream().map(SegmentContext::new).collect(Collectors.toList()); + }); return tableDataManager; } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java index 70e4c5aa71e0..30dbdffbf0f4 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java @@ -31,6 +31,8 @@ import org.apache.pinot.common.restlet.resources.SegmentErrorInfo; import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.segment.spi.ImmutableSegment; +import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.SegmentMetadata; import org.apache.pinot.spi.config.instance.InstanceDataManagerConfig; import org.apache.pinot.spi.config.table.TableConfig; @@ -238,6 +240,13 @@ default List acquireSegments(List segmentNames, */ Map getSegmentErrors(); + /** + * Get more context for the selected segments for query execution, e.g. getting the validDocIds bitmaps for segments + * in upsert tables. This method allows contexts of many segments to be obtained together, making it easier to + * ensure data consistency across those segments if needed. + */ + List getSegmentContexts(List selectedSegments, Map queryOptions); + /** * Interface to handle segment state transitions from CONSUMING to DROPPED * diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/BasePartitionUpsertMetadataManager.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/BasePartitionUpsertMetadataManager.java index 81e5dbaecd16..f8bacbf6ba02 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/BasePartitionUpsertMetadataManager.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/BasePartitionUpsertMetadataManager.java @@ -33,6 +33,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; @@ -65,6 +66,7 @@ import org.apache.pinot.segment.spi.ImmutableSegment; import org.apache.pinot.segment.spi.IndexSegment; import org.apache.pinot.segment.spi.MutableSegment; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.segment.spi.V1Constants; import org.apache.pinot.segment.spi.index.mutable.ThreadSafeMutableRoaringBitmap; import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths; @@ -1039,6 +1041,68 @@ public synchronized void close() _logger.info("Closed the metadata manager"); } + protected void replaceDocId(ThreadSafeMutableRoaringBitmap validDocIds, + @Nullable ThreadSafeMutableRoaringBitmap queryableDocIds, IndexSegment oldSegment, int oldDocId, int newDocId, + RecordInfo recordInfo) { + removeDocId(oldSegment, oldDocId); + addDocId(validDocIds, queryableDocIds, newDocId, recordInfo); + } + + protected void replaceDocId(ThreadSafeMutableRoaringBitmap validDocIds, + @Nullable ThreadSafeMutableRoaringBitmap queryableDocIds, int oldDocId, int newDocId, RecordInfo recordInfo) { + validDocIds.replace(oldDocId, newDocId); + if (queryableDocIds != null) { + if (recordInfo.isDeleteRecord()) { + queryableDocIds.remove(oldDocId); + } else { + queryableDocIds.replace(oldDocId, newDocId); + } + } + } + + protected void addDocId(ThreadSafeMutableRoaringBitmap validDocIds, + @Nullable ThreadSafeMutableRoaringBitmap queryableDocIds, int docId, RecordInfo recordInfo) { + validDocIds.add(docId); + if (queryableDocIds != null && !recordInfo.isDeleteRecord()) { + queryableDocIds.add(docId); + } + } + + protected void removeDocId(IndexSegment segment, int docId) { + Objects.requireNonNull(segment.getValidDocIds()).remove(docId); + ThreadSafeMutableRoaringBitmap currentQueryableDocIds = segment.getQueryableDocIds(); + if (currentQueryableDocIds != null) { + currentQueryableDocIds.remove(docId); + } + } + + /** + * Use the segmentContexts to collect the contexts for selected segments. Reuse the segmentContext object if + * present, to avoid overwriting the contexts specified at the others places. + */ + public void setSegmentContexts(List segmentContexts) { + for (SegmentContext segmentContext : segmentContexts) { + IndexSegment segment = segmentContext.getIndexSegment(); + if (_trackedSegments.contains(segment)) { + segmentContext.setQueryableDocIdsSnapshot(getQueryableDocIdsSnapshotFromSegment(segment)); + } + } + } + + private static MutableRoaringBitmap getQueryableDocIdsSnapshotFromSegment(IndexSegment segment) { + MutableRoaringBitmap queryableDocIdsSnapshot = null; + ThreadSafeMutableRoaringBitmap queryableDocIds = segment.getQueryableDocIds(); + if (queryableDocIds != null) { + queryableDocIdsSnapshot = queryableDocIds.getMutableRoaringBitmap(); + } else { + ThreadSafeMutableRoaringBitmap validDocIds = segment.getValidDocIds(); + if (validDocIds != null) { + queryableDocIdsSnapshot = validDocIds.getMutableRoaringBitmap(); + } + } + return queryableDocIdsSnapshot; + } + protected void doClose() throws IOException { } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/ConcurrentMapPartitionUpsertMetadataManager.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/ConcurrentMapPartitionUpsertMetadataManager.java index e513e3879eb7..c65f101aca79 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/ConcurrentMapPartitionUpsertMetadataManager.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/ConcurrentMapPartitionUpsertMetadataManager.java @@ -139,8 +139,7 @@ protected void addOrReplaceSegment(ImmutableSegmentImpl segment, ThreadSafeMutab && LLCSegmentName.isLLCSegment(currentSegmentName) && LLCSegmentName.getSequenceNumber(segmentName) > LLCSegmentName.getSequenceNumber( currentSegmentName))) { - removeDocId(currentSegment, currentDocId); - addDocId(validDocIds, queryableDocIds, newDocId, recordInfo); + replaceDocId(validDocIds, queryableDocIds, currentSegment, currentDocId, newDocId, recordInfo); return new RecordLocation(segment, newDocId, newComparisonValue); } else { return currentRecordLocation; @@ -173,34 +172,6 @@ protected void addSegmentWithoutUpsert(ImmutableSegmentImpl segment, ThreadSafeM } } - private static void replaceDocId(ThreadSafeMutableRoaringBitmap validDocIds, - @Nullable ThreadSafeMutableRoaringBitmap queryableDocIds, int oldDocId, int newDocId, RecordInfo recordInfo) { - validDocIds.replace(oldDocId, newDocId); - if (queryableDocIds != null) { - if (recordInfo.isDeleteRecord()) { - queryableDocIds.remove(oldDocId); - } else { - queryableDocIds.replace(oldDocId, newDocId); - } - } - } - - private static void addDocId(ThreadSafeMutableRoaringBitmap validDocIds, - @Nullable ThreadSafeMutableRoaringBitmap queryableDocIds, int docId, RecordInfo recordInfo) { - validDocIds.add(docId); - if (queryableDocIds != null && !recordInfo.isDeleteRecord()) { - queryableDocIds.add(docId); - } - } - - private static void removeDocId(IndexSegment segment, int docId) { - Objects.requireNonNull(segment.getValidDocIds()).remove(docId); - ThreadSafeMutableRoaringBitmap currentQueryableDocIds = segment.getQueryableDocIds(); - if (currentQueryableDocIds != null) { - currentQueryableDocIds.remove(docId); - } - } - @Override protected void removeSegment(IndexSegment segment, MutableRoaringBitmap validDocIds) { assert !validDocIds.isEmpty(); @@ -306,8 +277,7 @@ protected boolean doAddRecord(MutableSegment segment, RecordInfo recordInfo) { if (segment == currentSegment) { replaceDocId(validDocIds, queryableDocIds, currentDocId, newDocId, recordInfo); } else { - removeDocId(currentSegment, currentDocId); - addDocId(validDocIds, queryableDocIds, newDocId, recordInfo); + replaceDocId(validDocIds, queryableDocIds, currentSegment, currentDocId, newDocId, recordInfo); } return new RecordLocation(segment, newDocId, newComparisonValue); } else { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/ConcurrentMapTableUpsertMetadataManager.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/ConcurrentMapTableUpsertMetadataManager.java index 6d48464b692a..4e3e1684028c 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/ConcurrentMapTableUpsertMetadataManager.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/ConcurrentMapTableUpsertMetadataManager.java @@ -20,9 +20,11 @@ import java.io.IOException; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import javax.annotation.concurrent.ThreadSafe; +import org.apache.pinot.segment.spi.SegmentContext; /** @@ -55,6 +57,12 @@ public Map getPartitionToPrimaryKeyCount() { return partitionToPrimaryKeyCount; } + @Override + public void setSegmentContexts(List segmentContexts) { + _partitionMetadataManagerMap.forEach( + (partitionID, upsertMetadataManager) -> upsertMetadataManager.setSegmentContexts(segmentContexts)); + } + @Override public void close() throws IOException { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/TableUpsertMetadataManager.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/TableUpsertMetadataManager.java index a377ab22dba2..921c31dfa31a 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/TableUpsertMetadataManager.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/TableUpsertMetadataManager.java @@ -19,9 +19,11 @@ package org.apache.pinot.segment.local.upsert; import java.io.Closeable; +import java.util.List; import java.util.Map; import javax.annotation.concurrent.ThreadSafe; import org.apache.pinot.segment.local.data.manager.TableDataManager; +import org.apache.pinot.segment.spi.SegmentContext; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.UpsertConfig; import org.apache.pinot.spi.data.Schema; @@ -50,4 +52,7 @@ public interface TableUpsertMetadataManager extends Closeable { * @return A {@code Map} where keys are partition id and values are count of primary keys for that specific partition */ Map getPartitionToPrimaryKeyCount(); + + default void setSegmentContexts(List segmentContexts) { + } } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/SegmentContext.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/SegmentContext.java new file mode 100644 index 000000000000..743a9887693c --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/SegmentContext.java @@ -0,0 +1,43 @@ +/** + * 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.segment.spi; + +import org.roaringbitmap.buffer.MutableRoaringBitmap; + + +public class SegmentContext { + private final IndexSegment _indexSegment; + private MutableRoaringBitmap _queryableDocIdsSnapshot = null; + + public SegmentContext(IndexSegment indexSegment) { + _indexSegment = indexSegment; + } + + public IndexSegment getIndexSegment() { + return _indexSegment; + } + + public MutableRoaringBitmap getQueryableDocIdsSnapshot() { + return _queryableDocIdsSnapshot; + } + + public void setQueryableDocIdsSnapshot(MutableRoaringBitmap queryableDocIdsSnapshot) { + _queryableDocIdsSnapshot = queryableDocIdsSnapshot; + } +} From b5148ebfebc1106241a64701f00feb3bee70ce95 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 9 Apr 2024 13:33:17 -0700 Subject: [PATCH 28/72] Bump cloud.localstack:localstack-utils from 0.2.19 to 0.2.23 (#12842) --- pinot-integration-tests/pom.xml | 2 +- pinot-plugins/pinot-stream-ingestion/pinot-kinesis/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pinot-integration-tests/pom.xml b/pinot-integration-tests/pom.xml index baa472c1efb4..4c8ebc702b5d 100644 --- a/pinot-integration-tests/pom.xml +++ b/pinot-integration-tests/pom.xml @@ -34,7 +34,7 @@ ${basedir}/.. - 0.2.19 + 0.2.23 1.19.7 diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/pom.xml b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/pom.xml index 18e653a6f7eb..1c0b85bc30db 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/pom.xml +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/pom.xml @@ -38,7 +38,7 @@ package 5.2.0 1.0.2 - 0.2.19 + 0.2.23 From c722607743eb79026f72e0c315381f7efcba8531 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 9 Apr 2024 13:33:36 -0700 Subject: [PATCH 29/72] Bump aws.sdk.version from 2.25.17 to 2.25.27 (#12843) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 4a0036f95957..18787a945a74 100644 --- a/pom.xml +++ b/pom.xml @@ -167,7 +167,7 @@ 0.15.0 0.4.4 4.1 - 2.25.17 + 2.25.27 2.12.7 3.1.12 7.9.0 From 73ef57b3afddea604e041586c2f1c9fad40639e2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 9 Apr 2024 13:37:30 -0700 Subject: [PATCH 30/72] Bump org.apache:apache from 21 to 31 (#12846) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 18787a945a74..abd0fe18eda8 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache apache - 21 + 31 org.apache.pinot From abf45fc6e02793954db5025f990254453604b6a3 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 9 Apr 2024 13:39:02 -0700 Subject: [PATCH 31/72] Bump com.google.errorprone:error_prone_annotations from 2.25.0 to 2.26.1 (#12847) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index abd0fe18eda8..7a14c971de0c 100644 --- a/pom.xml +++ b/pom.xml @@ -865,7 +865,7 @@ com.google.errorprone error_prone_annotations - 2.25.0 + 2.26.1 From df5fcb7b0fdd31750e005a3d25c091d72cd34938 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 9 Apr 2024 13:40:24 -0700 Subject: [PATCH 32/72] Bump com.google.apis:google-api-services-storage (#12848) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 7a14c971de0c..b4ce9e4bb199 100644 --- a/pom.xml +++ b/pom.xml @@ -859,7 +859,7 @@ com.google.apis google-api-services-storage - v1-rev20240209-2.0.0 + v1-rev20240319-2.0.0 From e001a884e418ec65d0f6e2c665405afac1cabd6f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 9 Apr 2024 13:43:07 -0700 Subject: [PATCH 33/72] Bump com.gradle:common-custom-user-data-maven-extension from 1.13 to 2 (#12850) --- .mvn/extensions.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.mvn/extensions.xml b/.mvn/extensions.xml index 009c706a2258..8742ba0778ac 100644 --- a/.mvn/extensions.xml +++ b/.mvn/extensions.xml @@ -28,6 +28,6 @@ com.gradle common-custom-user-data-maven-extension - 1.13 + 2 From d9b1f040bf85076d02fd36e32d1e3fe1b8227a89 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 9 Apr 2024 13:43:55 -0700 Subject: [PATCH 34/72] Bump com.microsoft.azure:msal4j from 1.14.3 to 1.15.0 (#12853) --- pinot-plugins/pinot-file-system/pinot-adls/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml index a0e513d53a60..68cfffe5b132 100644 --- a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml +++ b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml @@ -77,7 +77,7 @@ com.microsoft.azure msal4j - 1.14.3 + 1.15.0 com.nimbusds From 3a7cef3155ac69e52cc1829f6d24a1368281d57a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 9 Apr 2024 13:44:14 -0700 Subject: [PATCH 35/72] Bump net.openhft:chronicle-core from 2.25ea10 to 2.25ea13 (#12854) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index b4ce9e4bb199..0028c85bfa76 100644 --- a/pom.xml +++ b/pom.xml @@ -1411,7 +1411,7 @@ net.openhft chronicle-core - 2.25ea10 + 2.25ea13 org.ow2.asm From 07682d24f80b841995370ec80e4ee473b98578f4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 9 Apr 2024 13:44:34 -0700 Subject: [PATCH 36/72] Bump com.azure:azure-identity from 1.11.3 to 1.12.0 (#12855) --- pinot-plugins/pinot-file-system/pinot-adls/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml index 68cfffe5b132..543b60078928 100644 --- a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml +++ b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml @@ -44,7 +44,7 @@ com.azure azure-identity - 1.11.3 + 1.12.0 From 7e4ea06b991397aa68bf94feab98902695f313b0 Mon Sep 17 00:00:00 2001 From: sullis Date: Tue, 9 Apr 2024 14:04:28 -0700 Subject: [PATCH 37/72] use static TypeReference object (#12776) --- .../pinot/common/minion/MinionClient.java | 34 ++++++++++--------- 1 file changed, 18 insertions(+), 16 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/minion/MinionClient.java b/pinot-common/src/main/java/org/apache/pinot/common/minion/MinionClient.java index 3260ced7b362..6e44f55ae168 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/minion/MinionClient.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/minion/MinionClient.java @@ -52,6 +52,9 @@ public class MinionClient { private static final String ACCEPT = "accept"; private static final String APPLICATION_JSON = "application/json"; private static final String HTTP = "http"; + private static final TypeReference> TYPEREF_MAP_STRING_STRING = + new TypeReference>() { + }; private final String _controllerUrl; private final AuthProvider _authProvider; @@ -73,26 +76,25 @@ public Map scheduleMinionTasks(@Nullable String taskType, @Nulla int statusCode = response.getStatusLine().getStatusCode(); final String responseString = EntityUtils.toString(response.getEntity()); if (statusCode >= 400) { - throw new HttpException(String - .format("Unable to schedule minion tasks. Error code %d, Error message: %s", statusCode, responseString)); + throw new HttpException( + String.format("Unable to schedule minion tasks. Error code %d, Error message: %s", statusCode, + responseString)); } - return JsonUtils.stringToObject(responseString, new TypeReference>() { - }); + return JsonUtils.stringToObject(responseString, TYPEREF_MAP_STRING_STRING); } public Map getTasksStates(String taskType) throws IOException, HttpException { - HttpGet httpGet = - createHttpGetRequest(MinionRequestURLBuilder.baseUrl(_controllerUrl).forTasksStates(taskType)); + HttpGet httpGet = createHttpGetRequest(MinionRequestURLBuilder.baseUrl(_controllerUrl).forTasksStates(taskType)); HttpResponse response = HTTP_CLIENT.execute(httpGet); int statusCode = response.getStatusLine().getStatusCode(); final String responseString = IOUtils.toString(response.getEntity().getContent()); if (statusCode >= 400) { - throw new HttpException(String - .format("Unable to get tasks states map. Error code %d, Error message: %s", statusCode, responseString)); + throw new HttpException( + String.format("Unable to get tasks states map. Error code %d, Error message: %s", statusCode, + responseString)); } - return JsonUtils.stringToObject(responseString, new TypeReference>() { - }); + return JsonUtils.stringToObject(responseString, TYPEREF_MAP_STRING_STRING); } public String getTaskState(String taskName) @@ -102,8 +104,8 @@ public String getTaskState(String taskName) int statusCode = response.getStatusLine().getStatusCode(); String responseString = EntityUtils.toString(response.getEntity()); if (statusCode >= 400) { - throw new HttpException(String - .format("Unable to get state for task: %s. Error code %d, Error message: %s", taskName, statusCode, + throw new HttpException( + String.format("Unable to get state for task: %s. Error code %d, Error message: %s", taskName, statusCode, responseString)); } return responseString; @@ -121,11 +123,11 @@ public Map executeTask(AdhocTaskConfig adhocTaskConfig, @Nullabl int statusCode = response.getStatusLine().getStatusCode(); final String responseString = IOUtils.toString(response.getEntity().getContent(), Charset.defaultCharset()); if (statusCode >= 400) { - throw new HttpException(String - .format("Unable to get tasks states map. Error code %d, Error message: %s", statusCode, responseString)); + throw new HttpException( + String.format("Unable to get tasks states map. Error code %d, Error message: %s", statusCode, + responseString)); } - return JsonUtils.stringToObject(responseString, new TypeReference>() { - }); + return JsonUtils.stringToObject(responseString, TYPEREF_MAP_STRING_STRING); } private HttpGet createHttpGetRequest(String uri) { From 425eea9e80dd81c633481488a12c1c62cc7a2c91 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 9 Apr 2024 14:09:28 -0700 Subject: [PATCH 38/72] Bump org.apache.maven.plugins:maven-javadoc-plugin from 3.2.0 to 3.6.3 (#12852) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0028c85bfa76..8b637cd7e8ad 100644 --- a/pom.xml +++ b/pom.xml @@ -1656,7 +1656,7 @@ org.apache.maven.plugins maven-javadoc-plugin - 3.2.0 + 3.6.3 none From c93de377657d8e0f7e3995053bb237347b4f49ea Mon Sep 17 00:00:00 2001 From: sullis Date: Tue, 9 Apr 2024 14:48:05 -0700 Subject: [PATCH 39/72] remove easymock library (#12859) --- .../pinot-kinesis/pom.xml | 6 +- .../stream/kinesis/KinesisConsumerTest.java | 41 ++++---- .../KinesisStreamMetadataProviderTest.java | 94 ++++++++++--------- 3 files changed, 71 insertions(+), 70 deletions(-) diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/pom.xml b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/pom.xml index 1c0b85bc30db..919bd1d19e29 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/pom.xml +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/pom.xml @@ -36,7 +36,6 @@ ${basedir}/../../.. package - 5.2.0 1.0.2 0.2.23 @@ -122,9 +121,8 @@ - org.easymock - easymock - ${easymock.version} + org.mockito + mockito-core test diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerTest.java index 95871cfa368b..04a8f114cc92 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerTest.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisConsumerTest.java @@ -25,7 +25,7 @@ import java.util.Map; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.stream.StreamConfigProperties; -import org.easymock.Capture; +import org.mockito.ArgumentCaptor; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; import software.amazon.awssdk.core.SdkBytes; @@ -38,10 +38,8 @@ import software.amazon.awssdk.services.kinesis.model.Record; import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; -import static org.easymock.EasyMock.capture; -import static org.easymock.EasyMock.createMock; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.replay; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; @@ -80,7 +78,7 @@ private KinesisConfig getKinesisConfig() { @BeforeMethod public void setupTest() { - _kinesisClient = createMock(KinesisClient.class); + _kinesisClient = mock(KinesisClient.class); _kinesisConfig = getKinesisConfig(); _records = new ArrayList<>(NUM_RECORDS); for (int i = 0; i < NUM_RECORDS; i++) { @@ -93,19 +91,18 @@ public void setupTest() { @Test public void testBasicConsumer() { - Capture getRecordsRequestCapture = Capture.newInstance(); - Capture getShardIteratorRequestCapture = Capture.newInstance(); + ArgumentCaptor getRecordsRequestCapture = ArgumentCaptor.forClass(GetRecordsRequest.class); + ArgumentCaptor getShardIteratorRequestCapture = + ArgumentCaptor.forClass(GetShardIteratorRequest.class); GetRecordsResponse getRecordsResponse = GetRecordsResponse.builder().nextShardIterator(null).records(_records).build(); GetShardIteratorResponse getShardIteratorResponse = GetShardIteratorResponse.builder().shardIterator(PLACEHOLDER).build(); - expect(_kinesisClient.getRecords(capture(getRecordsRequestCapture))).andReturn(getRecordsResponse).anyTimes(); - expect(_kinesisClient.getShardIterator(capture(getShardIteratorRequestCapture))).andReturn(getShardIteratorResponse) - .anyTimes(); - - replay(_kinesisClient); + when(_kinesisClient.getRecords(getRecordsRequestCapture.capture())).thenReturn(getRecordsResponse); + when(_kinesisClient.getShardIterator(getShardIteratorRequestCapture.capture())).thenReturn( + getShardIteratorResponse); KinesisConsumer kinesisConsumer = new KinesisConsumer(_kinesisConfig, _kinesisClient); KinesisPartitionGroupOffset startOffset = new KinesisPartitionGroupOffset("0", "1"); @@ -118,6 +115,8 @@ public void testBasicConsumer() { } assertFalse(kinesisMessageBatch.isEndOfPartitionGroup()); + assertEquals(getRecordsRequestCapture.getValue().shardIterator(), "DUMMY"); + assertEquals(getShardIteratorRequestCapture.getValue().shardId(), "0"); } @Test @@ -125,19 +124,18 @@ public void testBasicConsumerWithChildShard() { List shardList = new ArrayList<>(); shardList.add(ChildShard.builder().shardId(PLACEHOLDER).parentShards("0").build()); - Capture getRecordsRequestCapture = Capture.newInstance(); - Capture getShardIteratorRequestCapture = Capture.newInstance(); + ArgumentCaptor getRecordsRequestCapture = ArgumentCaptor.forClass(GetRecordsRequest.class); + ArgumentCaptor getShardIteratorRequestCapture = + ArgumentCaptor.forClass(GetShardIteratorRequest.class); GetRecordsResponse getRecordsResponse = GetRecordsResponse.builder().nextShardIterator(null).records(_records).childShards(shardList).build(); GetShardIteratorResponse getShardIteratorResponse = GetShardIteratorResponse.builder().shardIterator(PLACEHOLDER).build(); - expect(_kinesisClient.getRecords(capture(getRecordsRequestCapture))).andReturn(getRecordsResponse).anyTimes(); - expect(_kinesisClient.getShardIterator(capture(getShardIteratorRequestCapture))).andReturn(getShardIteratorResponse) - .anyTimes(); - - replay(_kinesisClient); + when(_kinesisClient.getRecords(getRecordsRequestCapture.capture())).thenReturn(getRecordsResponse); + when(_kinesisClient.getShardIterator(getShardIteratorRequestCapture.capture())).thenReturn( + getShardIteratorResponse); KinesisConsumer kinesisConsumer = new KinesisConsumer(_kinesisConfig, _kinesisClient); KinesisPartitionGroupOffset startOffset = new KinesisPartitionGroupOffset("0", "1"); @@ -149,6 +147,9 @@ public void testBasicConsumerWithChildShard() { for (int i = 0; i < NUM_RECORDS; i++) { assertEquals(baToString(kinesisMessageBatch.getStreamMessage(i).getValue()), DUMMY_RECORD_PREFIX + i); } + + assertEquals(getRecordsRequestCapture.getValue().shardIterator(), "DUMMY"); + assertEquals(getShardIteratorRequestCapture.getValue().shardId(), "0"); } public String baToString(byte[] bytes) { diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProviderTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProviderTest.java index 833f84d45b03..f536fe78d801 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProviderTest.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/src/test/java/org/apache/pinot/plugin/stream/kinesis/KinesisStreamMetadataProviderTest.java @@ -30,8 +30,7 @@ import org.apache.pinot.spi.stream.StreamConfigProperties; import org.apache.pinot.spi.stream.StreamConsumerFactory; import org.apache.pinot.spi.stream.StreamPartitionMsgOffset; -import org.easymock.Capture; -import org.easymock.CaptureType; +import org.mockito.ArgumentCaptor; import org.testng.Assert; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -39,7 +38,8 @@ import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; -import static org.easymock.EasyMock.*; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class KinesisStreamMetadataProviderTest { @@ -71,9 +71,9 @@ private StreamConfig getStreamConfig() { @BeforeMethod public void setupTest() { - _kinesisConnectionHandler = createMock(KinesisConnectionHandler.class); - _streamConsumerFactory = createMock(StreamConsumerFactory.class); - _partitionGroupConsumer = createNiceMock(PartitionGroupConsumer.class); + _kinesisConnectionHandler = mock(KinesisConnectionHandler.class); + _streamConsumerFactory = mock(StreamConsumerFactory.class); + _partitionGroupConsumer = mock(PartitionGroupConsumer.class); _kinesisStreamMetadataProvider = new KinesisStreamMetadataProvider(CLIENT_ID, getStreamConfig(), _kinesisConnectionHandler, _streamConsumerFactory); @@ -87,11 +87,11 @@ public void getPartitionsGroupInfoListTest() Shard shard1 = Shard.builder().shardId(SHARD_ID_1) .sequenceNumberRange(SequenceNumberRange.builder().startingSequenceNumber("1").build()).build(); - expect(_kinesisConnectionHandler.getShards()).andReturn(ImmutableList.of(shard0, shard1)).anyTimes(); - replay(_kinesisConnectionHandler); + when(_kinesisConnectionHandler.getShards()).thenReturn(ImmutableList.of(shard0, shard1)); - List result = _kinesisStreamMetadataProvider - .computePartitionGroupMetadata(CLIENT_ID, getStreamConfig(), new ArrayList<>(), TIMEOUT); + List result = + _kinesisStreamMetadataProvider.computePartitionGroupMetadata(CLIENT_ID, getStreamConfig(), new ArrayList<>(), + TIMEOUT); Assert.assertEquals(result.size(), 2); Assert.assertEquals(result.get(0).getPartitionGroupId(), 0); @@ -105,32 +105,33 @@ public void getPartitionsGroupInfoEndOfShardTest() KinesisPartitionGroupOffset kinesisPartitionGroupOffset = new KinesisPartitionGroupOffset("0", "1"); - currentPartitionGroupMeta.add(new PartitionGroupConsumptionStatus(0, 1, kinesisPartitionGroupOffset, - kinesisPartitionGroupOffset, "CONSUMING")); + currentPartitionGroupMeta.add( + new PartitionGroupConsumptionStatus(0, 1, kinesisPartitionGroupOffset, kinesisPartitionGroupOffset, + "CONSUMING")); - Capture checkpointArgs = newCapture(CaptureType.ALL); - Capture partitionGroupMetadataCapture = newCapture(CaptureType.ALL); - Capture intArguments = newCapture(CaptureType.ALL); - Capture stringCapture = newCapture(CaptureType.ALL); + ArgumentCaptor checkpointArgs = ArgumentCaptor.forClass(StreamPartitionMsgOffset.class); + ArgumentCaptor partitionGroupMetadataCapture = + ArgumentCaptor.forClass(PartitionGroupConsumptionStatus.class); + ArgumentCaptor intArguments = ArgumentCaptor.forClass(Integer.class); + ArgumentCaptor stringCapture = ArgumentCaptor.forClass(String.class); Shard shard0 = Shard.builder().shardId(SHARD_ID_0).sequenceNumberRange( SequenceNumberRange.builder().startingSequenceNumber("1").endingSequenceNumber("1").build()).build(); - Shard shard1 = Shard.builder().shardId(SHARD_ID_1).sequenceNumberRange( - SequenceNumberRange.builder().startingSequenceNumber("1").build()).build(); - expect(_kinesisConnectionHandler.getShards()).andReturn(ImmutableList.of(shard0, shard1)).anyTimes(); - expect(_streamConsumerFactory - .createPartitionGroupConsumer(capture(stringCapture), capture(partitionGroupMetadataCapture))) - .andReturn(_partitionGroupConsumer).anyTimes(); - expect(_partitionGroupConsumer.fetchMessages(capture(checkpointArgs), captureInt(intArguments))).andReturn( - new KinesisMessageBatch(new ArrayList<>(), kinesisPartitionGroupOffset, true)).anyTimes(); - - replay(_kinesisConnectionHandler, _streamConsumerFactory, _partitionGroupConsumer); + Shard shard1 = Shard.builder().shardId(SHARD_ID_1) + .sequenceNumberRange(SequenceNumberRange.builder().startingSequenceNumber("1").build()).build(); + when(_kinesisConnectionHandler.getShards()).thenReturn(ImmutableList.of(shard0, shard1)); + when(_streamConsumerFactory.createPartitionGroupConsumer(stringCapture.capture(), + partitionGroupMetadataCapture.capture())).thenReturn(_partitionGroupConsumer); + when(_partitionGroupConsumer.fetchMessages(checkpointArgs.capture(), intArguments.capture())).thenReturn( + new KinesisMessageBatch(new ArrayList<>(), kinesisPartitionGroupOffset, true)); - List result = _kinesisStreamMetadataProvider - .computePartitionGroupMetadata(CLIENT_ID, getStreamConfig(), currentPartitionGroupMeta, TIMEOUT); + List result = + _kinesisStreamMetadataProvider.computePartitionGroupMetadata(CLIENT_ID, getStreamConfig(), + currentPartitionGroupMeta, TIMEOUT); Assert.assertEquals(result.size(), 1); Assert.assertEquals(result.get(0).getPartitionGroupId(), 1); + Assert.assertEquals(partitionGroupMetadataCapture.getValue().getSequenceNumber(), 1); } @Test @@ -142,32 +143,33 @@ public void getPartitionsGroupInfoChildShardsest() shardToSequenceMap.put("1", "1"); KinesisPartitionGroupOffset kinesisPartitionGroupOffset = new KinesisPartitionGroupOffset("1", "1"); - currentPartitionGroupMeta.add(new PartitionGroupConsumptionStatus(0, 1, kinesisPartitionGroupOffset, - kinesisPartitionGroupOffset, "CONSUMING")); + currentPartitionGroupMeta.add( + new PartitionGroupConsumptionStatus(0, 1, kinesisPartitionGroupOffset, kinesisPartitionGroupOffset, + "CONSUMING")); - Capture checkpointArgs = newCapture(CaptureType.ALL); - Capture partitionGroupMetadataCapture = newCapture(CaptureType.ALL); - Capture intArguments = newCapture(CaptureType.ALL); - Capture stringCapture = newCapture(CaptureType.ALL); + ArgumentCaptor checkpointArgs = ArgumentCaptor.forClass(StreamPartitionMsgOffset.class); + ArgumentCaptor partitionGroupMetadataCapture = + ArgumentCaptor.forClass(PartitionGroupConsumptionStatus.class); + ArgumentCaptor intArguments = ArgumentCaptor.forClass(Integer.class); + ArgumentCaptor stringCapture = ArgumentCaptor.forClass(String.class); - Shard shard0 = Shard.builder().shardId(SHARD_ID_0).parentShardId(SHARD_ID_1).sequenceNumberRange( - SequenceNumberRange.builder().startingSequenceNumber("1").build()).build(); + Shard shard0 = Shard.builder().shardId(SHARD_ID_0).parentShardId(SHARD_ID_1) + .sequenceNumberRange(SequenceNumberRange.builder().startingSequenceNumber("1").build()).build(); Shard shard1 = Shard.builder().shardId(SHARD_ID_1).sequenceNumberRange( SequenceNumberRange.builder().startingSequenceNumber("1").endingSequenceNumber("1").build()).build(); - expect(_kinesisConnectionHandler.getShards()).andReturn(ImmutableList.of(shard0, shard1)).anyTimes(); - expect(_streamConsumerFactory - .createPartitionGroupConsumer(capture(stringCapture), capture(partitionGroupMetadataCapture))) - .andReturn(_partitionGroupConsumer).anyTimes(); - expect(_partitionGroupConsumer.fetchMessages(capture(checkpointArgs), captureInt(intArguments))).andReturn( - new KinesisMessageBatch(new ArrayList<>(), kinesisPartitionGroupOffset, true)).anyTimes(); - - replay(_kinesisConnectionHandler, _streamConsumerFactory, _partitionGroupConsumer); + when(_kinesisConnectionHandler.getShards()).thenReturn(ImmutableList.of(shard0, shard1)); + when(_streamConsumerFactory.createPartitionGroupConsumer(stringCapture.capture(), + partitionGroupMetadataCapture.capture())).thenReturn(_partitionGroupConsumer); + when(_partitionGroupConsumer.fetchMessages(checkpointArgs.capture(), intArguments.capture())).thenReturn( + new KinesisMessageBatch(new ArrayList<>(), kinesisPartitionGroupOffset, true)); - List result = _kinesisStreamMetadataProvider - .computePartitionGroupMetadata(CLIENT_ID, getStreamConfig(), currentPartitionGroupMeta, TIMEOUT); + List result = + _kinesisStreamMetadataProvider.computePartitionGroupMetadata(CLIENT_ID, getStreamConfig(), + currentPartitionGroupMeta, TIMEOUT); Assert.assertEquals(result.size(), 1); Assert.assertEquals(result.get(0).getPartitionGroupId(), 0); + Assert.assertEquals(partitionGroupMetadataCapture.getValue().getSequenceNumber(), 1); } } From 13673f11508f00ae35f5bb12f9cf97b6897ebfba Mon Sep 17 00:00:00 2001 From: lnbest0707 <106711887+lnbest0707-uber@users.noreply.github.com> Date: Tue, 9 Apr 2024 16:11:06 -0700 Subject: [PATCH 40/72] Add SchemaConformingTransformerV2 to enhance text search abilities (#12788) * Add SchemaConformingTransformerV2 to enhance text search abilities * Fix style * Update __mergedTextIndex field logics * Fix UT * Resolve comments and add fieldPathsToPreserveInput config --- .../pinot/common/metrics/ServerGauge.java | 1 + .../pinot/common/metrics/ServerMeter.java | 1 + .../pinot/queries/TransformQueriesTest.java | 2 +- .../CompositeTransformer.java | 12 +- .../SchemaConformingTransformer.java | 31 +- .../SchemaConformingTransformerV2.java | 727 ++++++++++++++ .../segment/local/utils/Base64Utils.java | 44 + .../segment/local/utils/IngestionUtils.java | 3 +- .../segment/local/utils/TableConfigUtils.java | 8 + .../SchemaConformingTransformerV2Test.java | 934 ++++++++++++++++++ .../segment/local/utils/Base64UtilsTest.java | 96 ++ .../table/ingestion/IngestionConfig.java | 15 + .../SchemaConformingTransformerConfig.java | 4 +- .../SchemaConformingTransformerV2Config.java | 253 +++++ 14 files changed, 2120 insertions(+), 11 deletions(-) create mode 100644 pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SchemaConformingTransformerV2.java create mode 100644 pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/Base64Utils.java create mode 100644 pinot-segment-local/src/test/java/org/apache/pinot/segment/local/recordtransformer/SchemaConformingTransformerV2Test.java create mode 100644 pinot-segment-local/src/test/java/org/apache/pinot/segment/local/utils/Base64UtilsTest.java create mode 100644 pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/SchemaConformingTransformerV2Config.java diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java index 45f34803a036..f0a1fdd1365c 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java @@ -38,6 +38,7 @@ public enum ServerGauge implements AbstractMetrics.Gauge { LAST_REALTIME_SEGMENT_CATCHUP_DURATION_SECONDS("seconds", false), LAST_REALTIME_SEGMENT_COMPLETION_DURATION_SECONDS("seconds", false), REALTIME_OFFHEAP_MEMORY_USED("bytes", false), + REALTIME_MERGED_TEXT_IDX_DOCUMENT_AVG_LEN("bytes", false), REALTIME_SEGMENT_NUM_PARTITIONS("realtimeSegmentNumPartitions", false), LLC_SIMULTANEOUS_SEGMENT_BUILDS("llcSimultaneousSegmentBuilds", true), // Upsert metrics diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerMeter.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerMeter.java index 02005a3814be..ed9769a68e61 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerMeter.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerMeter.java @@ -40,6 +40,7 @@ public enum ServerMeter implements AbstractMetrics.Meter { INVALID_REALTIME_ROWS_DROPPED("rows", false), INCOMPLETE_REALTIME_ROWS_CONSUMED("rows", false), REALTIME_CONSUMPTION_EXCEPTIONS("exceptions", true), + REALTIME_MERGED_TEXT_IDX_TRUNCATED_DOCUMENT_SIZE("bytes", false), REALTIME_OFFSET_COMMITS("commits", true), REALTIME_OFFSET_COMMIT_EXCEPTIONS("exceptions", false), STREAM_CONSUMER_CREATE_EXCEPTIONS("exceptions", false), diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/TransformQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/TransformQueriesTest.java index 1f04d16d3b1e..cfb570d80e0e 100644 --- a/pinot-core/src/test/java/org/apache/pinot/queries/TransformQueriesTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/queries/TransformQueriesTest.java @@ -135,7 +135,7 @@ protected void buildSegment() .setIngestionConfig(new IngestionConfig(null, null, null, null, Arrays.asList(new TransformConfig(M1_V2, "Groovy({INT_COL1_V3 == null || " + "INT_COL1_V3 == Integer.MIN_VALUE ? INT_COL1 : INT_COL1_V3 }, INT_COL1, INT_COL1_V3)")), - null, null, null)) + null, null, null, null)) .build(); Schema schema = new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension(D1, FieldSpec.DataType.STRING) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/CompositeTransformer.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/CompositeTransformer.java index cf88629f1064..a1bfcba52a20 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/CompositeTransformer.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/CompositeTransformer.java @@ -54,8 +54,13 @@ public class CompositeTransformer implements RecordTransformer { * records that have varying fields to a fixed schema without dropping any fields * *

  • - * {@link DataTypeTransformer} after {@link SchemaConformingTransformer} to convert values to comply with the - * schema + * Optional {@link SchemaConformingTransformerV2} after {@link FilterTransformer}, so that we can transform + * input records that have varying fields to a fixed schema and keep or drop other fields by configuration. We + * could also gain enhanced text search capabilities from it. + *
  • + *
  • + * {@link DataTypeTransformer} after {@link SchemaConformingTransformer} or {@link SchemaConformingTransformerV2} + * to convert values to comply with the schema *
  • *
  • * Optional {@link TimeValidationTransformer} after {@link DataTypeTransformer} so that time value is converted to @@ -78,7 +83,8 @@ public class CompositeTransformer implements RecordTransformer { */ public static List getDefaultTransformers(TableConfig tableConfig, Schema schema) { return Stream.of(new ExpressionTransformer(tableConfig, schema), new FilterTransformer(tableConfig), - new SchemaConformingTransformer(tableConfig, schema), new DataTypeTransformer(tableConfig, schema), + new SchemaConformingTransformer(tableConfig, schema), + new SchemaConformingTransformerV2(tableConfig, schema), new DataTypeTransformer(tableConfig, schema), new TimeValidationTransformer(tableConfig, schema), new SpecialValueTransformer(schema), new NullValueTransformer(tableConfig, schema), new SanitizationTransformer(schema)).filter(t -> !t.isNoOp()) .collect(Collectors.toList()); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SchemaConformingTransformer.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SchemaConformingTransformer.java index 0d01e68f35a5..b9cfdce5a8aa 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SchemaConformingTransformer.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SchemaConformingTransformer.java @@ -174,7 +174,7 @@ private static void validateSchemaFieldNames(Set schemaFields, /** * @return The field type for the given extras field */ - private static DataType getAndValidateExtrasFieldType(Schema schema, @Nonnull String extrasFieldName) { + static DataType getAndValidateExtrasFieldType(Schema schema, @Nonnull String extrasFieldName) { FieldSpec fieldSpec = schema.getFieldSpecFor(extrasFieldName); Preconditions.checkState(null != fieldSpec, "Field '%s' doesn't exist in schema", extrasFieldName); DataType fieldDataType = fieldSpec.getDataType(); @@ -250,7 +250,7 @@ private static Map validateSchemaAndCreateTree(@Nonnull Schema s * @param subKeys Returns the sub-keys * @throws IllegalArgumentException if any sub-key is empty */ - private static void getAndValidateSubKeys(String key, int firstKeySeparatorIdx, List subKeys) + static void getAndValidateSubKeys(String key, int firstKeySeparatorIdx, List subKeys) throws IllegalArgumentException { int subKeyBeginIdx = 0; int subKeyEndIdx = firstKeySeparatorIdx; @@ -511,7 +511,16 @@ public void addIndexableEntry(String key, Object value) { if (null == _indexableExtras) { _indexableExtras = new HashMap<>(); } - _indexableExtras.put(key, value); + if (key == null && value instanceof Map) { + // If the key is null, it means that the value is a map that should be merged with the indexable extras + _indexableExtras.putAll((Map) value); + } else if (_indexableExtras.containsKey(key) && _indexableExtras.get(key) instanceof Map && value instanceof Map) { + // If the key already exists in the indexable extras and both the existing value and the new value are maps, + // merge the two maps + ((Map) _indexableExtras.get(key)).putAll((Map) value); + } else { + _indexableExtras.put(key, value); + } } /** @@ -524,7 +533,17 @@ public void addUnindexableEntry(String key, Object value) { if (null == _unindexableExtras) { _unindexableExtras = new HashMap<>(); } - _unindexableExtras.put(key, value); + if (key == null && value instanceof Map) { + // If the key is null, it means that the value is a map that should be merged with the unindexable extras + _unindexableExtras.putAll((Map) value); + } else if (_unindexableExtras.containsKey(key) && _unindexableExtras.get(key) instanceof Map + && value instanceof Map) { + // If the key already exists in the uindexable extras and both the existing value and the new value are maps, + // merge the two maps + ((Map) _unindexableExtras.get(key)).putAll((Map) value); + } else { + _unindexableExtras.put(key, value); + } } /** @@ -542,4 +561,8 @@ public void addChild(String key, ExtraFieldsContainer child) { addUnindexableEntry(key, childUnindexableFields); } } + + public void addChild(ExtraFieldsContainer child) { + addChild(null, child); + } } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SchemaConformingTransformerV2.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SchemaConformingTransformerV2.java new file mode 100644 index 000000000000..5471f784bcf0 --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/SchemaConformingTransformerV2.java @@ -0,0 +1,727 @@ +/** + * 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.segment.local.recordtransformer; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.google.common.base.Preconditions; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Deque; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import org.apache.pinot.common.metrics.ServerGauge; +import org.apache.pinot.common.metrics.ServerMeter; +import org.apache.pinot.common.metrics.ServerMetrics; +import org.apache.pinot.segment.local.utils.Base64Utils; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.ingestion.SchemaConformingTransformerV2Config; +import org.apache.pinot.spi.data.DimensionFieldSpec; +import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.FieldSpec.DataType; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.metrics.PinotMeter; +import org.apache.pinot.spi.stream.StreamDataDecoderImpl; +import org.apache.pinot.spi.utils.JsonUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This transformer evolves from {@link SchemaConformingTransformer} and is designed to support extra cases for + * better text searching: + * - Support over-lapping schema fields, in which case it could support schema column "a" and "a.b" at the same time. + * And it only allows primitive type fields to be the value. + * - Extract flattened key-value pairs as mergedTextIndex for better text searching. + * - Add shingle index tokenization functionality for extremely large text fields. + *

    + * For example, consider this record: + *

    + * {
    + *   "a": 1,
    + *   "b": "2",
    + *   "c": {
    + *     "d": 3,
    + *     "e_noindex": 4,
    + *     "f_noindex": {
    + *       "g": 5
    + *      },
    + *     "x": {
    + *       "y": 9,
    + *       "z_noindex": 10
    + *     }
    + *   }
    + *   "h_noindex": "6",
    + *   "i_noindex": {
    + *     "j": 7,
    + *     "k": 8
    + *   }
    + * }
    + * 
    + * And let's say the table's schema contains these fields: + *
      + *
    • a
    • + *
    • c
    • + *
    • c.d
    • + *
    + *

    + * The record would be transformed into the following (refer to {@link SchemaConformingTransformerV2Config} for + * * default constant values): + *

    + * {
    + *   "a": 1,
    + *   "c": null,
    + *   "c.d": 3,
    + *   "json_data": {
    + *     "b": "2",
    + *     "c": {
    + *       "x": {
    + *         "y": 9
    + *       }
    + *     }
    + *   }
    + *   "json_data_no_idx": {
    + *     "c": {
    + *       "e_noindex": 4,
    + *       "f_noindex": {
    + *         "g": 5
    + *       },
    + *       "x": {
    + *         "z_noindex": 10
    + *       }
    + *     },
    + *     "h_noindex": "6",
    + *     "i_noindex": {
    + *       "j": 7,
    + *       "k": 8
    + *     }
    + *   },
    + *   "__mergedTextIndex": [
    + *     "1:a", "2:b", "3:c.d", "9:c.x.y"
    + *   ]
    + * }
    + * 
    + *

    + * The "__mergedTextIndex" could filter and manipulate the data based on the configuration in + * {@link SchemaConformingTransformerV2Config}. + */ +public class SchemaConformingTransformerV2 implements RecordTransformer { + private static final Logger _logger = LoggerFactory.getLogger(SchemaConformingTransformerV2.class); + private static final int MAXIMUM_LUCENE_DOCUMENT_SIZE = 32766; + private static final String MIN_DOCUMENT_LENGTH_DESCRIPTION = + "key length + `:` + shingle index overlap length + one non-overlap char"; + + private final boolean _continueOnError; + private final SchemaConformingTransformerV2Config _transformerConfig; + private final DataType _indexableExtrasFieldType; + private final DataType _unindexableExtrasFieldType; + private final DimensionFieldSpec _mergedTextIndexFieldSpec; + @Nullable + ServerMetrics _serverMetrics = null; + private SchemaTreeNode _schemaTree; + @Nullable + private PinotMeter _realtimeMergedTextIndexTruncatedDocumentSizeMeter = null; + private String _tableName; + private long _mergedTextIndexDocumentBytesCount = 0L; + private long _mergedTextIndexDocumentCount = 0L; + + public SchemaConformingTransformerV2(TableConfig tableConfig, Schema schema) { + if (null == tableConfig.getIngestionConfig() || null == tableConfig.getIngestionConfig() + .getSchemaConformingTransformerV2Config()) { + _continueOnError = false; + _transformerConfig = null; + _indexableExtrasFieldType = null; + _unindexableExtrasFieldType = null; + _mergedTextIndexFieldSpec = null; + return; + } + + _continueOnError = tableConfig.getIngestionConfig().isContinueOnError(); + _transformerConfig = tableConfig.getIngestionConfig().getSchemaConformingTransformerV2Config(); + String indexableExtrasFieldName = _transformerConfig.getIndexableExtrasField(); + _indexableExtrasFieldType = + indexableExtrasFieldName == null ? null : SchemaConformingTransformer.getAndValidateExtrasFieldType(schema, + indexableExtrasFieldName); + String unindexableExtrasFieldName = _transformerConfig.getUnindexableExtrasField(); + _unindexableExtrasFieldType = + unindexableExtrasFieldName == null ? null : SchemaConformingTransformer.getAndValidateExtrasFieldType(schema, + unindexableExtrasFieldName); + _mergedTextIndexFieldSpec = schema.getDimensionSpec(_transformerConfig.getMergedTextIndexField()); + _tableName = tableConfig.getTableName(); + _schemaTree = validateSchemaAndCreateTree(schema, _transformerConfig); + _serverMetrics = ServerMetrics.get(); + } + + /** + * Validates the schema against the given transformer's configuration. + */ + public static void validateSchema(@Nonnull Schema schema, + @Nonnull SchemaConformingTransformerV2Config transformerConfig) { + validateSchemaFieldNames(schema.getPhysicalColumnNames(), transformerConfig); + + String indexableExtrasFieldName = transformerConfig.getIndexableExtrasField(); + if (null != indexableExtrasFieldName) { + SchemaConformingTransformer.getAndValidateExtrasFieldType(schema, indexableExtrasFieldName); + } + String unindexableExtrasFieldName = transformerConfig.getUnindexableExtrasField(); + if (null != unindexableExtrasFieldName) { + SchemaConformingTransformer.getAndValidateExtrasFieldType(schema, indexableExtrasFieldName); + } + + validateSchemaAndCreateTree(schema, transformerConfig); + } + + /** + * Heuristic filter to detect whether a byte array is longer than a specified length and contains only base64 + * characters so that we treat it as encoded binary data. + * @param bytes array to check + * @param minLength byte array shorter than this length will not be treated as encoded binary data + * @return true if the input bytes is base64 encoded binary data by the heuristic above, false otherwise + */ + public static boolean base64ValueFilter(final byte[] bytes, int minLength) { + return bytes.length >= minLength && Base64Utils.isBase64IgnoreTrailingPeriods(bytes); + } + + /** + * Validates that none of the schema fields have names that conflict with the transformer's configuration. + */ + private static void validateSchemaFieldNames(Set schemaFields, + SchemaConformingTransformerV2Config transformerConfig) { + // Validate that none of the columns in the schema end with unindexableFieldSuffix + String unindexableFieldSuffix = transformerConfig.getUnindexableFieldSuffix(); + if (null != unindexableFieldSuffix) { + for (String field : schemaFields) { + Preconditions.checkState(!field.endsWith(unindexableFieldSuffix), "Field '%s' has no-index suffix '%s'", field, + unindexableFieldSuffix); + } + } + + // Validate that none of the columns in the schema end overlap with the fields in fieldPathsToDrop + Set fieldPathsToDrop = transformerConfig.getFieldPathsToDrop(); + if (null != fieldPathsToDrop) { + Set fieldIntersection = new HashSet<>(schemaFields); + fieldIntersection.retainAll(fieldPathsToDrop); + Preconditions.checkState(fieldIntersection.isEmpty(), "Fields in schema overlap with fieldPathsToDrop"); + } + } + + /** + * Validates the schema with a {@link SchemaConformingTransformerV2Config} instance and creates a tree representing + * the fields in the schema to be used when transforming input records. Refer to {@link SchemaTreeNode} for details. + * @throws IllegalArgumentException if schema validation fails in: + *

      + *
    • One of the fields in the schema has a name which when interpreted as a JSON path, corresponds to an object + * with an empty sub-key. E.g., the field name "a..b" corresponds to the JSON {"a": {"": {"b": ...}}}
    • + *
    + */ + private static SchemaTreeNode validateSchemaAndCreateTree(@Nonnull Schema schema, + @Nonnull SchemaConformingTransformerV2Config transformerConfig) + throws IllegalArgumentException { + Set schemaFields = schema.getPhysicalColumnNames(); + Map jsonKeyPathToColumnNameMap = new HashMap<>(); + for (Map.Entry entry : transformerConfig.getColumnNameToJsonKeyPathMap().entrySet()) { + String columnName = entry.getKey(); + String jsonKeyPath = entry.getValue(); + schemaFields.remove(columnName); + schemaFields.add(jsonKeyPath); + jsonKeyPathToColumnNameMap.put(jsonKeyPath, columnName); + } + + SchemaTreeNode rootNode = new SchemaTreeNode("", null, schema); + List subKeys = new ArrayList<>(); + for (String field : schemaFields) { + SchemaTreeNode currentNode = rootNode; + int keySeparatorIdx = field.indexOf(JsonUtils.KEY_SEPARATOR); + if (-1 == keySeparatorIdx) { + // Not a flattened key + currentNode = rootNode.getAndCreateChild(field, schema); + } else { + subKeys.clear(); + SchemaConformingTransformer.getAndValidateSubKeys(field, keySeparatorIdx, subKeys); + for (String subKey : subKeys) { + SchemaTreeNode childNode = currentNode.getAndCreateChild(subKey, schema); + currentNode = childNode; + } + } + currentNode.setColumn(jsonKeyPathToColumnNameMap.get(field)); + } + + return rootNode; + } + + @Override + public boolean isNoOp() { + return null == _transformerConfig; + } + + @Nullable + @Override + public GenericRow transform(GenericRow record) { + GenericRow outputRecord = new GenericRow(); + Map mergedTextIndexMap = new HashMap<>(); + + try { + Deque jsonPath = new ArrayDeque<>(); + ExtraFieldsContainer extraFieldsContainer = + new ExtraFieldsContainer(null != _transformerConfig.getUnindexableExtrasField()); + for (Map.Entry recordEntry : record.getFieldToValueMap().entrySet()) { + String recordKey = recordEntry.getKey(); + Object recordValue = recordEntry.getValue(); + jsonPath.addLast(recordKey); + ExtraFieldsContainer currentFieldsContainer = + processField(_schemaTree, jsonPath, recordValue, true, outputRecord, mergedTextIndexMap); + extraFieldsContainer.addChild(currentFieldsContainer); + jsonPath.removeLast(); + } + putExtrasField(_transformerConfig.getIndexableExtrasField(), _indexableExtrasFieldType, + extraFieldsContainer.getIndexableExtras(), outputRecord); + putExtrasField(_transformerConfig.getUnindexableExtrasField(), _unindexableExtrasFieldType, + extraFieldsContainer.getUnindexableExtras(), outputRecord); + + // Generate merged text index + if (null != _mergedTextIndexFieldSpec && !mergedTextIndexMap.isEmpty()) { + List luceneDocuments = getLuceneDocumentsFromMergedTextIndexMap(mergedTextIndexMap); + if (_mergedTextIndexFieldSpec.isSingleValueField()) { + outputRecord.putValue(_transformerConfig.getMergedTextIndexField(), String.join(" ", luceneDocuments)); + } else { + outputRecord.putValue(_transformerConfig.getMergedTextIndexField(), luceneDocuments); + } + } + } catch (Exception e) { + if (!_continueOnError) { + throw e; + } + _logger.error("Couldn't transform record: {}", record.toString(), e); + outputRecord.putValue(GenericRow.INCOMPLETE_RECORD_KEY, true); + } + + return outputRecord; + } + + /** + * The method traverses the record and schema tree at the same time. It would check the specs of record key/value + * pairs with the corresponding schema tree node and {#link SchemaConformingTransformerV2Config}. Finally drop or put + * them into the output record with the following logics: + * Taking example: + * { + * "a": 1, + * "b": { + * "c": 2, + * "d": 3, + * "d_noIdx": 4 + * } + * "b_noIdx": { + * "c": 5, + * "d": 6, + * } + * } + * with column "a", "b", "b.c" in schema + * There are two types of output: + * - flattened keys with values, e.g., + * - keyPath as column and value as leaf node, e.g., "a": 1, "b.c": 2. However, "b" is not a leaf node, so it would + * be skipped + * - __mergedTestIdx storing ["1:a", "2:b.c", "3:b.d"] as a string array + * - structured Json format, e.g., + * - indexableFields/json_data: {"a": 1, "b": {"c": 2, "d": 3}} + * - unindexableFields/json_data_noIdx: {"b": {"d_noIdx": 4} ,"b_noIdx": {"c": 5, "d": 6}} + * Expected behavior: + * - If the current key is special, it would be added to the outputRecord and skip subtree + * - If the keyJsonPath is in fieldPathsToDrop, it and its subtree would be skipped + * - At leaf node (base case in recursion): + * - Parse keyPath and value and add as flattened result to outputRecord + * - Return structured fields as ExtraFieldsContainer + * (leaf node is defined as node not as "Map" type. Leaf node is possible to be collection of or array of "Map". But + * for simplicity, we still treat it as leaf node and do not traverse its children) + * - For non-leaf node + * - Construct ExtraFieldsContainer based on children's result and return + * + * @param parentNode The parent node in the schema tree which might or might not has a child with the given key. If + * parentNode is null, it means the current key is out of the schema tree. + * @param jsonPath The key json path split by "." + * @param value The value of the current field + * @param isIndexable Whether the current field is indexable + * @param outputRecord The output record updated during traverse + * @param mergedTextIndexMap The merged text index map updated during traverse + * @return ExtraFieldsContainer carries the indexable and unindexable fields of the current node as well as its + * subtree + */ + private ExtraFieldsContainer processField(SchemaTreeNode parentNode, Deque jsonPath, Object value, + boolean isIndexable, GenericRow outputRecord, Map mergedTextIndexMap) { + // Common variables + boolean storeIndexableExtras = _transformerConfig.getIndexableExtrasField() != null; + boolean storeUnindexableExtras = _transformerConfig.getUnindexableExtrasField() != null; + String key = jsonPath.peekLast(); + ExtraFieldsContainer extraFieldsContainer = new ExtraFieldsContainer(storeUnindexableExtras); + + // Base case + if (StreamDataDecoderImpl.isSpecialKeyType(key) || GenericRow.isSpecialKeyType(key)) { + outputRecord.putValue(key, value); + return extraFieldsContainer; + } + + String keyJsonPath = String.join(".", jsonPath); + if (_transformerConfig.getFieldPathsToPreserveInput().contains(keyJsonPath)) { + outputRecord.putValue(keyJsonPath, value); + return extraFieldsContainer; + } + + Set fieldPathsToDrop = _transformerConfig.getFieldPathsToDrop(); + if (null != fieldPathsToDrop && fieldPathsToDrop.contains(keyJsonPath)) { + return extraFieldsContainer; + } + + SchemaTreeNode currentNode = parentNode == null ? null : parentNode.getChild(key); + String unindexableFieldSuffix = _transformerConfig.getUnindexableFieldSuffix(); + isIndexable = isIndexable && (null == unindexableFieldSuffix || !key.endsWith(unindexableFieldSuffix)); + if (!(value instanceof Map)) { + // leaf node + if (!isIndexable) { + extraFieldsContainer.addUnindexableEntry(key, value); + } else { + if (null != currentNode && currentNode.isColumn()) { + // In schema + outputRecord.putValue(currentNode.getColumnName(), currentNode.getValue(value)); + if (_transformerConfig.getFieldsToDoubleIngest().contains(keyJsonPath)) { + extraFieldsContainer.addIndexableEntry(key, value); + } + mergedTextIndexMap.put(keyJsonPath, value); + } else { + // Out of schema + if (storeIndexableExtras) { + extraFieldsContainer.addIndexableEntry(key, value); + mergedTextIndexMap.put(keyJsonPath, value); + } + } + } + return extraFieldsContainer; + } + // Traverse the subtree + Map valueAsMap = (Map) value; + for (Map.Entry entry : valueAsMap.entrySet()) { + jsonPath.addLast(entry.getKey()); + ExtraFieldsContainer childContainer = + processField(currentNode, jsonPath, entry.getValue(), isIndexable, outputRecord, mergedTextIndexMap); + extraFieldsContainer.addChild(key, childContainer); + jsonPath.removeLast(); + } + return extraFieldsContainer; + } + + /** + * Generate an Lucene document based on the provided key-value pair. + * The index document follows this format: "val:key". + * @param kv used to generate text index documents + * @param indexDocuments a list to store the generated index documents + * @param mergedTextIndexDocumentMaxLength which we enforce via truncation during document generation + */ + public void generateTextIndexLuceneDocument(Map.Entry kv, List indexDocuments, + Integer mergedTextIndexDocumentMaxLength) { + String key = kv.getKey(); + String val; + // To avoid redundant leading and tailing '"', only convert to JSON string if the value is a list or an array + if (kv.getValue() instanceof Collection || kv.getValue() instanceof Object[]) { + try { + val = JsonUtils.objectToString(kv.getValue()); + } catch (JsonProcessingException e) { + val = kv.getValue().toString(); + } + } else { + val = kv.getValue().toString(); + } + + // TODO: theoretically, the key length + 1 could cause integer overflow. But in reality, upstream message size + // limit usually could not reach that high. We should revisit this if we see any issue. + if (key.length() + 1 > MAXIMUM_LUCENE_DOCUMENT_SIZE) { + _logger.error("The provided key's length is too long, text index document cannot be truncated"); + return; + } + + // Truncate the value to ensure the generated index document is less or equal to mergedTextIndexDocumentMaxLength + // The value length should be the mergedTextIndexDocumentMaxLength minus ":" character (length 1) minus key length + int valueTruncationLength = mergedTextIndexDocumentMaxLength - 1 - key.length(); + if (val.length() > valueTruncationLength) { + _realtimeMergedTextIndexTruncatedDocumentSizeMeter = _serverMetrics + .addMeteredTableValue(_tableName, ServerMeter.REALTIME_MERGED_TEXT_IDX_TRUNCATED_DOCUMENT_SIZE, + key.length() + 1 + val.length(), _realtimeMergedTextIndexTruncatedDocumentSizeMeter); + val = val.substring(0, valueTruncationLength); + } + + _mergedTextIndexDocumentBytesCount += key.length() + 1 + val.length(); + _mergedTextIndexDocumentCount += 1; + _serverMetrics.setValueOfTableGauge(_tableName, ServerGauge.REALTIME_MERGED_TEXT_IDX_DOCUMENT_AVG_LEN, + _mergedTextIndexDocumentBytesCount / _mergedTextIndexDocumentCount); + + indexDocuments.add(val + ":" + key); + } + + /** + * Implement shingling for the merged text index based on the provided key-value pair. + * Each shingled index document retains the format of a standard index document: "val:key". However, "val" now + * denotes a sliding window of characters on the value. The total length of each shingled index document + * (key length + shingled value length + 1)must be less than or equal to shingleIndexMaxLength. The starting index + * of the sliding window for the value is increased by shinglingOverlapLength for every new shingled document. + * All shingle index documents, except for the last one, should have the maximum possible length. If the minimum + * document length (shingling overlap length + key length + 1) exceeds the maximum Lucene document size + * (MAXIMUM_LUCENE_DOCUMENT_SIZE), shingling is disabled, and the value is truncated to match the maximum Lucene + * document size. If shingleIndexMaxLength is lower than the required minimum document length and also lower than + * the maximum + * Lucene document size, shingleIndexMaxLength is adjusted to match the maximum Lucene document size. + * + * Note that the most important parameter, the shingleIndexOverlapLength, is the maximum search length that will yield + * results with 100% accuracy. + * + * Example: key-> "key", value-> "0123456789ABCDEF", max length: 10, shingling overlap length: 3 + * Generated documents: + * 012345:key + * 345678:key + * 6789AB:key + * 9ABCDE:key + * CDEF:key + * Any query with a length of 7 will yield no results, such as "0123456" or "6789ABC". + * Any query with a length of 3 will yield results with 100% accuracy (i.e. is always guaranteed to be searchable). + * Any query with a length between 4 and 6 (inclusive) has indeterminate accuracy. + * E.g. for queries with length 5, "12345", "789AB" will hit, while "23456" will miss. + * + * @param kv used to generate shingle text index documents + * @param shingleIndexDocuments a list to store the generated shingle index documents + * @param shingleIndexMaxLength the maximum length of each shingle index document. Needs to be greater than the + * length of the key and shingleIndexOverlapLength + 1, and must be lower or equal + * to MAXIMUM_LUCENE_DOCUMENT_SIZE. + * @param shingleIndexOverlapLength the number of characters in the kv-pair's value shared by two adjacent shingle + * index documents. If null, the overlap length will be defaulted to half of the max + * document length. + */ + public void generateShingleTextIndexDocument(Map.Entry kv, List shingleIndexDocuments, + int shingleIndexMaxLength, int shingleIndexOverlapLength) { + String key = kv.getKey(); + String val; + // To avoid redundant leading and tailing '"', only convert to JSON string if the value is a list or an array + if (kv.getValue() instanceof Collection || kv.getValue() instanceof Object[]) { + try { + val = JsonUtils.objectToString(kv.getValue()); + } catch (JsonProcessingException e) { + val = kv.getValue().toString(); + } + } else { + val = kv.getValue().toString(); + } + final int valLength = val.length(); + final int documentSuffixLength = key.length() + 1; + final int minDocumentLength = documentSuffixLength + shingleIndexOverlapLength + 1; + + if (shingleIndexOverlapLength >= valLength) { + if (_logger.isDebugEnabled()) { + _logger.warn("The shingleIndexOverlapLength " + shingleIndexOverlapLength + " is longer than the value length " + + valLength + ". Shingling will not be applied since only one document will be generated."); + } + generateTextIndexLuceneDocument(kv, shingleIndexDocuments, shingleIndexMaxLength); + return; + } + + if (minDocumentLength > MAXIMUM_LUCENE_DOCUMENT_SIZE) { + _logger.debug("The minimum document length " + minDocumentLength + " (" + MIN_DOCUMENT_LENGTH_DESCRIPTION + ") " + + " exceeds the limit of maximum Lucene document size " + MAXIMUM_LUCENE_DOCUMENT_SIZE + ". Value will be " + + "truncated and shingling will not be applied."); + generateTextIndexLuceneDocument(kv, shingleIndexDocuments, shingleIndexMaxLength); + return; + } + + // This logging becomes expensive if user accidentally sets a very low shingleIndexMaxLength + if (shingleIndexMaxLength < minDocumentLength) { + _logger.debug("The shingleIndexMaxLength " + shingleIndexMaxLength + " is smaller than the minimum document " + + "length " + minDocumentLength + " (" + MIN_DOCUMENT_LENGTH_DESCRIPTION + "). Increasing the " + + "shingleIndexMaxLength to maximum Lucene document size " + MAXIMUM_LUCENE_DOCUMENT_SIZE + "."); + shingleIndexMaxLength = MAXIMUM_LUCENE_DOCUMENT_SIZE; + } + + // Shingle window slide length is the index position on the value which we shall advance on every iteration. + // We ensure shingleIndexMaxLength >= minDocumentLength so that shingleWindowSlideLength >= 1. + int shingleWindowSlideLength = shingleIndexMaxLength - shingleIndexOverlapLength - documentSuffixLength; + + // Generate shingle index documents + // When starting_idx + shingleIndexOverlapLength >= valLength, there are no new characters to capture, then we stop + // the shingle document generation loop. + // We ensure that shingleIndexOverlapLength < valLength so that this loop will be entered at lease once. + for (int i = 0; i + shingleIndexOverlapLength < valLength; i += shingleWindowSlideLength) { + String documentValStr = val.substring(i, Math.min(i + shingleIndexMaxLength - documentSuffixLength, valLength)); + String shingleIndexDocument = documentValStr + ":" + key; + shingleIndexDocuments.add(shingleIndexDocument); + _mergedTextIndexDocumentBytesCount += shingleIndexDocument.length(); + ++_mergedTextIndexDocumentCount; + } + _serverMetrics.setValueOfTableGauge(_tableName, ServerGauge.REALTIME_MERGED_TEXT_IDX_DOCUMENT_AVG_LEN, + _mergedTextIndexDocumentBytesCount / _mergedTextIndexDocumentCount); + } + + /** + * Converts (if necessary) and adds the given extras field to the output record + */ + private void putExtrasField(String fieldName, DataType fieldType, Map field, + GenericRow outputRecord) { + if (null == field) { + return; + } + + switch (fieldType) { + case JSON: + outputRecord.putValue(fieldName, field); + break; + case STRING: + try { + outputRecord.putValue(fieldName, JsonUtils.objectToString(field)); + } catch (JsonProcessingException e) { + throw new RuntimeException("Failed to convert '" + fieldName + "' to string", e); + } + break; + default: + throw new UnsupportedOperationException("Cannot convert '" + fieldName + "' to " + fieldType.name()); + } + } + + private List getLuceneDocumentsFromMergedTextIndexMap(Map mergedTextIndexMap) { + final Integer mergedTextIndexDocumentMaxLength = _transformerConfig.getMergedTextIndexDocumentMaxLength(); + final @Nullable + Integer mergedTextIndexShinglingOverlapLength = _transformerConfig.getMergedTextIndexShinglingOverlapLength(); + List luceneDocuments = new ArrayList<>(); + mergedTextIndexMap.entrySet().stream().filter(kv -> null != kv.getKey() && null != kv.getValue()) + .filter(kv -> !_transformerConfig.getMergedTextIndexPathToExclude().contains(kv.getKey())).filter( + kv -> !base64ValueFilter(kv.getValue().toString().getBytes(), + _transformerConfig.getMergedTextIndexBinaryDocumentDetectionMinLength())).filter( + kv -> _transformerConfig.getMergedTextIndexSuffixToExclude().stream() + .anyMatch(suffix -> !kv.getKey().endsWith(suffix))).forEach(kv -> { + if (null == mergedTextIndexShinglingOverlapLength) { + generateTextIndexLuceneDocument(kv, luceneDocuments, mergedTextIndexDocumentMaxLength); + } else { + generateShingleTextIndexDocument(kv, luceneDocuments, mergedTextIndexDocumentMaxLength, + mergedTextIndexShinglingOverlapLength); + } + }); + return luceneDocuments; + } +} + +/** + * SchemaTreeNode represents the tree node when we construct the schema tree. The node could be either leaf node or + * non-leaf node. Both types of node could hold the volumn as a column in the schema. + * For example, the schema with columns a, b, c, d.e, d.f, x.y, x.y.z, x.y.w will have the following tree structure: + * root -- a* + * -- b* + * -- c* + * -- d -- e* + * -- f* + * -- x* -- y* -- z* + * -- w* + * where node with "*" could represent a valid column in the schema. + */ +class SchemaTreeNode { + private boolean _isColumn; + private Map _children; + // Taking the example of key "x.y.z", the keyName will be "z" and the parentPath will be "x.y" + // Root node would have keyName as "" and parentPath as null + // Root node's children will have keyName as the first level key and parentPath as "" + @Nonnull + private String _keyName; + @Nullable + private String _columnName; + @Nullable + private String _parentPath; + private FieldSpec _fieldSpec; + + public SchemaTreeNode(String keyName, String parentPath, Schema schema) { + _keyName = keyName; + _parentPath = parentPath; + _fieldSpec = schema.getFieldSpecFor(getJsonKeyPath()); + _children = new HashMap<>(); + } + + public boolean isColumn() { + return _isColumn; + } + + public void setColumn(String columnName) { + if (columnName == null) { + _columnName = getJsonKeyPath(); + } else { + _columnName = columnName; + } + _isColumn = true; + } + + public boolean hasChild(String key) { + return _children.containsKey(key); + } + + /** + * If does not have the child node, add a child node to the current node and return the child node. + * If the child node already exists, return the existing child node. + * @param key + * @return + */ + public SchemaTreeNode getAndCreateChild(String key, Schema schema) { + SchemaTreeNode child = _children.get(key); + if (child == null) { + child = new SchemaTreeNode(key, getJsonKeyPath(), schema); + _children.put(key, child); + } + return child; + } + + public SchemaTreeNode getChild(String key) { + return _children.get(key); + } + + public String getKeyName() { + return _keyName; + } + + public String getColumnName() { + return _columnName; + } + + public Object getValue(Object value) { + // In {#link DataTypeTransformer}, for a field type as SingleValueField, it does not allow the input value as a + // collection or array. To prevent the error, we serialize the value to a string if the field is a string type. + if (_fieldSpec != null && _fieldSpec.getDataType() == DataType.STRING && _fieldSpec.isSingleValueField()) { + try { + if (value instanceof Collection) { + return JsonUtils.objectToString(value); + } + if (value instanceof Object[]) { + return JsonUtils.objectToString(Arrays.asList((Object[]) value)); + } + } catch (JsonProcessingException e) { + return value.toString(); + } + } + return value; + } + + public String getJsonKeyPath() { + if (_parentPath == null || _parentPath.isEmpty()) { + return _keyName; + } + return _parentPath + JsonUtils.KEY_SEPARATOR + _keyName; + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/Base64Utils.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/Base64Utils.java new file mode 100644 index 000000000000..12a1652a242b --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/Base64Utils.java @@ -0,0 +1,44 @@ +/** + * 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.segment.local.utils; + +import org.apache.commons.codec.binary.Base64; + +/** + * Simple wrapper class over codec's Base64 implementation to handle Pinot-specific Base64 encoded binary data. + */ +public class Base64Utils extends Base64 { + public static boolean isBase64IgnoreWhiteSpace(byte[] arrayOctet) { + return isBase64(arrayOctet); + } + + public static boolean isBase64IgnoreTrailingPeriods(byte[] arrayOctet) { + int i = arrayOctet.length - 1; + while (i >= 0 && '.' == arrayOctet[i]) { + --i; + } + while (i >= 0) { + if (!isBase64(arrayOctet[i])) { + return false; + } + --i; + } + return true; + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java index 203d7d930a45..556b0258558f 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/IngestionUtils.java @@ -310,7 +310,8 @@ private static void registerPinotFS(String fileURIScheme, String fsClass, PinotC public static Set getFieldsForRecordExtractor(@Nullable IngestionConfig ingestionConfig, Schema schema) { Set fieldsForRecordExtractor = new HashSet<>(); - if (null != ingestionConfig && null != ingestionConfig.getSchemaConformingTransformerConfig()) { + if (null != ingestionConfig && (null != ingestionConfig.getSchemaConformingTransformerConfig() + || null != ingestionConfig.getSchemaConformingTransformerV2Config())) { // The SchemaConformingTransformer requires that all fields are extracted, indicated by returning an empty set // here. Compared to extracting the fields specified below, extracting all fields should be a superset. return fieldsForRecordExtractor; diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java index 8d31f5d39941..14c4040a600e 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java @@ -46,6 +46,7 @@ import org.apache.pinot.segment.local.function.FunctionEvaluator; import org.apache.pinot.segment.local.function.FunctionEvaluatorFactory; import org.apache.pinot.segment.local.recordtransformer.SchemaConformingTransformer; +import org.apache.pinot.segment.local.recordtransformer.SchemaConformingTransformerV2; import org.apache.pinot.segment.local.segment.creator.impl.inv.BitSlicedRangeIndexCreator; import org.apache.pinot.segment.spi.AggregationFunctionType; import org.apache.pinot.segment.spi.index.DictionaryIndexConfig; @@ -77,6 +78,7 @@ import org.apache.pinot.spi.config.table.ingestion.FilterConfig; import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; import org.apache.pinot.spi.config.table.ingestion.SchemaConformingTransformerConfig; +import org.apache.pinot.spi.config.table.ingestion.SchemaConformingTransformerV2Config; import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig; import org.apache.pinot.spi.config.table.ingestion.TransformConfig; import org.apache.pinot.spi.data.FieldSpec; @@ -574,6 +576,12 @@ public static void validateIngestionConfig(TableConfig tableConfig, @Nullable Sc if (null != schemaConformingTransformerConfig && null != schema) { SchemaConformingTransformer.validateSchema(schema, schemaConformingTransformerConfig); } + + SchemaConformingTransformerV2Config schemaConformingTransformerV2Config = + ingestionConfig.getSchemaConformingTransformerV2Config(); + if (null != schemaConformingTransformerV2Config && null != schema) { + SchemaConformingTransformerV2.validateSchema(schema, schemaConformingTransformerV2Config); + } } } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/recordtransformer/SchemaConformingTransformerV2Test.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/recordtransformer/SchemaConformingTransformerV2Test.java new file mode 100644 index 000000000000..6189f14d42f9 --- /dev/null +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/recordtransformer/SchemaConformingTransformerV2Test.java @@ -0,0 +1,934 @@ +/** + * 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.segment.local.recordtransformer; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.NullNode; +import com.fasterxml.jackson.databind.node.NumericNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.databind.node.TextNode; +import java.io.IOException; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import javax.annotation.Nonnull; +import org.apache.pinot.common.metrics.ServerMetrics; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; +import org.apache.pinot.spi.config.table.ingestion.SchemaConformingTransformerV2Config; +import org.apache.pinot.spi.data.FieldSpec.DataType; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.utils.builder.TableConfigBuilder; +import org.testng.Assert; +import org.testng.annotations.Test; + +import static org.mockito.Mockito.mock; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import static org.testng.AssertJUnit.fail; + + +public class SchemaConformingTransformerV2Test { + private static final String INDEXABLE_EXTRAS_FIELD_NAME = "json_data"; + private static final String UNINDEXABLE_EXTRAS_FIELD_NAME = "json_data_no_idx"; + private static final String UNINDEXABLE_FIELD_SUFFIX = "_noIndex"; + private static final String MERGED_TEXT_INDEX_FIELD_NAME = "__mergedTextIndex"; + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final JsonNodeFactory N = OBJECT_MAPPER.getNodeFactory(); + private static final String TEST_JSON_ARRAY_FIELD_NAME = "arrayField"; + private static final String TEST_JSON_NULL_FIELD_NAME = "nullField"; + private static final String TEST_JSON_STRING_FIELD_NAME = "stringField"; + private static final String TEST_JSON_MAP_FIELD_NAME = "mapField"; + private static final String TEST_JSON_MAP_NO_IDX_FIELD_NAME = "mapField_noIndex"; + private static final String TEST_JSON_NESTED_MAP_FIELD_NAME = "nestedFields"; + private static final String TEST_JSON_INT_NO_IDX_FIELD_NAME = "intField_noIndex"; + private static final String TEST_JSON_STRING_NO_IDX_FIELD_NAME = "stringField_noIndex"; + private static final ArrayNode TEST_JSON_ARRAY_NODE = N.arrayNode().add(0).add(1).add(2).add(3); + private static final NullNode TEST_JSON_NULL_NODE = N.nullNode(); + private static final TextNode TEST_JSON_STRING_NODE = N.textNode("a"); + private static final NumericNode TEST_INT_NODE = N.numberNode(9); + private static final TextNode TEST_JSON_STRING_NO_IDX_NODE = N.textNode("z"); + private static final CustomObjectNode TEST_JSON_MAP_NODE = + CustomObjectNode.create().set(TEST_JSON_ARRAY_FIELD_NAME, TEST_JSON_ARRAY_NODE) + .set(TEST_JSON_NULL_FIELD_NAME, TEST_JSON_NULL_NODE).set(TEST_JSON_STRING_FIELD_NAME, TEST_JSON_STRING_NODE); + private static final CustomObjectNode TEST_JSON_MAP_NO_IDX_NODE = + CustomObjectNode.create().set(TEST_JSON_INT_NO_IDX_FIELD_NAME, TEST_INT_NODE) + .set(TEST_JSON_STRING_NO_IDX_FIELD_NAME, TEST_JSON_STRING_NO_IDX_NODE); + private static final CustomObjectNode TEST_JSON_MAP_NODE_WITH_NO_IDX = + CustomObjectNode.create().set(TEST_JSON_ARRAY_FIELD_NAME, TEST_JSON_ARRAY_NODE) + .set(TEST_JSON_NULL_FIELD_NAME, TEST_JSON_NULL_NODE).set(TEST_JSON_STRING_FIELD_NAME, TEST_JSON_STRING_NODE) + .set(TEST_JSON_INT_NO_IDX_FIELD_NAME, TEST_INT_NODE) + .set(TEST_JSON_STRING_NO_IDX_FIELD_NAME, TEST_JSON_STRING_NO_IDX_NODE); + static { + ServerMetrics.register(mock(ServerMetrics.class)); + } + private static final SchemaConformingTransformerV2 _RECORD_TRANSFORMER = + new SchemaConformingTransformerV2(createDefaultBasicTableConfig(), createDefaultSchema()); + + private static TableConfig createDefaultBasicTableConfig() { + IngestionConfig ingestionConfig = new IngestionConfig(); + SchemaConformingTransformerV2Config schemaConformingTransformerV2Config = + new SchemaConformingTransformerV2Config(true, INDEXABLE_EXTRAS_FIELD_NAME, true, UNINDEXABLE_EXTRAS_FIELD_NAME, + UNINDEXABLE_FIELD_SUFFIX, null, null, null, null, null, null, null, null); + ingestionConfig.setSchemaConformingTransformerV2Config(schemaConformingTransformerV2Config); + return new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setIngestionConfig(ingestionConfig) + .build(); + } + + private static TableConfig createDefaultTableConfig(String indexableExtrasField, String unindexableExtrasField, + String unindexableFieldSuffix, Set fieldPathsToDrop, Set fieldPathsToPreserve, + String mergedTextIndexField) { + IngestionConfig ingestionConfig = new IngestionConfig(); + SchemaConformingTransformerV2Config schemaConformingTransformerV2Config = + new SchemaConformingTransformerV2Config(indexableExtrasField != null, indexableExtrasField, + unindexableExtrasField != null, unindexableExtrasField, unindexableFieldSuffix, fieldPathsToDrop, + fieldPathsToPreserve, mergedTextIndexField, null, null, null, null, null); + ingestionConfig.setSchemaConformingTransformerV2Config(schemaConformingTransformerV2Config); + return new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setIngestionConfig(ingestionConfig) + .build(); + } + + private static Schema createDefaultSchema() { + return createDefaultSchemaBuilder().addSingleValueDimension("intField", DataType.INT).build(); + } + + private static Schema.SchemaBuilder createDefaultSchemaBuilder() { + return new Schema.SchemaBuilder().addSingleValueDimension(INDEXABLE_EXTRAS_FIELD_NAME, DataType.JSON) + .addSingleValueDimension(UNINDEXABLE_EXTRAS_FIELD_NAME, DataType.JSON); + } + + @Test + public void testWithNoUnindexableFields() { + /* + { + "arrayField" : [ 0, 1, 2, 3 ], + "nullField" : null, + "stringField" : "a", + "mapField" : { + "arrayField" : [ 0, 1, 2, 3 ], + "nullField" : null, + "stringField" : "a" + }, + "nestedField" : { + "arrayField" : [ 0, 1, 2, 3 ], + "nullField" : null, + "stringField" : "a", + "mapField" : { + "arrayField" : [ 0, 1, 2, 3 ], + "nullField" : null, + "stringField" : "a" + } + } + } + */ + final CustomObjectNode inputJsonNode = + CustomObjectNode.create().setAll(TEST_JSON_MAP_NODE).set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NODE) + .set(TEST_JSON_NESTED_MAP_FIELD_NAME, + CustomObjectNode.create().setAll(TEST_JSON_MAP_NODE).set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NODE)); + + CustomObjectNode expectedJsonNode; + Schema schema; + + // No dedicated columns, everything moved under INDEXABLE_EXTRAS_FIELD_NAME + /* + { + "json_data" : { + "arrayField" : [ 0, 1, 2, 3 ], + "nullField" : null, + "stringField" : "a", + "mapField" : { + "arrayField" : [ 0, 1, 2, 3 ], + "nullField" : null, + "stringField" : "a" + }, + "nestedField" : { + "arrayField" : [ 0, 1, 2, 3 ], + "nullField" : null, + "stringField" : "a", + "mapField" : { + "arrayField" : [ 0, 1, 2, 3 ], + "nullField" : null, + "stringField" : "a" + } + } + } + } + */ + schema = createDefaultSchemaBuilder().build(); + expectedJsonNode = CustomObjectNode.create().set(INDEXABLE_EXTRAS_FIELD_NAME, inputJsonNode); + transformWithIndexableFields(schema, inputJsonNode, expectedJsonNode); + + // Three dedicated columns in schema, only two are populated, one ignored + /* + { + "arrayField":[0, 1, 2, 3], + "nestedFields.stringField":"a", + "":{ + "mapField": { + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a" + }, + "nullField":null, + "stringField":"a", + "nestedFields":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "mapField":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a" + } + } + } + } + */ + schema = createDefaultSchemaBuilder().addMultiValueDimension(TEST_JSON_ARRAY_FIELD_NAME, DataType.INT) + .addSingleValueDimension(TEST_JSON_MAP_FIELD_NAME, DataType.STRING) + .addSingleValueDimension(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_STRING_FIELD_NAME, DataType.STRING) + .build(); + expectedJsonNode = CustomObjectNode.create().set(TEST_JSON_ARRAY_FIELD_NAME, TEST_JSON_ARRAY_NODE) + .set(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_STRING_FIELD_NAME, TEST_JSON_STRING_NODE) + + .set(INDEXABLE_EXTRAS_FIELD_NAME, CustomObjectNode.create().set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NODE) + .setAll(TEST_JSON_MAP_NODE.deepCopy().removeAndReturn(TEST_JSON_ARRAY_FIELD_NAME)) + .set(TEST_JSON_NESTED_MAP_FIELD_NAME, CustomObjectNode.create() + .setAll(TEST_JSON_MAP_NODE.deepCopy().removeAndReturn(TEST_JSON_STRING_FIELD_NAME)) + .set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NODE))); + transformWithIndexableFields(schema, inputJsonNode, expectedJsonNode); + + // 8 dedicated columns, only 6 are populated + /* + { + "arrayField" : [ 0, 1, 2, 3 ], + "nullField" : null, + "stringField" : "a", + "nestedField.arrayField" : [ 0, 1, 2, 3 ], + "nestedField.nullField" : null, + "nestedField.stringField" : "a", + "json_data" : { + "mapField" : { + "arrayField" : [ 0, 1, 2, 3 ], + "nullField" : null, + "stringField" : "a" + }, + "nestedField" : { + "mapField" : { + "arrayField" : [ 0, 1, 2, 3 ], + "nullField" : null, + "stringField" : "a" + } + } + } + } + */ + schema = createDefaultSchemaBuilder().addMultiValueDimension(TEST_JSON_ARRAY_FIELD_NAME, DataType.INT) + .addSingleValueDimension(TEST_JSON_NULL_FIELD_NAME, DataType.STRING) + .addSingleValueDimension(TEST_JSON_STRING_FIELD_NAME, DataType.STRING) + .addSingleValueDimension(TEST_JSON_MAP_FIELD_NAME, DataType.JSON) + .addMultiValueDimension(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_ARRAY_FIELD_NAME, DataType.INT) + .addSingleValueDimension(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_NULL_FIELD_NAME, DataType.STRING) + .addSingleValueDimension(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_STRING_FIELD_NAME, DataType.STRING) + .addSingleValueDimension(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_MAP_FIELD_NAME, DataType.JSON) + .build(); + expectedJsonNode = CustomObjectNode.create().setAll(TEST_JSON_MAP_NODE) + .set(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_ARRAY_FIELD_NAME, TEST_JSON_ARRAY_NODE) + .set(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_NULL_FIELD_NAME, TEST_JSON_NULL_NODE) + .set(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_STRING_FIELD_NAME, TEST_JSON_STRING_NODE) + .set(INDEXABLE_EXTRAS_FIELD_NAME, CustomObjectNode.create().set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NODE) + .set(TEST_JSON_NESTED_MAP_FIELD_NAME, + CustomObjectNode.create().set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NODE))); + transformWithIndexableFields(schema, inputJsonNode, expectedJsonNode); + } + + @Test + public void testWithUnindexableFieldsAndMergedTextIndex() { + /* + { + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a", + "intField_noIndex":9, + "string_noIndex":"z", + "mapField":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a", + "intField_noIndex":9, + "string_noIndex":"z" + }, + "mapField_noIndex":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a", + }, + "nestedFields":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a", + "intField_noIndex":9, + "string_noIndex":"z", + "mapField":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a", + "intField_noIndex":9, + "string_noIndex":"z" + } + } + } + */ + final CustomObjectNode inputJsonNode = + CustomObjectNode.create().setAll(TEST_JSON_MAP_NODE).set(TEST_JSON_ARRAY_FIELD_NAME, TEST_JSON_ARRAY_NODE) + .set(TEST_JSON_NULL_FIELD_NAME, TEST_JSON_NULL_NODE).set(TEST_JSON_STRING_FIELD_NAME, TEST_JSON_STRING_NODE) + .set(TEST_JSON_INT_NO_IDX_FIELD_NAME, TEST_INT_NODE) + .set(TEST_JSON_STRING_NO_IDX_FIELD_NAME, TEST_JSON_STRING_NO_IDX_NODE) + .set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NODE_WITH_NO_IDX) + .set(TEST_JSON_MAP_NO_IDX_FIELD_NAME, TEST_JSON_MAP_NODE).set(TEST_JSON_NESTED_MAP_FIELD_NAME, + CustomObjectNode.create().setAll(TEST_JSON_MAP_NODE).set(TEST_JSON_ARRAY_FIELD_NAME, TEST_JSON_ARRAY_NODE) + .set(TEST_JSON_NULL_FIELD_NAME, TEST_JSON_NULL_NODE) + .set(TEST_JSON_STRING_FIELD_NAME, TEST_JSON_STRING_NODE) + .set(TEST_JSON_INT_NO_IDX_FIELD_NAME, TEST_INT_NODE) + .set(TEST_JSON_STRING_NO_IDX_FIELD_NAME, TEST_JSON_STRING_NO_IDX_NODE) + .set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NODE_WITH_NO_IDX)); + + CustomObjectNode expectedJsonNode; + CustomObjectNode expectedJsonNodeWithMergedTextIndex; + Schema.SchemaBuilder schemaBuilder; + + // No schema + schemaBuilder = createDefaultSchemaBuilder(); + /* + { + "indexableExtras":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a", + "mapField":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a" + }, + "nestedFields":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a", + "mapField":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a" + } + } + }, + "unindexableExtras":{ + "intField_noIndex":9, + "string_noIndex":"z", + "mapField":{ + "intField_noIndex":9, + "string_noIndex":"z" + }, + "mapField_noIndex":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a", + }, + "nestedFields":{ + "intField_noIndex":9, + "string_noIndex":"z", + "mapField":{ + "intField_noIndex":9, + "string_noIndex":"z" + } + } + }, + __mergedTextIndex: [ + "[0, 1, 2, 3]:arrayField", "a:stringField", + "[0, 1, 2, 3]:mapField.arrayField", "a:mapField.stringField", + "[0, 1, 2, 3]:nestedFields.arrayField", "a:nestedFields.stringField", + "[0, 1, 2, 3]:nestedFields.mapField.arrayField", "a:nestedFields.mapField.stringField", + ] + } + */ + expectedJsonNode = CustomObjectNode.create().set(INDEXABLE_EXTRAS_FIELD_NAME, + CustomObjectNode.create().set(TEST_JSON_ARRAY_FIELD_NAME, TEST_JSON_ARRAY_NODE) + .set(TEST_JSON_NULL_FIELD_NAME, TEST_JSON_NULL_NODE).set(TEST_JSON_STRING_FIELD_NAME, TEST_JSON_STRING_NODE) + .set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NODE).set(TEST_JSON_NESTED_MAP_FIELD_NAME, + CustomObjectNode.create().set(TEST_JSON_ARRAY_FIELD_NAME, TEST_JSON_ARRAY_NODE) + .set(TEST_JSON_NULL_FIELD_NAME, TEST_JSON_NULL_NODE) + .set(TEST_JSON_STRING_FIELD_NAME, TEST_JSON_STRING_NODE) + .set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NODE))) + + .set(UNINDEXABLE_EXTRAS_FIELD_NAME, + CustomObjectNode.create().set(TEST_JSON_INT_NO_IDX_FIELD_NAME, TEST_INT_NODE) + .set(TEST_JSON_STRING_NO_IDX_FIELD_NAME, TEST_JSON_STRING_NO_IDX_NODE) + .set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NO_IDX_NODE) + .set(TEST_JSON_MAP_NO_IDX_FIELD_NAME, TEST_JSON_MAP_NODE).set(TEST_JSON_NESTED_MAP_FIELD_NAME, + CustomObjectNode.create().set(TEST_JSON_INT_NO_IDX_FIELD_NAME, TEST_INT_NODE) + .set(TEST_JSON_STRING_NO_IDX_FIELD_NAME, TEST_JSON_STRING_NO_IDX_NODE) + .set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NO_IDX_NODE))); + transformWithUnIndexableFieldsAndMergedTextIndex(schemaBuilder.build(), inputJsonNode, expectedJsonNode); + + expectedJsonNodeWithMergedTextIndex = expectedJsonNode.deepCopy().set(MERGED_TEXT_INDEX_FIELD_NAME, + N.arrayNode().add("[0,1,2,3]:arrayField").add("a:stringField").add("[0,1,2,3]:mapField.arrayField") + .add("a:mapField.stringField").add("[0,1,2,3]:nestedFields.arrayField").add("a:nestedFields.stringField") + .add("[0,1,2,3]:nestedFields.mapField.arrayField").add("a:nestedFields.mapField.stringField")); + transformWithUnIndexableFieldsAndMergedTextIndex( + schemaBuilder.addMultiValueDimension(MERGED_TEXT_INDEX_FIELD_NAME, DataType.STRING).build(), inputJsonNode, + expectedJsonNodeWithMergedTextIndex); + + // With schema, mapField is not indexed + schemaBuilder = createDefaultSchemaBuilder().addMultiValueDimension("arrayField", DataType.INT) + .addSingleValueDimension(TEST_JSON_MAP_FIELD_NAME, DataType.STRING) + .addSingleValueDimension(TEST_JSON_NESTED_MAP_FIELD_NAME, DataType.JSON) + .addSingleValueDimension(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_STRING_FIELD_NAME, DataType.STRING); + /* + { + "arrayField":[0, 1, 2, 3], + "nestedFields.stringField":"a", + "indexableExtras":{ + "nullField":null, + "stringField":"a", + "mapField":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a" + }, + "nestedFields":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "mapField":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a" + } + } + }, + "unindexableExtras":{ + "intField_noIndex":9, + "string_noIndex":"z", + "mapField":{ + "intField_noIndex":9, + "string_noIndex":"z" + }, + "mapField_noIndex":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a", + }, + "nestedFields":{ + "intField_noIndex":9, + "string_noIndex":"z", + "mapField":{ + "intField_noIndex":9, + "string_noIndex":"z" + } + } + }, + __mergedTextIndex: [ + "[0, 1, 2, 3]:arrayField", "a:stringField", + "[0, 1, 2, 3]:mapField.arrayField", "a:mapField.stringField", + "[0, 1, 2, 3]:nestedFields.arrayField", "a:nestedFields.stringField", + "[0, 1, 2, 3]:nestedFields.mapField.arrayField", "a:nestedFields.mapField.stringField", + ] + } + */ + expectedJsonNode = CustomObjectNode.create().set(TEST_JSON_ARRAY_FIELD_NAME, TEST_JSON_ARRAY_NODE) + .set(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_STRING_FIELD_NAME, TEST_JSON_STRING_NODE) + .set(INDEXABLE_EXTRAS_FIELD_NAME, CustomObjectNode.create().set(TEST_JSON_NULL_FIELD_NAME, TEST_JSON_NULL_NODE) + .set(TEST_JSON_STRING_FIELD_NAME, TEST_JSON_STRING_NODE).set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NODE) + .set(TEST_JSON_NESTED_MAP_FIELD_NAME, + CustomObjectNode.create().set(TEST_JSON_ARRAY_FIELD_NAME, TEST_JSON_ARRAY_NODE) + .set(TEST_JSON_NULL_FIELD_NAME, TEST_JSON_NULL_NODE) + .set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NODE))) + + .set(UNINDEXABLE_EXTRAS_FIELD_NAME, + CustomObjectNode.create().set(TEST_JSON_INT_NO_IDX_FIELD_NAME, TEST_INT_NODE) + .set(TEST_JSON_STRING_NO_IDX_FIELD_NAME, TEST_JSON_STRING_NO_IDX_NODE) + .set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NO_IDX_NODE) + .set(TEST_JSON_MAP_NO_IDX_FIELD_NAME, TEST_JSON_MAP_NODE).set(TEST_JSON_NESTED_MAP_FIELD_NAME, + CustomObjectNode.create().set(TEST_JSON_INT_NO_IDX_FIELD_NAME, TEST_INT_NODE) + .set(TEST_JSON_STRING_NO_IDX_FIELD_NAME, TEST_JSON_STRING_NO_IDX_NODE) + .set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NO_IDX_NODE))); + transformWithUnIndexableFieldsAndMergedTextIndex(schemaBuilder.build(), inputJsonNode, expectedJsonNode); + + expectedJsonNodeWithMergedTextIndex = expectedJsonNode.deepCopy().set(MERGED_TEXT_INDEX_FIELD_NAME, + N.arrayNode().add("[0,1,2,3]:arrayField").add("a:stringField").add("[0,1,2,3]:mapField.arrayField") + .add("a:mapField.stringField").add("[0,1,2,3]:nestedFields.arrayField").add("a:nestedFields.stringField") + .add("[0,1,2,3]:nestedFields.mapField.arrayField").add("a:nestedFields.mapField.stringField")); + transformWithUnIndexableFieldsAndMergedTextIndex( + schemaBuilder.addMultiValueDimension(MERGED_TEXT_INDEX_FIELD_NAME, DataType.STRING).build(), inputJsonNode, + expectedJsonNodeWithMergedTextIndex); + + // With all fields in schema, but map field would not be indexed + schemaBuilder = createDefaultSchemaBuilder().addMultiValueDimension(TEST_JSON_ARRAY_FIELD_NAME, DataType.INT) + .addSingleValueDimension(TEST_JSON_NULL_FIELD_NAME, DataType.STRING) + .addSingleValueDimension(TEST_JSON_STRING_FIELD_NAME, DataType.STRING) + .addSingleValueDimension(TEST_JSON_MAP_FIELD_NAME, DataType.JSON) + .addMultiValueDimension(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_ARRAY_FIELD_NAME, DataType.INT) + .addSingleValueDimension(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_NULL_FIELD_NAME, DataType.STRING) + .addSingleValueDimension(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_STRING_FIELD_NAME, DataType.STRING) + .addSingleValueDimension(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_MAP_FIELD_NAME, DataType.JSON); + /* + { + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a", + "nestedFields.arrayField":[0, 1, 2, 3], + "nestedFields.nullField":null, + "nestedFields.stringField":"a", + "indexableExtras":{ + "mapField":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a" + }, + "nestedFields":{ + mapField":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a" + } + } + }, + "unindexableExtras":{ + "intField_noIndex":9, + "string_noIndex":"z", + "mapField":{ + "intField_noIndex":9, + "string_noIndex":"z" + }, + "mapField_noIndex":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a", + }, + "nestedFields":{ + "intField_noIndex":9, + "string_noIndex":"z", + "mapField":{ + "intField_noIndex":9, + "string_noIndex":"z" + } + } + }, + __mergedTextIndex: [ + "[0, 1, 2, 3]:arrayField", "a:stringField", + "[0, 1, 2, 3]:mapField.arrayField", "a:mapField.stringField", + "[0, 1, 2, 3]:nestedFields.arrayField", "a:nestedFields.stringField", + "[0, 1, 2, 3]:nestedFields.mapField.arrayField", "a:nestedFields.mapField.stringField", + ] + } + */ + expectedJsonNode = CustomObjectNode.create().set(TEST_JSON_ARRAY_FIELD_NAME, TEST_JSON_ARRAY_NODE) + .set(TEST_JSON_NULL_FIELD_NAME, TEST_JSON_NULL_NODE).set(TEST_JSON_STRING_FIELD_NAME, TEST_JSON_STRING_NODE) + .set(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_ARRAY_FIELD_NAME, TEST_JSON_ARRAY_NODE) + .set(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_NULL_FIELD_NAME, TEST_JSON_NULL_NODE) + .set(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_STRING_FIELD_NAME, TEST_JSON_STRING_NODE) + + .set(INDEXABLE_EXTRAS_FIELD_NAME, CustomObjectNode.create().set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NODE) + .set(TEST_JSON_NESTED_MAP_FIELD_NAME, + CustomObjectNode.create().set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NODE))) + + .set(UNINDEXABLE_EXTRAS_FIELD_NAME, + CustomObjectNode.create().set(TEST_JSON_INT_NO_IDX_FIELD_NAME, TEST_INT_NODE) + .set(TEST_JSON_STRING_NO_IDX_FIELD_NAME, TEST_JSON_STRING_NO_IDX_NODE) + .set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NO_IDX_NODE) + .set(TEST_JSON_MAP_NO_IDX_FIELD_NAME, TEST_JSON_MAP_NODE).set(TEST_JSON_NESTED_MAP_FIELD_NAME, + CustomObjectNode.create().set(TEST_JSON_INT_NO_IDX_FIELD_NAME, TEST_INT_NODE) + .set(TEST_JSON_STRING_NO_IDX_FIELD_NAME, TEST_JSON_STRING_NO_IDX_NODE) + .set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NO_IDX_NODE))); + transformWithUnIndexableFieldsAndMergedTextIndex(schemaBuilder.build(), inputJsonNode, expectedJsonNode); + expectedJsonNodeWithMergedTextIndex = expectedJsonNode.deepCopy().set(MERGED_TEXT_INDEX_FIELD_NAME, + N.arrayNode().add("[0,1,2,3]:arrayField").add("a:stringField").add("[0,1,2,3]:mapField.arrayField") + .add("a:mapField.stringField").add("[0,1,2,3]:nestedFields.arrayField").add("a:nestedFields.stringField") + .add("[0,1,2,3]:nestedFields.mapField.arrayField").add("a:nestedFields.mapField.stringField")); + transformWithUnIndexableFieldsAndMergedTextIndex( + schemaBuilder.addMultiValueDimension(MERGED_TEXT_INDEX_FIELD_NAME, DataType.STRING).build(), inputJsonNode, + expectedJsonNodeWithMergedTextIndex); + } + + @Test + public void testKeyValueTransformation() { + /* + { + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a", + "intField_noIndex":9, + "string_noIndex":"z", + "mapField":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a", + "intField_noIndex":9, + "string_noIndex":"z" + }, + "mapField_noIndex":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a", + }, + "nestedFields":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a", + "intField_noIndex":9, + "string_noIndex":"z", + "mapField":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a", + "intField_noIndex":9, + "string_noIndex":"z" + } + } + } + */ + final CustomObjectNode inputJsonNode = + CustomObjectNode.create().setAll(TEST_JSON_MAP_NODE).set(TEST_JSON_ARRAY_FIELD_NAME, TEST_JSON_ARRAY_NODE) + .set(TEST_JSON_NULL_FIELD_NAME, TEST_JSON_NULL_NODE).set(TEST_JSON_STRING_FIELD_NAME, TEST_JSON_STRING_NODE) + .set(TEST_JSON_INT_NO_IDX_FIELD_NAME, TEST_INT_NODE) + .set(TEST_JSON_STRING_NO_IDX_FIELD_NAME, TEST_JSON_STRING_NO_IDX_NODE) + .set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NODE_WITH_NO_IDX) + .set(TEST_JSON_MAP_NO_IDX_FIELD_NAME, TEST_JSON_MAP_NODE).set(TEST_JSON_NESTED_MAP_FIELD_NAME, + CustomObjectNode.create().setAll(TEST_JSON_MAP_NODE).set(TEST_JSON_ARRAY_FIELD_NAME, TEST_JSON_ARRAY_NODE) + .set(TEST_JSON_NULL_FIELD_NAME, TEST_JSON_NULL_NODE) + .set(TEST_JSON_STRING_FIELD_NAME, TEST_JSON_STRING_NODE) + .set(TEST_JSON_INT_NO_IDX_FIELD_NAME, TEST_INT_NODE) + .set(TEST_JSON_STRING_NO_IDX_FIELD_NAME, TEST_JSON_STRING_NO_IDX_NODE) + .set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NODE_WITH_NO_IDX)); + + CustomObjectNode expectedJsonNode; + CustomObjectNode expectedJsonNodeWithMergedTextIndex; + Schema.SchemaBuilder schemaBuilder; + + String destColumnName = "someMeaningfulName"; + // make array field as single value STRING, test the conversion function + // ignore the column nestedFields + // preserve the entire mapField value + // map the column someMeaningfulName to nestedFields.stringField + schemaBuilder = createDefaultSchemaBuilder().addSingleValueDimension("arrayField", DataType.STRING) + .addSingleValueDimension(TEST_JSON_MAP_FIELD_NAME, DataType.STRING) + .addSingleValueDimension(TEST_JSON_NESTED_MAP_FIELD_NAME, DataType.JSON) + .addSingleValueDimension(destColumnName, DataType.STRING); + + Map keyMapping = new HashMap<>() { + { + put(destColumnName, TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_STRING_FIELD_NAME); + } + }; + Set pathToDrop = new HashSet<>() { + { + add(TEST_JSON_NESTED_MAP_FIELD_NAME + "." + TEST_JSON_MAP_FIELD_NAME); + } + }; + Set pathToPreserve = new HashSet<>() { + { + add(TEST_JSON_MAP_FIELD_NAME); + } + }; + + /* + { + "arrayField":[0,1,2,3], + "nestedFields.stringField":"a", + "mapField":{ + "arrayField":[0,1,2,3], + "nullField":null, + "stringField":"a", + "intField_noIndex":9, + "string_noIndex":"z" + } + "indexableExtras":{ + "nullField":null, + "stringField":"a", + "nestedFields":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + } + }, + "unindexableExtras":{ + "intField_noIndex":9, + "string_noIndex":"z", + "mapField_noIndex":{ + "arrayField":[0, 1, 2, 3], + "nullField":null, + "stringField":"a", + }, + "nestedFields":{ + "intField_noIndex":9, + "string_noIndex":"z" + } + }, + __mergedTextIndex: [ + "[0, 1, 2, 3]:arrayField", "a:stringField", + "[0, 1, 2, 3]:nestedFields.arrayField", "a:nestedFields.stringField", + ] + } + */ + expectedJsonNode = CustomObjectNode.create() + .set(TEST_JSON_ARRAY_FIELD_NAME, N.textNode("[0,1,2,3]")) + .set(destColumnName, TEST_JSON_STRING_NODE) + .set(TEST_JSON_MAP_FIELD_NAME, TEST_JSON_MAP_NODE_WITH_NO_IDX) + .set(INDEXABLE_EXTRAS_FIELD_NAME, + CustomObjectNode.create().set(TEST_JSON_NULL_FIELD_NAME, TEST_JSON_NULL_NODE) + .set(TEST_JSON_STRING_FIELD_NAME, TEST_JSON_STRING_NODE) + .set(TEST_JSON_NESTED_MAP_FIELD_NAME, + CustomObjectNode.create().set(TEST_JSON_ARRAY_FIELD_NAME, TEST_JSON_ARRAY_NODE) + .set(TEST_JSON_NULL_FIELD_NAME, TEST_JSON_NULL_NODE))) + + .set(UNINDEXABLE_EXTRAS_FIELD_NAME, + CustomObjectNode.create().set(TEST_JSON_INT_NO_IDX_FIELD_NAME, TEST_INT_NODE) + .set(TEST_JSON_STRING_NO_IDX_FIELD_NAME, TEST_JSON_STRING_NO_IDX_NODE) + .set(TEST_JSON_MAP_NO_IDX_FIELD_NAME, TEST_JSON_MAP_NODE).set(TEST_JSON_NESTED_MAP_FIELD_NAME, + CustomObjectNode.create().set(TEST_JSON_INT_NO_IDX_FIELD_NAME, TEST_INT_NODE) + .set(TEST_JSON_STRING_NO_IDX_FIELD_NAME, TEST_JSON_STRING_NO_IDX_NODE))); + + expectedJsonNodeWithMergedTextIndex = expectedJsonNode.deepCopy().set(MERGED_TEXT_INDEX_FIELD_NAME, + N.arrayNode().add("[0,1,2,3]:arrayField").add("a:stringField").add("[0,1,2,3]:nestedFields.arrayField").add( + "a:nestedFields.stringField")); + transformKeyValueTransformation( + schemaBuilder.addMultiValueDimension(MERGED_TEXT_INDEX_FIELD_NAME, DataType.STRING).build(), keyMapping, + pathToDrop, pathToPreserve, inputJsonNode, expectedJsonNodeWithMergedTextIndex); + } + + private void transformWithIndexableFields(Schema schema, JsonNode inputRecordJsonNode, JsonNode ouputRecordJsonNode) { + testTransform(INDEXABLE_EXTRAS_FIELD_NAME, null, null, schema, null, null, null, inputRecordJsonNode.toString(), + ouputRecordJsonNode.toString()); + } + + private void transformWithUnIndexableFieldsAndMergedTextIndex(Schema schema, JsonNode inputRecordJsonNode, + JsonNode ouputRecordJsonNode) { + testTransform(INDEXABLE_EXTRAS_FIELD_NAME, UNINDEXABLE_EXTRAS_FIELD_NAME, MERGED_TEXT_INDEX_FIELD_NAME, schema, + null, null, null, inputRecordJsonNode.toString(), ouputRecordJsonNode.toString()); + } + + private void transformKeyValueTransformation(Schema schema, Map keyMapping, + Set fieldPathsToDrop, Set fieldPathsToPreserve, JsonNode inputRecordJsonNode, + JsonNode ouputRecordJsonNode) { + testTransform(INDEXABLE_EXTRAS_FIELD_NAME, UNINDEXABLE_EXTRAS_FIELD_NAME, MERGED_TEXT_INDEX_FIELD_NAME, schema, + keyMapping, fieldPathsToDrop, fieldPathsToPreserve, inputRecordJsonNode.toString(), + ouputRecordJsonNode.toString()); + } + + private void testTransform(String indexableExtrasField, String unindexableExtrasField, String mergedTextIndexField, + Schema schema, Map keyMapping, Set fieldPathsToDrop, Set fieldPathsToPreserve, + String inputRecordJSONString, + String expectedOutputRecordJSONString) { + TableConfig tableConfig = + createDefaultTableConfig(indexableExtrasField, unindexableExtrasField, UNINDEXABLE_FIELD_SUFFIX, + fieldPathsToDrop, fieldPathsToPreserve, mergedTextIndexField); + tableConfig.getIngestionConfig().getSchemaConformingTransformerV2Config().setColumnNameToJsonKeyPathMap(keyMapping); + GenericRow outputRecord = transformRow(tableConfig, schema, inputRecordJSONString); + Map expectedOutputRecordMap = jsonStringToMap(expectedOutputRecordJSONString); + + // Merged text index field does not need to have deterministic order + Object mergedTextIndexValue = outputRecord.getFieldToValueMap().get(MERGED_TEXT_INDEX_FIELD_NAME); + Object expectedMergedTextIndexValue = expectedOutputRecordMap.get(MERGED_TEXT_INDEX_FIELD_NAME); + if (mergedTextIndexValue != null) { + ((List) mergedTextIndexValue).sort(null); + } + if (expectedMergedTextIndexValue != null) { + ((List) expectedMergedTextIndexValue).sort(null); + } + + Assert.assertNotNull(outputRecord); + Assert.assertEquals(outputRecord.getFieldToValueMap(), expectedOutputRecordMap); + } + + /** + * Transforms the given row (given as a JSON string) using the transformer + * @return The transformed row + */ + private GenericRow transformRow(TableConfig tableConfig, Schema schema, String inputRecordJSONString) { + Map inputRecordMap = jsonStringToMap(inputRecordJSONString); + GenericRow inputRecord = createRowFromMap(inputRecordMap); + SchemaConformingTransformerV2 schemaConformingTransformerV2 = + new SchemaConformingTransformerV2(tableConfig, schema); + return schemaConformingTransformerV2.transform(inputRecord); + } + + /** + * @return A map representing the given JSON string + */ + @Nonnull + private Map jsonStringToMap(String jsonString) { + try { + TypeReference> typeRef = new TypeReference<>() { + }; + return OBJECT_MAPPER.readValue(jsonString, typeRef); + } catch (IOException e) { + fail(e.getMessage()); + } + // Should never reach here + return null; + } + + /** + * @return A new generic row with all the kv-pairs from the given map + */ + private GenericRow createRowFromMap(Map map) { + GenericRow record = new GenericRow(); + for (Map.Entry entry : map.entrySet()) { + record.putValue(entry.getKey(), entry.getValue()); + } + return record; + } + + @Test + public void testOverlappingSchemaFields() { + try { + Schema schema = createDefaultSchemaBuilder().addSingleValueDimension("a.b", DataType.STRING) + .addSingleValueDimension("a.b.c", DataType.INT).build(); + SchemaConformingTransformerV2.validateSchema(schema, + new SchemaConformingTransformerV2Config(null, INDEXABLE_EXTRAS_FIELD_NAME, null, null, null, null, null, null, + null, null, null, null, null)); + } catch (Exception ex) { + fail("Should not have thrown any exception when overlapping schema occurs"); + } + + try { + // This is a repeat of the previous test but with fields reversed just in case they are processed in order + Schema schema = createDefaultSchemaBuilder().addSingleValueDimension("a.b.c", DataType.INT) + .addSingleValueDimension("a.b", DataType.STRING).build(); + SchemaConformingTransformerV2.validateSchema(schema, + new SchemaConformingTransformerV2Config(null, INDEXABLE_EXTRAS_FIELD_NAME, null, null, null, null, null, null, + null, null, null, null, null)); + } catch (Exception ex) { + fail("Should not have thrown any exception when overlapping schema occurs"); + } + } + + @Test + public void testBase64ValueFilter() { + String text = "Hello world"; + String binaryData = "ABCxyz12345-_+/="; + String binaryDataWithTrailingPeriods = "ABCxyz12345-_+/=.."; + String binaryDataWithRandomPeriods = "A.BCxy.z12345-_+/=.."; + String shortBinaryData = "short"; + int minLength = 10; + + assertFalse(_RECORD_TRANSFORMER.base64ValueFilter(text.getBytes(), minLength)); + assertTrue(_RECORD_TRANSFORMER.base64ValueFilter(binaryData.getBytes(), minLength)); + assertTrue(_RECORD_TRANSFORMER.base64ValueFilter(binaryDataWithTrailingPeriods.getBytes(), minLength)); + assertFalse(_RECORD_TRANSFORMER.base64ValueFilter(binaryDataWithRandomPeriods.getBytes(), minLength)); + assertFalse(_RECORD_TRANSFORMER.base64ValueFilter(shortBinaryData.getBytes(), minLength)); + } + + @Test + public void testShingleIndexTokenization() { + String key = "key"; + String value = "0123456789ABCDEFGHIJ"; + int shingleIndexMaxLength; + int shingleIndexOverlapLength; + List expectedTokenValues; + + shingleIndexMaxLength = 8; + shingleIndexOverlapLength = 1; + expectedTokenValues = new ArrayList<>( + Arrays.asList("0123:key", "3456:key", "6789:key", "9ABC:key", "CDEF:key", "FGHI:key", "IJ:key")); + testShingleIndexWithParams(key, value, shingleIndexMaxLength, shingleIndexOverlapLength, expectedTokenValues); + + shingleIndexMaxLength = 8; + shingleIndexOverlapLength = 2; + expectedTokenValues = new ArrayList<>(Arrays + .asList("0123:key", "2345:key", "4567:key", "6789:key", "89AB:key", "ABCD:key", "CDEF:key", "EFGH:key", + "GHIJ:key")); + testShingleIndexWithParams(key, value, shingleIndexMaxLength, shingleIndexOverlapLength, expectedTokenValues); + + // If shingleIndexMaxLength is lower than the minimum required length for merged text index token + // (length of the key + shingling overlap length + 1), then the shingleIndexMaxLength is adjusted to + // the maximum Lucene token size (32766) + shingleIndexMaxLength = 1; + shingleIndexOverlapLength = 5; + expectedTokenValues = new ArrayList<>(Arrays.asList(value + ":" + key)); + testShingleIndexWithParams(key, value, shingleIndexMaxLength, shingleIndexOverlapLength, expectedTokenValues); + + // If shingleIndexOverlapLength is equal to or longer than the length of the value, shingling cannot be applied and + // only one token is generated. + shingleIndexMaxLength = 32766; + shingleIndexOverlapLength = 100; + expectedTokenValues = new ArrayList<>(Arrays.asList(value + ":" + key)); + testShingleIndexWithParams(key, value, shingleIndexMaxLength, shingleIndexOverlapLength, expectedTokenValues); + + // Other corner cases, where the result would be the same as if shingling has not been applied + shingleIndexMaxLength = 300; + shingleIndexOverlapLength = 10; + expectedTokenValues = new ArrayList<>(Arrays.asList(value + ":" + key)); + testShingleIndexWithParams(key, value, shingleIndexMaxLength, shingleIndexOverlapLength, expectedTokenValues); + } + + private void testShingleIndexWithParams(String key, String value, Integer shingleIndexMaxLength, + Integer shingleIndexOverlapLength, List expectedTokenValues) { + Map.Entry kv = new AbstractMap.SimpleEntry<>(key, value); + List shingleIndexTokens = new ArrayList<>(); + _RECORD_TRANSFORMER + .generateShingleTextIndexDocument(kv, shingleIndexTokens, shingleIndexMaxLength, shingleIndexOverlapLength); + int numTokens = shingleIndexTokens.size(); + assertEquals(numTokens, expectedTokenValues.size()); + for (int i = 0; i < numTokens; i++) { + assertEquals(shingleIndexTokens.get(i), expectedTokenValues.get(i)); + } + } + + static class CustomObjectNode extends ObjectNode { + public CustomObjectNode() { + super(OBJECT_MAPPER.getNodeFactory()); + } + + public static CustomObjectNode create() { + return new CustomObjectNode(); + } + + public CustomObjectNode set(String fieldName, JsonNode value) { + super.set(fieldName, value); + return this; + } + + public CustomObjectNode setAll(ObjectNode other) { + super.setAll(other); + return this; + } + + public CustomObjectNode removeAndReturn(String fieldName) { + super.remove(fieldName); + return this; + } + + public CustomObjectNode deepCopy() { + return CustomObjectNode.create().setAll(this); + } + } + + static { + ServerMetrics.register(mock(ServerMetrics.class)); + } +} diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/utils/Base64UtilsTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/utils/Base64UtilsTest.java new file mode 100644 index 000000000000..e067321404f5 --- /dev/null +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/utils/Base64UtilsTest.java @@ -0,0 +1,96 @@ +/** + * 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.segment.local.utils; + +import java.nio.charset.StandardCharsets; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; + +public class Base64UtilsTest { + private static final String SPECIAL_CHARS = "+/=-_"; + private static final String UPPER_CASE_CHARS = "ABCDEFGHIJKLMNOPQRSTUVWXYZ"; + private static final String LOWER_CASE_CHARS = "abcdefghijklmnopqrstuvwxyz"; + private static final String NUMBER_CHARS = "0123456789"; + private static final String[] BASE64_STRINGS = { + UPPER_CASE_CHARS, + LOWER_CASE_CHARS, + SPECIAL_CHARS, + NUMBER_CHARS, + SPECIAL_CHARS + NUMBER_CHARS + LOWER_CASE_CHARS + UPPER_CASE_CHARS, + UPPER_CASE_CHARS + SPECIAL_CHARS + LOWER_CASE_CHARS + NUMBER_CHARS + }; + private static final String[] BASE64_STRINGS_WITH_WHITE_SPACE = { + SPECIAL_CHARS + "\n" + NUMBER_CHARS + "\t" + LOWER_CASE_CHARS + " " + UPPER_CASE_CHARS, + UPPER_CASE_CHARS + "\n" + SPECIAL_CHARS + "\t" + LOWER_CASE_CHARS + " " + NUMBER_CHARS + }; + private static final String[] NON_BASE64_STRINGS = { + UPPER_CASE_CHARS + "!", + LOWER_CASE_CHARS + "@", + SPECIAL_CHARS + "#", + NUMBER_CHARS + "$", + SPECIAL_CHARS + ".." + NUMBER_CHARS + "?" + LOWER_CASE_CHARS + "^" + UPPER_CASE_CHARS + "*" + }; + + @Test + public void testBase64IgnoreWhiteSpace() { + for (final String s : BASE64_STRINGS) { + assertTrue(Base64Utils.isBase64IgnoreWhiteSpace(s.getBytes(StandardCharsets.UTF_8))); + assertFalse(Base64Utils.isBase64IgnoreWhiteSpace((s + "..").getBytes(StandardCharsets.UTF_8))); + } + + for (final String s : BASE64_STRINGS_WITH_WHITE_SPACE) { + assertTrue(Base64Utils.isBase64IgnoreWhiteSpace(s.getBytes(StandardCharsets.UTF_8))); + assertFalse(Base64Utils.isBase64IgnoreWhiteSpace((s + "..").getBytes(StandardCharsets.UTF_8))); + } + + for (final String s : NON_BASE64_STRINGS) { + assertFalse(Base64Utils.isBase64IgnoreWhiteSpace(s.getBytes(StandardCharsets.UTF_8))); + assertFalse(Base64Utils.isBase64IgnoreWhiteSpace((s + "..").getBytes(StandardCharsets.UTF_8))); + } + } + + @Test + public void testBase64IgnoreTrailingPeriods() { + for (final String s : BASE64_STRINGS) { + String testStr = s; + for (int i = 0; i < 10; i++) { + assertTrue(Base64Utils.isBase64IgnoreTrailingPeriods(testStr.getBytes(StandardCharsets.UTF_8))); + testStr = testStr + "."; + } + } + + for (final String s : BASE64_STRINGS_WITH_WHITE_SPACE) { + String testStr = s; + for (int i = 0; i < 2; i++) { + assertFalse(Base64Utils.isBase64IgnoreTrailingPeriods(testStr.getBytes(StandardCharsets.UTF_8))); + testStr = testStr + "."; + } + } + + for (final String s : NON_BASE64_STRINGS) { + String testStr = s; + for (int i = 0; i < 2; i++) { + assertFalse(Base64Utils.isBase64IgnoreTrailingPeriods(testStr.getBytes(StandardCharsets.UTF_8))); + testStr = testStr + "."; + } + } + } +} diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/IngestionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/IngestionConfig.java index 5af9cdcc5076..358cf35a43ac 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/IngestionConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/IngestionConfig.java @@ -51,6 +51,9 @@ public class IngestionConfig extends BaseJsonConfig { @JsonPropertyDescription("Config related to the SchemaConformingTransformer") private SchemaConformingTransformerConfig _schemaConformingTransformerConfig; + @JsonPropertyDescription("Config related to the SchemaConformingTransformerV2") + private SchemaConformingTransformerV2Config _schemaConformingTransformerV2Config; + @JsonPropertyDescription("Configs related to record aggregation function applied during ingestion") private List _aggregationConfigs; @@ -69,6 +72,7 @@ public IngestionConfig(@Nullable BatchIngestionConfig batchIngestionConfig, @Nullable List enrichmentConfigs, @Nullable List transformConfigs, @Nullable ComplexTypeConfig complexTypeConfig, @Nullable SchemaConformingTransformerConfig schemaConformingTransformerConfig, + @Nullable SchemaConformingTransformerV2Config schemaConformingTransformerV2Config, @Nullable List aggregationConfigs) { _batchIngestionConfig = batchIngestionConfig; _streamIngestionConfig = streamIngestionConfig; @@ -77,6 +81,7 @@ public IngestionConfig(@Nullable BatchIngestionConfig batchIngestionConfig, _transformConfigs = transformConfigs; _complexTypeConfig = complexTypeConfig; _schemaConformingTransformerConfig = schemaConformingTransformerConfig; + _schemaConformingTransformerV2Config = schemaConformingTransformerV2Config; _aggregationConfigs = aggregationConfigs; } @@ -118,6 +123,11 @@ public SchemaConformingTransformerConfig getSchemaConformingTransformerConfig() return _schemaConformingTransformerConfig; } + @Nullable + public SchemaConformingTransformerV2Config getSchemaConformingTransformerV2Config() { + return _schemaConformingTransformerV2Config; + } + @Nullable public List getAggregationConfigs() { return _aggregationConfigs; @@ -164,6 +174,11 @@ public void setSchemaConformingTransformerConfig( _schemaConformingTransformerConfig = schemaConformingTransformerConfig; } + public void setSchemaConformingTransformerV2Config( + SchemaConformingTransformerV2Config schemaConformingTransformerV2Config) { + _schemaConformingTransformerV2Config = schemaConformingTransformerV2Config; + } + public void setAggregationConfigs(List aggregationConfigs) { _aggregationConfigs = aggregationConfigs; } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/SchemaConformingTransformerConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/SchemaConformingTransformerConfig.java index 96231f39d9ab..e51eb65e4aef 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/SchemaConformingTransformerConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/SchemaConformingTransformerConfig.java @@ -31,8 +31,8 @@ public class SchemaConformingTransformerConfig extends BaseJsonConfig { @JsonPropertyDescription("Name of the field that should contain extra fields that are not part of the schema.") private final String _indexableExtrasField; - @JsonPropertyDescription( - "Like indexableExtrasField except it only contains fields with the suffix in unindexableFieldSuffix.") + @JsonPropertyDescription("Like indexableExtrasField except it only contains fields with the suffix in " + + "unindexableFieldSuffix.") private final String _unindexableExtrasField; @JsonPropertyDescription("The suffix of fields that must be stored in unindexableExtrasField") diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/SchemaConformingTransformerV2Config.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/SchemaConformingTransformerV2Config.java new file mode 100644 index 000000000000..1d58d76f8145 --- /dev/null +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/SchemaConformingTransformerV2Config.java @@ -0,0 +1,253 @@ +/** + * 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.spi.config.table.ingestion; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.pinot.spi.config.BaseJsonConfig; + +public class SchemaConformingTransformerV2Config extends BaseJsonConfig { + @JsonPropertyDescription("Enable indexable extras") + private boolean _enableIndexableExtras = true; + + @JsonPropertyDescription("Name of the field that should contain extra fields that are not part of the schema.") + private String _indexableExtrasField = "json_data"; + + @JsonPropertyDescription("Enable unindexable extras") + private boolean _enableUnindexableExtras = true; + + @JsonPropertyDescription( + "Like indexableExtrasField except it only contains fields with the suffix in unindexableFieldSuffix.") + private String _unindexableExtrasField = "json_data_no_idx"; + + @JsonPropertyDescription("The suffix of fields that must be stored in unindexableExtrasField") + private String _unindexableFieldSuffix = "_noindex"; + + @JsonPropertyDescription("Array of flattened (dot-delimited) object paths to drop") + private Set _fieldPathsToDrop = new HashSet<>(); + + @JsonPropertyDescription("Array of flattened (dot-delimited) object paths not to traverse further and keep same as " + + "input. This will also skip building mergedTextIndex for the field.") + private Set _fieldPathsToPreserveInput = new HashSet<>(); + + @JsonPropertyDescription("Map from customized meaningful column name to json key path") + private Map _columnNameToJsonKeyPathMap = new HashMap<>(); + + @JsonPropertyDescription("mergedTextIndex field") + private String _mergedTextIndexField = "__mergedTextIndex"; + + @JsonPropertyDescription("mergedTextIndex document max length") + private int _mergedTextIndexDocumentMaxLength = 32766; + + @JsonPropertyDescription( + "Recall that merged text index document is in the format of . " + + "The mergedTextIndex shingling overlap length refers to the " + + "maximum search length of the value that will yield results with " + + "100% accuracy. If the value is null, shingle index will be turned off " + + "and the value will be truncated such that the document is equal to " + + "_mergedTextIndexDocumentMaxLength" + ) + private @Nullable Integer _mergedTextIndexShinglingOverlapLength = null; + + @JsonPropertyDescription("mergedTextIndex binary document detection minimum length") + private Integer _mergedTextIndexBinaryDocumentDetectionMinLength = 512; + + @JsonPropertyDescription("Array of paths to exclude from merged text index.") + private Set _mergedTextIndexPathToExclude = new HashSet<>(); + + // TODO: set default value from CLPRewriter once it open sourced + @JsonPropertyDescription("Array of suffix to exclude from merged text index.") + private List _mergedTextIndexSuffixToExclude = Arrays.asList("_logtype", "_dictionaryVars", "_encodedVars"); + + @JsonPropertyDescription("Dedicated fields to double ingest into json_data column") + private Set _fieldsToDoubleIngest = new HashSet<>(); + + @JsonCreator + public SchemaConformingTransformerV2Config( + @JsonProperty("enableIndexableExtras") @Nullable Boolean enableIndexableExtras, + @JsonProperty("indexableExtrasField") String indexableExtrasField, + @JsonProperty("enableUnindexableExtras") @Nullable Boolean enableUnindexableExtras, + @JsonProperty("unindexableExtrasField") @Nullable String unindexableExtrasField, + @JsonProperty("unindexableFieldSuffix") @Nullable String unindexableFieldSuffix, + @JsonProperty("fieldPathsToDrop") @Nullable Set fieldPathsToDrop, + @JsonProperty("fieldPathsToKeepSameAsInput") @Nullable Set fieldPathsToPreserveInput, + @JsonProperty("mergedTextIndexField") @Nullable String mergedTextIndexField, + @JsonProperty("mergedTextIndexDocumentMaxLength") @Nullable Integer mergedTextIndexDocumentMaxLength, + @JsonProperty("mergedTextIndexShinglingOverlapLength") @Nullable Integer mergedTextIndexShinglingOverlapLength, + @JsonProperty("mergedTextIndexBinaryDocumentDetectionMinLength") + @Nullable Integer mergedTextIndexBinaryDocumentDetectionMinLength, + @JsonProperty("mergedTextIndexPathToExclude") @Nullable Set mergedTextIndexPathToExclude, + @JsonProperty("fieldsToDoubleIngest") @Nullable Set fieldsToDoubleIngest + ) { + setEnableIndexableExtras(enableIndexableExtras); + setIndexableExtrasField(indexableExtrasField); + setEnableUnindexableExtras(enableUnindexableExtras); + setUnindexableExtrasField(unindexableExtrasField); + setUnindexableFieldSuffix(unindexableFieldSuffix); + setFieldPathsToDrop(fieldPathsToDrop); + setFieldPathsToPreserveInput(fieldPathsToPreserveInput); + + setMergedTextIndexField(mergedTextIndexField); + setMergedTextIndexDocumentMaxLength(mergedTextIndexDocumentMaxLength); + setMergedTextIndexShinglingDocumentOverlapLength(mergedTextIndexShinglingOverlapLength); + setMergedTextIndexBinaryDocumentDetectionMinLength(mergedTextIndexBinaryDocumentDetectionMinLength); + setMergedTextIndexPathToExclude(mergedTextIndexPathToExclude); + setFieldsToDoubleIngest(fieldsToDoubleIngest); + } + + public SchemaConformingTransformerV2Config setEnableIndexableExtras(Boolean enableIndexableExtras) { + _enableIndexableExtras = enableIndexableExtras == null ? _enableUnindexableExtras : enableIndexableExtras; + return this; + } + + public String getIndexableExtrasField() { + return _enableIndexableExtras ? _indexableExtrasField : null; + } + + public SchemaConformingTransformerV2Config setIndexableExtrasField(String indexableExtrasField) { + _indexableExtrasField = indexableExtrasField == null ? _indexableExtrasField : indexableExtrasField; + return this; + } + + public SchemaConformingTransformerV2Config setEnableUnindexableExtras(Boolean enableUnindexableExtras) { + _enableUnindexableExtras = enableUnindexableExtras == null ? _enableUnindexableExtras : enableUnindexableExtras; + return this; + } + + public String getUnindexableExtrasField() { + return _enableUnindexableExtras ? _unindexableExtrasField : null; + } + + public SchemaConformingTransformerV2Config setUnindexableExtrasField(String unindexableExtrasField) { + _unindexableExtrasField = unindexableExtrasField == null ? _unindexableExtrasField : unindexableExtrasField; + return this; + } + + public String getUnindexableFieldSuffix() { + return _unindexableFieldSuffix; + } + + public SchemaConformingTransformerV2Config setUnindexableFieldSuffix(String unindexableFieldSuffix) { + _unindexableFieldSuffix = unindexableFieldSuffix == null ? _unindexableFieldSuffix : unindexableFieldSuffix; + return this; + } + + public Set getFieldPathsToDrop() { + return _fieldPathsToDrop; + } + + public SchemaConformingTransformerV2Config setFieldPathsToDrop(Set fieldPathsToDrop) { + _fieldPathsToDrop = fieldPathsToDrop == null ? _fieldPathsToDrop : fieldPathsToDrop; + return this; + } + + public Set getFieldPathsToPreserveInput() { + return _fieldPathsToPreserveInput; + } + + public SchemaConformingTransformerV2Config setFieldPathsToPreserveInput(Set fieldPathsToPreserveInput) { + _fieldPathsToPreserveInput = fieldPathsToPreserveInput == null ? _fieldPathsToPreserveInput + : fieldPathsToPreserveInput; + return this; + } + + public Map getColumnNameToJsonKeyPathMap() { + return _columnNameToJsonKeyPathMap; + } + + public SchemaConformingTransformerV2Config setColumnNameToJsonKeyPathMap( + Map columnNameToJsonKeyPathMap) { + _columnNameToJsonKeyPathMap = columnNameToJsonKeyPathMap == null + ? _columnNameToJsonKeyPathMap : columnNameToJsonKeyPathMap; + return this; + } + + public String getMergedTextIndexField() { + return _mergedTextIndexField; + } + + public SchemaConformingTransformerV2Config setMergedTextIndexField(String mergedTextIndexField) { + _mergedTextIndexField = mergedTextIndexField == null ? _mergedTextIndexField : mergedTextIndexField; + return this; + } + + public Integer getMergedTextIndexDocumentMaxLength() { + return _mergedTextIndexDocumentMaxLength; + } + + public SchemaConformingTransformerV2Config setMergedTextIndexDocumentMaxLength( + Integer mergedTextIndexDocumentMaxLength + ) { + _mergedTextIndexDocumentMaxLength = mergedTextIndexDocumentMaxLength == null + ? _mergedTextIndexDocumentMaxLength : mergedTextIndexDocumentMaxLength; + return this; + } + + public Integer getMergedTextIndexShinglingOverlapLength() { + return _mergedTextIndexShinglingOverlapLength; + } + + public SchemaConformingTransformerV2Config setMergedTextIndexShinglingDocumentOverlapLength( + Integer mergedTextIndexShinglingOverlapLength) { + _mergedTextIndexShinglingOverlapLength = mergedTextIndexShinglingOverlapLength; + return this; + } + + public Integer getMergedTextIndexBinaryDocumentDetectionMinLength() { + return _mergedTextIndexBinaryDocumentDetectionMinLength; + } + + public SchemaConformingTransformerV2Config setMergedTextIndexBinaryDocumentDetectionMinLength( + Integer mergedTextIndexBinaryDocumentDetectionMinLength) { + _mergedTextIndexBinaryDocumentDetectionMinLength = mergedTextIndexBinaryDocumentDetectionMinLength == null + ? _mergedTextIndexBinaryDocumentDetectionMinLength : mergedTextIndexBinaryDocumentDetectionMinLength; + return this; + } + + public Set getMergedTextIndexPathToExclude() { + return _mergedTextIndexPathToExclude; + } + + public List getMergedTextIndexSuffixToExclude() { + return _mergedTextIndexSuffixToExclude; + } + + public SchemaConformingTransformerV2Config setMergedTextIndexPathToExclude(Set mergedTextIndexPathToExclude) { + _mergedTextIndexPathToExclude = mergedTextIndexPathToExclude == null + ? _mergedTextIndexPathToExclude : mergedTextIndexPathToExclude; + return this; + } + + public Set getFieldsToDoubleIngest() { + return _fieldsToDoubleIngest; + } + + public SchemaConformingTransformerV2Config setFieldsToDoubleIngest(Set fieldsToDoubleIngest) { + _fieldsToDoubleIngest = fieldsToDoubleIngest == null ? _fieldsToDoubleIngest : fieldsToDoubleIngest; + return this; + } +} From 85806e4398e4f0b2b775ef5d35e0c6ffcea7ae6a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 10 Apr 2024 10:05:14 -0700 Subject: [PATCH 41/72] Bump org.testng:testng from 7.9.0 to 7.10.1 (#12868) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 8b637cd7e8ad..ed89edda35e7 100644 --- a/pom.xml +++ b/pom.xml @@ -170,7 +170,7 @@ 2.25.27 2.12.7 3.1.12 - 7.9.0 + 7.10.1 6.4.0 8.2.0 From 02dd1339521fb066db88ee101dfdb0a1a1613269 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 10 Apr 2024 10:10:14 -0700 Subject: [PATCH 42/72] Bump org.codehaus.woodstox:stax2-api from 4.1 to 4.2.2 (#12851) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index ed89edda35e7..78ce816feb97 100644 --- a/pom.xml +++ b/pom.xml @@ -166,7 +166,7 @@ 1.37 0.15.0 0.4.4 - 4.1 + 4.2.2 2.25.27 2.12.7 3.1.12 From 6bc2450a73791e3d5ba4594a02651a7b07745da6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 10 Apr 2024 10:17:39 -0700 Subject: [PATCH 43/72] Bump org.glassfish.tyrus.bundles:tyrus-standalone-client (#12869) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 78ce816feb97..8d309851a4f4 100644 --- a/pom.xml +++ b/pom.xml @@ -1373,7 +1373,7 @@ org.glassfish.tyrus.bundles tyrus-standalone-client - 2.1.4 + 2.1.5 From 0e3a9a64affb3ee69e14657256ff0f83d8be7128 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 10 Apr 2024 10:19:07 -0700 Subject: [PATCH 44/72] Bump com.google.j2objc:j2objc-annotations from 2.8 to 3.0.0 (#12870) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 8d309851a4f4..4cdfe1dfc2ff 100644 --- a/pom.xml +++ b/pom.xml @@ -882,7 +882,7 @@ com.google.j2objc j2objc-annotations - 2.8 + 3.0.0 From ae819e2321b821da5bc73906e5313de9675b7924 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 10 Apr 2024 10:36:36 -0700 Subject: [PATCH 45/72] Bump com.github.eirslett:frontend-maven-plugin from 1.11.0 to 1.15.0 (#12874) --- pinot-controller/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-controller/pom.xml b/pinot-controller/pom.xml index 9b995c23e971..f70dde647129 100644 --- a/pinot-controller/pom.xml +++ b/pinot-controller/pom.xml @@ -144,7 +144,7 @@ com.github.eirslett frontend-maven-plugin - 1.11.0 + 1.15.0 From c39a97845a6d4672e3a5ba1a1dc3accb2ea5e3dd Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 10 Apr 2024 10:37:10 -0700 Subject: [PATCH 46/72] Bump org.apache.spark:spark-launcher_2.12 from 3.5.0 to 3.5.1 (#12877) --- pinot-tools/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-tools/pom.xml b/pinot-tools/pom.xml index 323f4675c845..26c466ee0c5c 100644 --- a/pinot-tools/pom.xml +++ b/pinot-tools/pom.xml @@ -33,7 +33,7 @@ https://pinot.apache.org/ ${basedir}/.. - 3.5.0 + 3.5.1 From 870fa2a810835af559572c7f8549aef72e236bdd Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 10 Apr 2024 10:37:33 -0700 Subject: [PATCH 47/72] Bump org.apache.maven.plugins:maven-project-info-reports-plugin (#12876) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 4cdfe1dfc2ff..5cad15b642ac 100644 --- a/pom.xml +++ b/pom.xml @@ -2330,7 +2330,7 @@ org.apache.maven.plugins maven-project-info-reports-plugin - 2.7 + 3.5.0 false From 30a3cd5a9c3fb7ccad53f43ff4800956c59379cd Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 10 Apr 2024 10:40:33 -0700 Subject: [PATCH 48/72] Bump io.grpc:grpc-context from 1.60.1 to 1.63.0 (#12872) --- pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml index 9ad9a4b62e64..99af7c084abc 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml @@ -38,7 +38,7 @@ package ${basedir}/../../.. 0.16.0 - 1.60.1 + 1.63.0 1.62.2 2.6.2 1.17 From 9af7cd6c102ce78b6c0d33cd10440e45d27d1190 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 10 Apr 2024 11:05:58 -0700 Subject: [PATCH 49/72] Bump com.fasterxml.woodstox:woodstox-core from 6.4.0 to 6.6.2 (#12873) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 5cad15b642ac..fb6071f09204 100644 --- a/pom.xml +++ b/pom.xml @@ -171,7 +171,7 @@ 2.12.7 3.1.12 7.10.1 - 6.4.0 + 6.6.2 8.2.0 From f2d27da607e666e098b04ca5ad16754601cb965d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 10 Apr 2024 11:06:29 -0700 Subject: [PATCH 50/72] Bump aws.sdk.version from 2.25.27 to 2.25.28 (#12871) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index fb6071f09204..a807169f97bb 100644 --- a/pom.xml +++ b/pom.xml @@ -167,7 +167,7 @@ 0.15.0 0.4.4 4.2.2 - 2.25.27 + 2.25.28 2.12.7 3.1.12 7.10.1 From 9b64b5cdd070aeade025b440f97b20a26ef32f83 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Thu, 11 Apr 2024 02:25:31 +0800 Subject: [PATCH 51/72] Support ValueWindowFunction for LEAD/LAG/FIRST_VALUE/LAST_VALUE (#12878) --- .../PinotWindowExchangeNodeInsertRule.java | 2 +- .../pinot/query/QueryEnvironmentTestBase.java | 2 + .../operator/WindowAggregateOperator.java | 36 +++++++++++-- .../operator/utils/AggregationUtils.java | 2 +- .../window/FirstValueWindowFunction.java | 40 ++++++++++++++ .../window/LagValueWindowFunction.java | 48 +++++++++++++++++ .../window/LastValueWindowFunction.java | 40 ++++++++++++++ .../window/LeadValueWindowFunction.java | 48 +++++++++++++++++ .../operator/window/ValueWindowFunction.java | 54 +++++++++++++++++++ .../operator/window/WindowFunction.java | 31 +++++++++++ 10 files changed, 296 insertions(+), 7 deletions(-) create mode 100644 pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/FirstValueWindowFunction.java create mode 100644 pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/LagValueWindowFunction.java create mode 100644 pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/LastValueWindowFunction.java create mode 100644 pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/LeadValueWindowFunction.java create mode 100644 pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/ValueWindowFunction.java create mode 100644 pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/WindowFunction.java diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java index 317a406332ae..e9caf1216fb1 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotWindowExchangeNodeInsertRule.java @@ -65,7 +65,7 @@ public class PinotWindowExchangeNodeInsertRule extends RelOptRule { // OTHER_FUNCTION supported are: BOOL_AND, BOOL_OR private static final Set SUPPORTED_WINDOW_FUNCTION_KIND = ImmutableSet.of(SqlKind.SUM, SqlKind.SUM0, SqlKind.MIN, SqlKind.MAX, SqlKind.COUNT, SqlKind.ROW_NUMBER, SqlKind.RANK, SqlKind.DENSE_RANK, - SqlKind.OTHER_FUNCTION); + SqlKind.LAG, SqlKind.LEAD, SqlKind.FIRST_VALUE, SqlKind.LAST_VALUE, SqlKind.OTHER_FUNCTION); public PinotWindowExchangeNodeInsertRule(RelBuilderFactory factory) { super(operand(LogicalWindow.class, any()), factory, null); diff --git a/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryEnvironmentTestBase.java b/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryEnvironmentTestBase.java index 9a97e75b887e..6b3b8a363170 100644 --- a/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryEnvironmentTestBase.java +++ b/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryEnvironmentTestBase.java @@ -189,6 +189,8 @@ protected Object[][] provideQueries() { + "RANK() OVER(ORDER BY count(*) DESC) AS rank FROM a GROUP BY a.col1) WHERE rank < 5" }, new Object[]{"SELECT RANK() OVER(PARTITION BY a.col2 ORDER BY a.col1) FROM a"}, + new Object[]{"SELECT a.col1, LEAD(a.col3) OVER (PARTITION BY a.col2 ORDER BY a.col3) FROM a"}, + new Object[]{"SELECT a.col1, LAG(a.col3) OVER (PARTITION BY a.col2 ORDER BY a.col3) FROM a"}, new Object[]{"SELECT DENSE_RANK() OVER(ORDER BY a.col1) FROM a"}, new Object[]{"SELECT a.col1, SUM(a.col3) OVER (ORDER BY a.col2), MIN(a.col3) OVER (ORDER BY a.col2) FROM a"}, new Object[]{ diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java index 259abaea1b60..d2e37598a0f1 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/WindowAggregateOperator.java @@ -44,6 +44,7 @@ import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.operator.utils.AggregationUtils; import org.apache.pinot.query.runtime.operator.utils.TypeUtils; +import org.apache.pinot.query.runtime.operator.window.ValueWindowFunction; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -175,6 +176,11 @@ protected TransferableBlock getNextBlock() { private void validateAggregationCalls(String functionName, Map> mergers) { + if (ValueWindowFunction.VALUE_WINDOW_FUNCTION_MAP.containsKey(functionName)) { + Preconditions.checkState(_windowFrame.getWindowFrameType() == WindowNode.WindowFrameType.RANGE, + String.format("Only RANGE type frames are supported at present for VALUE function: %s", functionName)); + return; + } if (!mergers.containsKey(functionName)) { throw new IllegalStateException("Unexpected aggregation function name: " + functionName); } @@ -219,13 +225,18 @@ private TransferableBlock produceWindowAggregatedBlock() { for (Map.Entry> e : _partitionRows.entrySet()) { Key partitionKey = e.getKey(); List rowList = e.getValue(); - for (Object[] existingRow : rowList) { + for (int rowId = 0; rowId < rowList.size(); rowId++) { + Object[] existingRow = rowList.get(rowId); Object[] row = new Object[existingRow.length + _aggCalls.size()]; Key orderKey = (_isPartitionByOnly && CollectionUtils.isEmpty(_orderSetInfo.getOrderSet())) ? emptyOrderKey : AggregationUtils.extractRowKey(existingRow, _orderSetInfo.getOrderSet()); System.arraycopy(existingRow, 0, row, 0, existingRow.length); for (int i = 0; i < _windowAccumulators.length; i++) { - row[i + existingRow.length] = _windowAccumulators[i].getRangeResultForKeys(partitionKey, orderKey); + if (_windowAccumulators[i]._valueWindowFunction == null) { + row[i + existingRow.length] = _windowAccumulators[i].getRangeResultForKeys(partitionKey, orderKey); + } else { + row[i + existingRow.length] = _windowAccumulators[i].getValueResultForKeys(orderKey, rowId, rowList); + } } // Convert the results from Accumulator to the desired type TypeUtils.convertRow(row, resultStoredTypes); @@ -288,7 +299,9 @@ private TransferableBlock consumeInputBlocks() { : AggregationUtils.extractRowKey(row, _orderSetInfo.getOrderSet()); int aggCallsSize = _aggCalls.size(); for (int i = 0; i < aggCallsSize; i++) { - _windowAccumulators[i].accumulateRangeResults(key, orderKey, row); + if (_windowAccumulators[i]._valueWindowFunction == null) { + _windowAccumulators[i].accumulateRangeResults(key, orderKey, row); + } } } } else { @@ -430,11 +443,15 @@ public Long merge(Object left, Object right) { private static class WindowAggregateAccumulator extends AggregationUtils.Accumulator { private static final Map> WIN_AGG_MERGERS = ImmutableMap.>builder() - .putAll(AggregationUtils.Accumulator.MERGERS).put("ROW_NUMBER", cdt -> new MergeRowNumber()) - .put("RANK", cdt -> new MergeRank()).put("DENSE_RANK", cdt -> new MergeDenseRank()).build(); + .putAll(AggregationUtils.Accumulator.MERGERS) + .put("ROW_NUMBER", cdt -> new MergeRowNumber()) + .put("RANK", cdt -> new MergeRank()) + .put("DENSE_RANK", cdt -> new MergeDenseRank()) + .build(); private final boolean _isPartitionByOnly; private final boolean _isRankingWindowFunction; + private final ValueWindowFunction _valueWindowFunction; // Fields needed only for RANGE frame type queries (ORDER BY) private final Map _orderByResults = new HashMap<>(); @@ -445,6 +462,7 @@ private static class WindowAggregateAccumulator extends AggregationUtils.Accumul super(aggCall, merger, functionName, inputSchema); _isPartitionByOnly = CollectionUtils.isEmpty(orderSetInfo.getOrderSet()) || orderSetInfo.isPartitionByOnly(); _isRankingWindowFunction = RANKING_FUNCTION_NAMES.contains(functionName); + _valueWindowFunction = ValueWindowFunction.construnctValueWindowFunction(functionName); } /** @@ -514,6 +532,14 @@ public Map getRangeOrderByResults() { return _orderByResults; } + public Object getValueResultForKeys(Key orderKey, int rowId, List partitionRows) { + Object[] row = _valueWindowFunction.processRow(rowId, partitionRows); + if (row == null) { + return null; + } + return _inputRef == -1 ? _literal : row[_inputRef]; + } + static class OrderKeyResult { final Map _orderByResults; Key _previousOrderByKey; diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/AggregationUtils.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/AggregationUtils.java index ed9b0acba240..049da0522039 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/AggregationUtils.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/AggregationUtils.java @@ -223,7 +223,7 @@ public Accumulator(RexExpression.FunctionCall aggCall, _literal = ((RexExpression.Literal) rexExpression).getValue(); _dataType = rexExpression.getDataType(); } - _merger = merger.get(functionName).apply(_dataType); + _merger = merger.containsKey(functionName) ? merger.get(functionName).apply(_dataType) : null; } public void accumulate(Key key, Object[] row) { diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/FirstValueWindowFunction.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/FirstValueWindowFunction.java new file mode 100644 index 000000000000..5d2ae7595030 --- /dev/null +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/FirstValueWindowFunction.java @@ -0,0 +1,40 @@ +/** + * 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.query.runtime.operator.window; + +import java.util.ArrayList; +import java.util.List; + + +public class FirstValueWindowFunction extends ValueWindowFunction { + + @Override + public Object[] processRow(int rowId, List partitionedRows) { + return partitionedRows.get(0); + } + + @Override + public List processRows(List rows) { + List result = new ArrayList<>(); + for (int i = 0; i < rows.size(); i++) { + result.add(rows.get(0)); + } + return result; + } +} diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/LagValueWindowFunction.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/LagValueWindowFunction.java new file mode 100644 index 000000000000..9bca8ec93096 --- /dev/null +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/LagValueWindowFunction.java @@ -0,0 +1,48 @@ +/** + * 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.query.runtime.operator.window; + +import java.util.ArrayList; +import java.util.List; + + +public class LagValueWindowFunction extends ValueWindowFunction { + + @Override + public Object[] processRow(int rowId, List partitionedRows) { + if (rowId == 0) { + return null; + } else { + return partitionedRows.get(rowId - 1); + } + } + + @Override + public List processRows(List rows) { + List result = new ArrayList<>(); + for (int i = 0; i < rows.size(); i++) { + if (i == 0) { + result.add(null); + } else { + result.add(rows.get(i - 1)); + } + } + return result; + } +} diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/LastValueWindowFunction.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/LastValueWindowFunction.java new file mode 100644 index 000000000000..cc7db910d27f --- /dev/null +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/LastValueWindowFunction.java @@ -0,0 +1,40 @@ +/** + * 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.query.runtime.operator.window; + +import java.util.ArrayList; +import java.util.List; + + +public class LastValueWindowFunction extends ValueWindowFunction { + + @Override + public Object[] processRow(int rowId, List partitionedRows) { + return partitionedRows.get(partitionedRows.size() - 1); + } + + @Override + public List processRows(List rows) { + List result = new ArrayList<>(); + for (int i = 0; i < rows.size(); i++) { + result.add(rows.get(rows.size() - 1)); + } + return result; + } +} diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/LeadValueWindowFunction.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/LeadValueWindowFunction.java new file mode 100644 index 000000000000..bd8a50ea48f2 --- /dev/null +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/LeadValueWindowFunction.java @@ -0,0 +1,48 @@ +/** + * 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.query.runtime.operator.window; + +import java.util.ArrayList; +import java.util.List; + + +public class LeadValueWindowFunction extends ValueWindowFunction { + + @Override + public Object[] processRow(int rowId, List partitionedRows) { + if (rowId == partitionedRows.size() - 1) { + return null; + } else { + return partitionedRows.get(rowId + 1); + } + } + + @Override + public List processRows(List rows) { + List result = new ArrayList<>(); + for (int i = 0; i < rows.size(); i++) { + if (i == rows.size() - 1) { + result.add(null); + } else { + result.add(rows.get(i + 1)); + } + } + return result; + } +} diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/ValueWindowFunction.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/ValueWindowFunction.java new file mode 100644 index 000000000000..c327bcf0ba19 --- /dev/null +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/ValueWindowFunction.java @@ -0,0 +1,54 @@ +/** + * 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.query.runtime.operator.window; + +import com.google.common.collect.ImmutableMap; +import java.lang.reflect.InvocationTargetException; +import java.util.List; +import java.util.Map; + + +public abstract class ValueWindowFunction implements WindowFunction { + public static final Map> VALUE_WINDOW_FUNCTION_MAP = + ImmutableMap.>builder() + .put("LEAD", LeadValueWindowFunction.class) + .put("LAG", LagValueWindowFunction.class) + .put("FIRST_VALUE", FirstValueWindowFunction.class) + .put("LAST_VALUE", LastValueWindowFunction.class) + .build(); + + /** + * @param rowId Row id to process + * @param partitionedRows List of rows for reference + * @return Row with the window function applied + */ + public abstract Object[] processRow(int rowId, List partitionedRows); + + public static ValueWindowFunction construnctValueWindowFunction(String functionName) { + Class valueWindowFunctionClass = VALUE_WINDOW_FUNCTION_MAP.get(functionName); + if (valueWindowFunctionClass == null) { + return null; + } + try { + return valueWindowFunctionClass.getDeclaredConstructor().newInstance(); + } catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { + throw new RuntimeException("Failed to instantiate ValueWindowFunction for function: " + functionName, e); + } + } +} diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/WindowFunction.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/WindowFunction.java new file mode 100644 index 000000000000..56d893badfd2 --- /dev/null +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/window/WindowFunction.java @@ -0,0 +1,31 @@ +/** + * 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.query.runtime.operator.window; + +import java.util.List; + + +public interface WindowFunction { + + /** + * @param rows List of rows to process + * @return List of rows with the window function applied + */ + List processRows(List rows); +} From 6725ca8c9241b1d5d74950ac5c4b23225a05f413 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Thu, 11 Apr 2024 04:18:41 +0800 Subject: [PATCH 52/72] Bump net.minidev:json-smart from 2.5.0 to 2.5.1 (#12881) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a807169f97bb..9d1efc0832f6 100644 --- a/pom.xml +++ b/pom.xml @@ -147,7 +147,7 @@ 5.13.0 3.3.6 2.9.0 - 2.5.0 + 2.5.1 2.3.2 1.36.0 9.8.0 From 0da1d4e006324ff93c396ba391710711118444ab Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 10 Apr 2024 13:21:05 -0700 Subject: [PATCH 53/72] Bump com.puppycrawl.tools:checkstyle from 8.45.1 to 10.15.0 (#12784) --- .../controller/api/resources/PinotTenantRestletResource.java | 2 +- .../apache/pinot/integration/tests/custom/GeoSpatialTest.java | 2 +- pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTenantRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTenantRestletResource.java index 0d41216aa5f2..0b8725b10e71 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTenantRestletResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTenantRestletResource.java @@ -588,7 +588,7 @@ public String changeTenantState( } } - boolean enable = StateType.ENABLE.name().equalsIgnoreCase(state) ? true : false; + boolean enable = StateType.ENABLE.name().equalsIgnoreCase(state); ObjectNode instanceResult = JsonUtils.newObjectNode(); String instance = null; try { diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/custom/GeoSpatialTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/custom/GeoSpatialTest.java index 88ee0f400e38..ff41c76b1337 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/custom/GeoSpatialTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/custom/GeoSpatialTest.java @@ -398,7 +398,7 @@ public void testStWithinQuery(boolean useMultiStageQueryEngine) JsonNode rows = pinotResponse.get("resultTable").get("rows"); for (int i = 0; i < rows.size(); i++) { JsonNode row = rows.get(i); - boolean actualResult = row.get(0).intValue() == 1 ? true : false; + boolean actualResult = row.get(0).intValue() == 1; boolean expectedResult = row.get(1).booleanValue(); Assert.assertEquals(actualResult, expectedResult); } diff --git a/pom.xml b/pom.xml index 9d1efc0832f6..a17f74ea9552 100644 --- a/pom.xml +++ b/pom.xml @@ -1935,7 +1935,7 @@ com.puppycrawl.tools checkstyle - 8.45.1 + 10.15.0 From 549cb60cacc19928f549a23b4d6f0091ed0aed5c Mon Sep 17 00:00:00 2001 From: "Xiaotian (Jackie) Jiang" <17555551+Jackie-Jiang@users.noreply.github.com> Date: Wed, 10 Apr 2024 13:21:13 -0700 Subject: [PATCH 54/72] Bump org.scalatest:scalatest_2.12 from 3.1.1 to 3.2.18 (#12879) --- pinot-connectors/pinot-spark-2-connector/pom.xml | 2 +- pinot-connectors/pinot-spark-3-connector/pom.xml | 2 +- pinot-connectors/pinot-spark-common/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pinot-connectors/pinot-spark-2-connector/pom.xml b/pinot-connectors/pinot-spark-2-connector/pom.xml index 1463cff56fc7..14904aad2884 100644 --- a/pinot-connectors/pinot-spark-2-connector/pom.xml +++ b/pinot-connectors/pinot-spark-2-connector/pom.xml @@ -36,7 +36,7 @@ 2.4.8 2.8 1.3.0 - 3.1.1 + 3.2.18 org.apache.pinot.\$internal diff --git a/pinot-connectors/pinot-spark-3-connector/pom.xml b/pinot-connectors/pinot-spark-3-connector/pom.xml index 96efe295f345..a39548d22019 100644 --- a/pinot-connectors/pinot-spark-3-connector/pom.xml +++ b/pinot-connectors/pinot-spark-3-connector/pom.xml @@ -34,7 +34,7 @@ ${basedir}/../.. 3.5.1 - 3.1.1 + 3.2.18 org.apache.pinot.\$internal diff --git a/pinot-connectors/pinot-spark-common/pom.xml b/pinot-connectors/pinot-spark-common/pom.xml index e130a295b732..a4f72ed076e4 100644 --- a/pinot-connectors/pinot-spark-common/pom.xml +++ b/pinot-connectors/pinot-spark-common/pom.xml @@ -36,7 +36,7 @@ 0.14.6 2.8 1.3.0 - 3.1.1 + 3.2.18 From 5a909743e9e17af2c78c285dadaabdc897323ce6 Mon Sep 17 00:00:00 2001 From: "Xiaotian (Jackie) Jiang" <17555551+Jackie-Jiang@users.noreply.github.com> Date: Thu, 11 Apr 2024 11:44:49 -0700 Subject: [PATCH 55/72] Move some mispositioned tests under pinot-core (#12884) --- .../request/context/LiteralContextTest.java | 0 .../utils/request/RequestUtilsTest.java | 31 ++++++++++++---- .../utils/request/RequestUtilsTest.java | 37 ------------------- .../IntDoubleIndexedPriorityQueueTest.java | 3 +- .../IntObjectIndexedPriorityQueueTest.java | 3 +- .../util/QueryMultiThreadingUtilsTest.java | 3 +- .../RealtimeSegmentConverterTest.java | 4 +- .../spi/crypt/PinotCrypterFactoryTest.java | 0 8 files changed, 27 insertions(+), 54 deletions(-) rename {pinot-core => pinot-common}/src/test/java/org/apache/pinot/common/request/context/LiteralContextTest.java (100%) delete mode 100644 pinot-core/src/test/java/org/apache/pinot/common/utils/request/RequestUtilsTest.java rename pinot-core/src/test/java/org/apache/pinot/{ => core}/util/IntDoubleIndexedPriorityQueueTest.java (97%) rename pinot-core/src/test/java/org/apache/pinot/{ => core}/util/IntObjectIndexedPriorityQueueTest.java (97%) rename pinot-core/src/test/java/org/apache/pinot/{ => core}/util/QueryMultiThreadingUtilsTest.java (97%) rename {pinot-core/src/test/java/org/apache/pinot => pinot-segment-local/src/test/java/org/apache/pinot/segment/local}/realtime/converter/RealtimeSegmentConverterTest.java (99%) rename {pinot-core => pinot-spi}/src/test/java/org/apache/pinot/spi/crypt/PinotCrypterFactoryTest.java (100%) diff --git a/pinot-core/src/test/java/org/apache/pinot/common/request/context/LiteralContextTest.java b/pinot-common/src/test/java/org/apache/pinot/common/request/context/LiteralContextTest.java similarity index 100% rename from pinot-core/src/test/java/org/apache/pinot/common/request/context/LiteralContextTest.java rename to pinot-common/src/test/java/org/apache/pinot/common/request/context/LiteralContextTest.java diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/request/RequestUtilsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/request/RequestUtilsTest.java index 7b32c46c9806..13f5b762b481 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/utils/request/RequestUtilsTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/request/RequestUtilsTest.java @@ -19,35 +19,50 @@ package org.apache.pinot.common.utils.request; import org.apache.calcite.sql.SqlDialect; +import org.apache.calcite.sql.SqlLiteral; +import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.pinot.common.request.Expression; +import org.apache.pinot.common.request.ExpressionType; import org.apache.pinot.sql.parsers.PinotSqlType; import org.apache.pinot.sql.parsers.SqlNodeAndOptions; -import org.testng.Assert; import org.testng.annotations.Test; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; + public class RequestUtilsTest { + + @Test + public void testNullLiteralParsing() { + SqlLiteral nullLiteral = SqlLiteral.createNull(SqlParserPos.ZERO); + Expression nullExpr = RequestUtils.getLiteralExpression(nullLiteral); + assertEquals(nullExpr.getType(), ExpressionType.LITERAL); + assertTrue(nullExpr.getLiteral().getNullValue()); + } + // please check comments inside RequestUtils.getLiteralExpression() for why we need this test @Test public void testGetLiteralExpressionForObject() { Expression literalExpression = RequestUtils.getLiteralExpression(Float.valueOf(0.06f)); - Assert.assertEquals((literalExpression.getLiteral().getDoubleValue()), 0.06); + assertEquals((literalExpression.getLiteral().getDoubleValue()), 0.06); } @Test public void testGetLiteralExpressionForPrimitiveLong() { Expression literalExpression = RequestUtils.getLiteralExpression(4500L); - Assert.assertTrue(literalExpression.getLiteral().isSetLongValue()); - Assert.assertFalse(literalExpression.getLiteral().isSetDoubleValue()); - Assert.assertEquals(literalExpression.getLiteral().getLongValue(), 4500L); + assertTrue(literalExpression.getLiteral().isSetLongValue()); + assertFalse(literalExpression.getLiteral().isSetDoubleValue()); + assertEquals(literalExpression.getLiteral().getLongValue(), 4500L); } @Test public void testParseQuery() { SqlNodeAndOptions result = RequestUtils.parseQuery("select foo from countries where bar > 1"); - Assert.assertTrue(result.getParseTimeNs() > 0); - Assert.assertEquals(result.getSqlType(), PinotSqlType.DQL); - Assert.assertEquals(result.getSqlNode().toSqlString((SqlDialect) null).toString(), + assertTrue(result.getParseTimeNs() > 0); + assertEquals(result.getSqlType(), PinotSqlType.DQL); + assertEquals(result.getSqlNode().toSqlString((SqlDialect) null).toString(), "SELECT `foo`\n" + "FROM `countries`\n" + "WHERE `bar` > 1"); } } diff --git a/pinot-core/src/test/java/org/apache/pinot/common/utils/request/RequestUtilsTest.java b/pinot-core/src/test/java/org/apache/pinot/common/utils/request/RequestUtilsTest.java deleted file mode 100644 index 3c36a0de7fe4..000000000000 --- a/pinot-core/src/test/java/org/apache/pinot/common/utils/request/RequestUtilsTest.java +++ /dev/null @@ -1,37 +0,0 @@ -/** - * 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.common.utils.request; - -import org.apache.calcite.sql.SqlLiteral; -import org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.pinot.common.request.Expression; -import org.apache.pinot.common.request.ExpressionType; -import org.testng.Assert; -import org.testng.annotations.Test; - - -public class RequestUtilsTest { - @Test - public void testNullLiteralParsing() { - SqlLiteral nullLiteral = SqlLiteral.createNull(SqlParserPos.ZERO); - Expression nullExpr = RequestUtils.getLiteralExpression(nullLiteral); - Assert.assertEquals(nullExpr.getType(), ExpressionType.LITERAL); - Assert.assertEquals(nullExpr.getLiteral().getNullValue(), true); - } -} diff --git a/pinot-core/src/test/java/org/apache/pinot/util/IntDoubleIndexedPriorityQueueTest.java b/pinot-core/src/test/java/org/apache/pinot/core/util/IntDoubleIndexedPriorityQueueTest.java similarity index 97% rename from pinot-core/src/test/java/org/apache/pinot/util/IntDoubleIndexedPriorityQueueTest.java rename to pinot-core/src/test/java/org/apache/pinot/core/util/IntDoubleIndexedPriorityQueueTest.java index 7e9ca3e78e36..642f44464bae 100644 --- a/pinot-core/src/test/java/org/apache/pinot/util/IntDoubleIndexedPriorityQueueTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/util/IntDoubleIndexedPriorityQueueTest.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pinot.util; +package org.apache.pinot.core.util; import it.unimi.dsi.fastutil.ints.Int2DoubleMap; import it.unimi.dsi.fastutil.ints.Int2DoubleOpenHashMap; @@ -24,7 +24,6 @@ import java.util.Collections; import java.util.List; import java.util.Random; -import org.apache.pinot.core.util.IntDoubleIndexedPriorityQueue; import org.apache.pinot.spi.utils.Pairs; import org.testng.Assert; import org.testng.annotations.Test; diff --git a/pinot-core/src/test/java/org/apache/pinot/util/IntObjectIndexedPriorityQueueTest.java b/pinot-core/src/test/java/org/apache/pinot/core/util/IntObjectIndexedPriorityQueueTest.java similarity index 97% rename from pinot-core/src/test/java/org/apache/pinot/util/IntObjectIndexedPriorityQueueTest.java rename to pinot-core/src/test/java/org/apache/pinot/core/util/IntObjectIndexedPriorityQueueTest.java index e67e8d18cf0e..aeb51ff7c108 100644 --- a/pinot-core/src/test/java/org/apache/pinot/util/IntObjectIndexedPriorityQueueTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/util/IntObjectIndexedPriorityQueueTest.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pinot.util; +package org.apache.pinot.core.util; import java.util.ArrayList; import java.util.Collections; @@ -24,7 +24,6 @@ import java.util.List; import java.util.Map; import java.util.Random; -import org.apache.pinot.core.util.IntObjectIndexedPriorityQueue; import org.apache.pinot.segment.local.customobject.AvgPair; import org.apache.pinot.spi.utils.Pairs; import org.testng.Assert; diff --git a/pinot-core/src/test/java/org/apache/pinot/util/QueryMultiThreadingUtilsTest.java b/pinot-core/src/test/java/org/apache/pinot/core/util/QueryMultiThreadingUtilsTest.java similarity index 97% rename from pinot-core/src/test/java/org/apache/pinot/util/QueryMultiThreadingUtilsTest.java rename to pinot-core/src/test/java/org/apache/pinot/core/util/QueryMultiThreadingUtilsTest.java index 92ac06f9677b..0c5c6ca555da 100644 --- a/pinot-core/src/test/java/org/apache/pinot/util/QueryMultiThreadingUtilsTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/util/QueryMultiThreadingUtilsTest.java @@ -16,13 +16,12 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pinot.util; +package org.apache.pinot.core.util; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.pinot.core.util.QueryMultiThreadingUtils; import org.testng.Assert; import org.testng.annotations.Test; diff --git a/pinot-core/src/test/java/org/apache/pinot/realtime/converter/RealtimeSegmentConverterTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverterTest.java similarity index 99% rename from pinot-core/src/test/java/org/apache/pinot/realtime/converter/RealtimeSegmentConverterTest.java rename to pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverterTest.java index 50f1396c1026..ded9e85b692c 100644 --- a/pinot-core/src/test/java/org/apache/pinot/realtime/converter/RealtimeSegmentConverterTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverterTest.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pinot.realtime.converter; +package org.apache.pinot.segment.local.realtime.converter; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; @@ -34,8 +34,6 @@ import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentImpl; import org.apache.pinot.segment.local.indexsegment.mutable.MutableSegmentImpl; import org.apache.pinot.segment.local.io.writer.impl.DirectMemoryManager; -import org.apache.pinot.segment.local.realtime.converter.ColumnIndicesForRealtimeTable; -import org.apache.pinot.segment.local.realtime.converter.RealtimeSegmentConverter; import org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentConfig; import org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentStatsHistory; import org.apache.pinot.segment.local.segment.index.column.PhysicalColumnIndexContainer; diff --git a/pinot-core/src/test/java/org/apache/pinot/spi/crypt/PinotCrypterFactoryTest.java b/pinot-spi/src/test/java/org/apache/pinot/spi/crypt/PinotCrypterFactoryTest.java similarity index 100% rename from pinot-core/src/test/java/org/apache/pinot/spi/crypt/PinotCrypterFactoryTest.java rename to pinot-spi/src/test/java/org/apache/pinot/spi/crypt/PinotCrypterFactoryTest.java From ed0ecdd2e4a4e803113a57ae4bda0d5a9f6ecd77 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 11 Apr 2024 11:46:30 -0700 Subject: [PATCH 56/72] Bump io.projectreactor:reactor-core from 3.6.4 to 3.6.5 (#12897) --- pinot-plugins/pinot-file-system/pinot-adls/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml index 543b60078928..3ad8428473d1 100644 --- a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml +++ b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml @@ -62,7 +62,7 @@ io.projectreactor reactor-core - 3.6.4 + 3.6.5 net.java.dev.jna From 6ca21baf4d004accc69f082ed57051a444bc6266 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 11 Apr 2024 11:46:51 -0700 Subject: [PATCH 57/72] Bump com.azure:azure-core from 1.45.1 to 1.48.0 (#12896) --- pinot-plugins/pinot-file-system/pinot-adls/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml index 3ad8428473d1..30d7cbc051be 100644 --- a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml +++ b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml @@ -57,7 +57,7 @@ com.azure azure-core - 1.45.1 + 1.48.0 io.projectreactor From 2063690b11279ed743ac8039f93086471df5ee8d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 11 Apr 2024 11:47:09 -0700 Subject: [PATCH 58/72] Bump aws.sdk.version from 2.25.28 to 2.25.29 (#12894) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a17f74ea9552..832d2388b73f 100644 --- a/pom.xml +++ b/pom.xml @@ -167,7 +167,7 @@ 0.15.0 0.4.4 4.2.2 - 2.25.28 + 2.25.29 2.12.7 3.1.12 7.10.1 From be2b3e59705a41d631251522514507dd19c3c491 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 11 Apr 2024 11:47:26 -0700 Subject: [PATCH 59/72] Bump com.github.jnr:jnr-constants from 0.10.3 to 0.10.4 (#12893) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 832d2388b73f..0d9484fa2407 100644 --- a/pom.xml +++ b/pom.xml @@ -1363,7 +1363,7 @@ com.github.jnr jnr-constants - 0.10.3 + 0.10.4 info.picocli From f5d629c10e19a800c34e9118665a586c38a9c123 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 11 Apr 2024 11:48:56 -0700 Subject: [PATCH 60/72] Bump org.apache.spark:spark-core_2.12 from 3.5.0 to 3.5.1 (#12903) --- .../pinot-batch-ingestion/pinot-batch-ingestion-spark-3/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark-3/pom.xml b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark-3/pom.xml index fd36cd868ce9..1dfba13c999b 100644 --- a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark-3/pom.xml +++ b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark-3/pom.xml @@ -36,7 +36,7 @@ ${basedir}/../../.. package - 3.5.0 + 3.5.1 From 6ab2c89a6bbec0379464dff87f3f3df6caa8f5b5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 11 Apr 2024 12:14:43 -0700 Subject: [PATCH 61/72] Bump org.apache.httpcomponents:httpcore from 4.4.13 to 4.4.16 (#12902) --- pinot-plugins/pinot-file-system/pinot-s3/pom.xml | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pinot-plugins/pinot-file-system/pinot-s3/pom.xml b/pinot-plugins/pinot-file-system/pinot-s3/pom.xml index b0322f92b077..5427cf5cbc07 100644 --- a/pinot-plugins/pinot-file-system/pinot-s3/pom.xml +++ b/pinot-plugins/pinot-file-system/pinot-s3/pom.xml @@ -36,7 +36,7 @@ ${basedir}/../../.. 4.5.14 - 4.4.13 + 4.4.16 2.12.2 package diff --git a/pom.xml b/pom.xml index 0d9484fa2407..0556e598a170 100644 --- a/pom.xml +++ b/pom.xml @@ -481,7 +481,7 @@ org.apache.httpcomponents httpcore - 4.4.13 + 4.4.16 From 8ee2663c2010eef3a9777068e93dd8ee15180b78 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 11 Apr 2024 12:18:40 -0700 Subject: [PATCH 62/72] Bump com.github.seancfoley:ipaddress from 5.3.4 to 5.5.0 (#12901) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0556e598a170..75984c5591e9 100644 --- a/pom.xml +++ b/pom.xml @@ -1400,7 +1400,7 @@ com.github.seancfoley ipaddress - 5.3.4 + 5.5.0 From 8d6a892a2699ba1f7d4c808cc6c393d397498ceb Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 11 Apr 2024 12:25:21 -0700 Subject: [PATCH 63/72] Bump io.grpc:grpc-protobuf-lite from 1.62.2 to 1.63.0 (#12900) --- pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml index 99af7c084abc..2037493483d2 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml @@ -39,7 +39,7 @@ ${basedir}/../../.. 0.16.0 1.63.0 - 1.62.2 + 1.63.0 2.6.2 1.17 From 183144f07f72d8fa9eb6e2196328a981a7ce04ec Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi <45308220+deepthi912@users.noreply.github.com> Date: Thu, 11 Apr 2024 15:46:48 -0400 Subject: [PATCH 64/72] Upgrade reflections from 0.9.11 to 0.10.2 (#12885) --- .../org/apache/pinot/spi/utils/PinotReflectionUtils.java | 6 +++--- pom.xml | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/PinotReflectionUtils.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/PinotReflectionUtils.java index bcc55cd530a9..582f4ae8ab62 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/PinotReflectionUtils.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/PinotReflectionUtils.java @@ -53,7 +53,7 @@ public static Set> getClassesThroughReflection(String packageName, Stri try { synchronized (REFLECTION_LOCK) { return new Reflections(new ConfigurationBuilder().setUrls(ClasspathHelper.forPackage(packageName)) - .filterInputsBy(new FilterBuilder.Include(regexPattern))).getTypesAnnotatedWith(annotation); + .filterInputsBy(new FilterBuilder().includePattern(regexPattern))).getTypesAnnotatedWith(annotation); } } catch (Throwable t) { // Log an error then re-throw it because this method is usually called in a static block, where exception might @@ -73,7 +73,7 @@ public static Set> getClassesThroughReflection(List packages, S urls.addAll(ClasspathHelper.forPackage(packageName)); } return new Reflections(new ConfigurationBuilder().setUrls(urls) - .filterInputsBy(new FilterBuilder.Include(regexPattern))).getTypesAnnotatedWith(annotation); + .filterInputsBy(new FilterBuilder().includePattern(regexPattern))).getTypesAnnotatedWith(annotation); } } catch (Throwable t) { // Log an error then re-throw it because this method is usually called in a static block, where exception might @@ -93,7 +93,7 @@ public static Set getMethodsThroughReflection(String packageName, String try { synchronized (REFLECTION_LOCK) { return new Reflections(new ConfigurationBuilder().setUrls(ClasspathHelper.forPackage(packageName)) - .filterInputsBy(new FilterBuilder.Include(regexPattern)) + .filterInputsBy(new FilterBuilder().includePattern(regexPattern)) .setScanners(new MethodAnnotationsScanner())).getMethodsAnnotatedWith(annotation); } } catch (Throwable t) { diff --git a/pom.xml b/pom.xml index 75984c5591e9..f2fdb36efd43 100644 --- a/pom.xml +++ b/pom.xml @@ -151,7 +151,7 @@ 2.3.2 1.36.0 9.8.0 - 0.9.11 + 0.10.2 4.2.25 1.1.10.5 From 1c409c2e8cf551397ce7b0f6de01da817a686413 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 11 Apr 2024 13:04:10 -0700 Subject: [PATCH 65/72] Bump org.apache.rat:apache-rat-plugin from 0.13 to 0.16.1 (#12895) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index f2fdb36efd43..90f9f37e5141 100644 --- a/pom.xml +++ b/pom.xml @@ -2161,7 +2161,7 @@ org.apache.rat apache-rat-plugin - 0.13 + 0.16.1 verify From 7b2005dfa8445b4506d76b3b9edda4c81058b255 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 11 Apr 2024 13:08:41 -0700 Subject: [PATCH 66/72] Bump confluent.version from 7.6.0 to 7.6.1 (#12891) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 90f9f37e5141..b2215d83c752 100644 --- a/pom.xml +++ b/pom.xml @@ -220,7 +220,7 @@ 3.25.2 1.61.1 - 7.6.0 + 7.6.1 2.12.18 From ec56ba601ee2c3d8a0bc571b8efde2809bff438b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 11 Apr 2024 13:09:01 -0700 Subject: [PATCH 67/72] Bump com.gradle:gradle-enterprise-maven-extension from 1.19.2 to 1.20.1 (#12889) --- .mvn/extensions.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.mvn/extensions.xml b/.mvn/extensions.xml index 8742ba0778ac..177fc5d6f1e8 100644 --- a/.mvn/extensions.xml +++ b/.mvn/extensions.xml @@ -23,7 +23,7 @@ com.gradle gradle-enterprise-maven-extension - 1.19.2 + 1.20.1 com.gradle From 7401798d1e33a74043f806621e2764cca50e96da Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 11 Apr 2024 13:09:24 -0700 Subject: [PATCH 68/72] Bump commons-io:commons-io from 2.15.1 to 2.16.1 (#12890) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index b2215d83c752..bf3c246c418f 100644 --- a/pom.xml +++ b/pom.xml @@ -182,7 +182,7 @@ 3.6.1 1.10.0 2.10.1 - 2.15.1 + 2.16.1 1.16.1 1.6.0 3.10.0 From ce2d0dbac107b9f102ed6140f261630b78c2d5b7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 11 Apr 2024 14:18:31 -0700 Subject: [PATCH 69/72] Bump com.github.jnr:jnr-posix from 3.1.15 to 3.1.19 (#12892) --- pom.xml | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index bf3c246c418f..3f5903d1b554 100644 --- a/pom.xml +++ b/pom.xml @@ -1358,7 +1358,12 @@ com.github.jnr jnr-posix - 3.1.15 + 3.1.19 + + + com.github.jnr + jnr-ffi + 2.2.16 com.github.jnr From 1f17c465f2e31c31d87a1127c6fac08c83b95528 Mon Sep 17 00:00:00 2001 From: "Xiaotian (Jackie) Jiang" <17555551+Jackie-Jiang@users.noreply.github.com> Date: Thu, 11 Apr 2024 14:56:21 -0700 Subject: [PATCH 70/72] Cleanup HTTP components dependencies and upgrade Thrift (#12905) --- .../pinot-input-format/pinot-thrift/pom.xml | 4 ++ .../thrift/ThriftRecordExtractor.java | 3 +- pom.xml | 63 ++++++++++++------- 3 files changed, 47 insertions(+), 23 deletions(-) diff --git a/pinot-plugins/pinot-input-format/pinot-thrift/pom.xml b/pinot-plugins/pinot-input-format/pinot-thrift/pom.xml index bb5ed9c94de4..ab2b565f1bc0 100644 --- a/pinot-plugins/pinot-input-format/pinot-thrift/pom.xml +++ b/pinot-plugins/pinot-input-format/pinot-thrift/pom.xml @@ -41,5 +41,9 @@ org.apache.thrift libthrift + + org.apache.commons + commons-lang3 + diff --git a/pinot-plugins/pinot-input-format/pinot-thrift/src/main/java/org/apache/pinot/plugin/inputformat/thrift/ThriftRecordExtractor.java b/pinot-plugins/pinot-input-format/pinot-thrift/src/main/java/org/apache/pinot/plugin/inputformat/thrift/ThriftRecordExtractor.java index d10557d9b8b9..4be6374c85a5 100644 --- a/pinot-plugins/pinot-input-format/pinot-thrift/src/main/java/org/apache/pinot/plugin/inputformat/thrift/ThriftRecordExtractor.java +++ b/pinot-plugins/pinot-input-format/pinot-thrift/src/main/java/org/apache/pinot/plugin/inputformat/thrift/ThriftRecordExtractor.java @@ -97,7 +97,8 @@ protected boolean isRecord(Object value) { protected Object convertRecord(Object value) { TBase record = (TBase) value; Map convertedRecord = new HashMap<>(); - for (TFieldIdEnum tFieldIdEnum : FieldMetaData.getStructMetaDataMap(record.getClass()).keySet()) { + Set tFieldIdEnums = FieldMetaData.getStructMetaDataMap(record.getClass()).keySet(); + for (TFieldIdEnum tFieldIdEnum : tFieldIdEnums) { Object fieldValue = record.getFieldValue(tFieldIdEnum); if (fieldValue != null) { fieldValue = convert(fieldValue); diff --git a/pom.xml b/pom.xml index 3f5903d1b554..255f13e13911 100644 --- a/pom.xml +++ b/pom.xml @@ -157,6 +157,7 @@ 1.1.10.5 1.5.6-2 1.8.0 + 0.20.0 2.23.1 2.0.12 4.1.108.Final @@ -211,6 +212,12 @@ 1.1.1 2.2 + + 4.5.14 + 4.4.16 + 5.3.1 + 5.2.4 + 26.37.0 1.23.0 @@ -462,27 +469,6 @@ - - org.apache.httpcomponents - httpmime - 4.5.13 - - - org.apache.httpcomponents - httpclient - 4.5.14 - - - commons-logging - commons-logging - - - - - org.apache.httpcomponents - httpcore - 4.4.16 - @@ -556,7 +542,7 @@ org.apache.thrift libthrift - 0.15.0 + ${libthrift.verion} org.quartz-scheduler @@ -847,6 +833,39 @@ ${javax.jsp-api.version} + + + org.apache.httpcomponents + httpmime + ${httpclient.version} + + + org.apache.httpcomponents + httpclient + ${httpclient.version} + + + commons-logging + commons-logging + + + + + org.apache.httpcomponents + httpcore + ${httpcore.version} + + + org.apache.httpcomponents.client5 + httpclient5 + ${httpclient5.version} + + + org.apache.httpcomponents.core5 + httpcore5 + ${httpcore5.version} + + com.google.cloud From bc68041592dc278b6f0cb71f4e0f1c7454a9f9a8 Mon Sep 17 00:00:00 2001 From: "Xiaotian (Jackie) Jiang" <17555551+Jackie-Jiang@users.noreply.github.com> Date: Thu, 11 Apr 2024 14:56:35 -0700 Subject: [PATCH 71/72] Cleanup unnecessary dependencies under pinot-s3 (#12904) --- .../pinot-file-system/pinot-s3/pom.xml | 79 +------------------ 1 file changed, 3 insertions(+), 76 deletions(-) diff --git a/pinot-plugins/pinot-file-system/pinot-s3/pom.xml b/pinot-plugins/pinot-file-system/pinot-s3/pom.xml index 5427cf5cbc07..4eec102625a2 100644 --- a/pinot-plugins/pinot-file-system/pinot-s3/pom.xml +++ b/pinot-plugins/pinot-file-system/pinot-s3/pom.xml @@ -35,8 +35,6 @@ https://pinot.apache.org ${basedir}/../../.. - 4.5.14 - 4.4.16 2.12.2 package @@ -46,99 +44,28 @@ org.apache.pinot pinot-spi - + software.amazon.awssdk s3 - - - org.apache.logging.log4j - log4j-to-slf4j - - - org.apache.httpcomponents - httpcore - - - org.apache.httpcomponents - httpclient - - - software.amazon.awssdk sts - - - org.apache.logging.log4j - log4j-to-slf4j - - - org.apache.httpcomponents - httpcore - - - org.apache.httpcomponents - httpclient - - software.amazon.awssdk apache-client - - org.apache.httpcomponents - httpclient - ${http.client.version} - - - org.apache.httpcomponents - httpcore - - - org.apache.logging.log4j - log4j-to-slf4j - - - - - org.apache.httpcomponents - httpcore - ${http.core.version} - - - org.reactivestreams - reactive-streams - - - io.netty - netty-codec - - - io.netty - netty-buffer - - - io.netty - netty-transport - - - io.netty - netty-common - + com.adobe.testing s3mock-testcontainers ${s3mock.version} test - - com.fasterxml.woodstox - woodstox-core - + build-shaded-jar From eacbf12ee2d7525a17eb64e0f64ab28520fd4c72 Mon Sep 17 00:00:00 2001 From: "Xiaotian (Jackie) Jiang" <17555551+Jackie-Jiang@users.noreply.github.com> Date: Thu, 11 Apr 2024 14:56:54 -0700 Subject: [PATCH 72/72] Enhance Pulsar consumer (#12812) --- .../plugin/stream/pulsar/PulsarConfig.java | 71 ++--- ...PulsarPartitionLevelConnectionHandler.java | 14 +- .../pulsar/PulsarPartitionLevelConsumer.java | 96 +++---- .../plugin/stream/pulsar/PulsarUtils.java | 10 - .../stream/pulsar/PulsarConsumerTest.java | 264 ++++++------------ 5 files changed, 145 insertions(+), 310 deletions(-) diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfig.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfig.java index 637899d649a6..8011e2c14843 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfig.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarConfig.java @@ -30,10 +30,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; -import org.apache.pinot.spi.stream.OffsetCriteria; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.stream.StreamConfigProperties; -import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.SubscriptionInitialPosition; @@ -46,17 +44,15 @@ public class PulsarConfig { public static final String BOOTSTRAP_SERVERS = "bootstrap.servers"; public static final String AUTHENTICATION_TOKEN = "authenticationToken"; public static final String TLS_TRUST_CERTS_FILE_PATH = "tlsTrustCertsFilePath"; - public static final String OAUTH_ISSUER_URL = "issuerUrl"; public static final String OAUTH_CREDS_FILE_PATH = "credsFilePath"; public static final String OAUTH_AUDIENCE = "audience"; public static final String ENABLE_KEY_VALUE_STITCH = "enableKeyValueStitch"; public static final String METADATA_FIELDS = "metadata.fields"; //list of the metadata fields comma separated - private final String _pulsarTopicName; private final String _subscriberId; + private final String _pulsarTopicName; private final String _bootstrapServers; - private final MessageId _initialMessageId; private final SubscriptionInitialPosition _subscriptionInitialPosition; private final String _authenticationToken; private final String _tlsTrustCertsFilePath; @@ -74,6 +70,7 @@ public class PulsarConfig { private final boolean _enableKeyValueStitch; private final boolean _populateMetadata; private final Set _metadataFields; + public PulsarConfig(StreamConfig streamConfig, String subscriberId) { Map streamConfigMap = streamConfig.getStreamConfigsMap(); _subscriberId = subscriberId; @@ -82,28 +79,24 @@ public PulsarConfig(StreamConfig streamConfig, String subscriberId) { _bootstrapServers = getConfigValue(streamConfigMap, BOOTSTRAP_SERVERS); Preconditions.checkNotNull(_bootstrapServers, "No brokers provided in the config"); + _subscriptionInitialPosition = PulsarUtils.offsetCriteriaToSubscription(streamConfig.getOffsetCriteria()); _authenticationToken = getConfigValue(streamConfigMap, AUTHENTICATION_TOKEN); _tlsTrustCertsFilePath = getConfigValue(streamConfigMap, TLS_TRUST_CERTS_FILE_PATH); - _enableKeyValueStitch = Boolean.parseBoolean(getConfigValue(streamConfigMap, ENABLE_KEY_VALUE_STITCH)); - - OffsetCriteria offsetCriteria = streamConfig.getOffsetCriteria(); + _issuerUrl = getConfigValue(streamConfigMap, OAUTH_ISSUER_URL); + _credentialsFilePath = getConfigValue(streamConfigMap, OAUTH_CREDS_FILE_PATH); + if (StringUtils.isNotBlank(_credentialsFilePath)) { + validateOAuthCredFile(); + } + _audience = getConfigValue(streamConfigMap, OAUTH_AUDIENCE); - _subscriptionInitialPosition = PulsarUtils.offsetCriteriaToSubscription(offsetCriteria); - _initialMessageId = PulsarUtils.offsetCriteriaToMessageId(offsetCriteria); - _populateMetadata = Boolean.parseBoolean(getConfigValueOrDefault(streamConfigMap, - StreamConfigProperties.METADATA_POPULATE, "false")); + _enableKeyValueStitch = Boolean.parseBoolean(getConfigValue(streamConfigMap, ENABLE_KEY_VALUE_STITCH)); + _populateMetadata = Boolean.parseBoolean(getConfigValue(streamConfigMap, StreamConfigProperties.METADATA_POPULATE)); String metadataFieldsToExtractCSV = getConfigValueOrDefault(streamConfigMap, METADATA_FIELDS, ""); if (StringUtils.isBlank(metadataFieldsToExtractCSV) || !_populateMetadata) { _metadataFields = Collections.emptySet(); } else { _metadataFields = parseConfigStringToEnumSet(metadataFieldsToExtractCSV); } - _issuerUrl = getConfigValue(streamConfigMap, OAUTH_ISSUER_URL); - _credentialsFilePath = getConfigValue(streamConfigMap, OAUTH_CREDS_FILE_PATH); - if (StringUtils.isNotBlank(_credentialsFilePath)) { - validateOAuthCredFile(); - } - _audience = getConfigValue(streamConfigMap, OAUTH_AUDIENCE); } @VisibleForTesting @@ -111,13 +104,13 @@ private void validateOAuthCredFile() { try { URL credFilePathUrl = new URL(_credentialsFilePath); if (!"file".equals(credFilePathUrl.getProtocol())) { - throw new IllegalArgumentException("Invalid credentials file path: " + _credentialsFilePath - + ". URL protocol must be file://"); + throw new IllegalArgumentException( + "Invalid credentials file path: " + _credentialsFilePath + ". URL protocol must be file://"); } File credFile = new File(credFilePathUrl.getPath()); if (!credFile.exists()) { - throw new IllegalArgumentException("Invalid credentials file path: " + _credentialsFilePath - + ". File does not exist."); + throw new IllegalArgumentException( + "Invalid credentials file path: " + _credentialsFilePath + ". File does not exist."); } } catch (MalformedURLException mue) { throw new IllegalArgumentException("Invalid credentials file path: " + _credentialsFilePath, mue); @@ -136,16 +129,14 @@ private Set parseConfigS String listOfMetadataFields) { try { String[] metadataFieldsArr = listOfMetadataFields.split(","); - return Stream.of(metadataFieldsArr) - .map(String::trim) - .filter(StringUtils::isNotBlank) - .map(PulsarStreamMessageMetadata.PulsarMessageMetadataValue::findByKey) - .filter(Objects::nonNull) + return Stream.of(metadataFieldsArr).map(String::trim).filter(StringUtils::isNotBlank) + .map(PulsarStreamMessageMetadata.PulsarMessageMetadataValue::findByKey).filter(Objects::nonNull) .collect(Collectors.toSet()); } catch (Exception e) { throw new IllegalArgumentException("Invalid metadata fields list: " + listOfMetadataFields, e); } } + public String getPulsarTopicName() { return _pulsarTopicName; } @@ -158,10 +149,6 @@ public String getBootstrapServers() { return _bootstrapServers; } - public MessageId getInitialMessageId() { - return _initialMessageId; - } - public SubscriptionInitialPosition getInitialSubscriberPosition() { return _subscriptionInitialPosition; } @@ -173,16 +160,6 @@ public String getAuthenticationToken() { public String getTlsTrustCertsFilePath() { return _tlsTrustCertsFilePath; } - public boolean getEnableKeyValueStitch() { - return _enableKeyValueStitch; - } - public boolean isPopulateMetadata() { - return _populateMetadata; - } - - public Set getMetadataFields() { - return _metadataFields; - } public String getIssuerUrl() { return _issuerUrl; @@ -195,4 +172,16 @@ public String getCredentialsFilePath() { public String getAudience() { return _audience; } + + public boolean getEnableKeyValueStitch() { + return _enableKeyValueStitch; + } + + public boolean isPopulateMetadata() { + return _populateMetadata; + } + + public Set getMetadataFields() { + return _metadataFields; + } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConnectionHandler.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConnectionHandler.java index 444d05514534..673ea0c0e82c 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConnectionHandler.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConnectionHandler.java @@ -18,9 +18,9 @@ */ package org.apache.pinot.plugin.stream.pulsar; +import java.io.Closeable; import java.io.IOException; import java.net.URL; -import java.util.List; import org.apache.commons.lang3.StringUtils; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pulsar.client.api.AuthenticationFactory; @@ -33,7 +33,7 @@ /** * Manages the Pulsar client connection, given the partition id and {@link PulsarConfig} */ -public class PulsarPartitionLevelConnectionHandler { +public class PulsarPartitionLevelConnectionHandler implements Closeable { protected final PulsarConfig _config; protected final String _clientId; protected final PulsarClient _pulsarClient; @@ -41,7 +41,7 @@ public class PulsarPartitionLevelConnectionHandler { /** * Creates a new instance of {@link PulsarClient} and {@link Reader} */ - public PulsarPartitionLevelConnectionHandler(String clientId, StreamConfig streamConfig) { + protected PulsarPartitionLevelConnectionHandler(String clientId, StreamConfig streamConfig) { _config = new PulsarConfig(streamConfig, clientId); _clientId = clientId; try { @@ -70,13 +70,7 @@ public PulsarPartitionLevelConnectionHandler(String clientId, StreamConfig strea } } - protected Reader createReaderForPartition(int partitionId) - throws Exception { - List partitions = _pulsarClient.getPartitionsForTopic(_config.getPulsarTopicName()).get(); - return _pulsarClient.newReader().topic(partitions.get(partitionId)).startMessageId(_config.getInitialMessageId()) - .startMessageIdInclusive().create(); - } - + @Override public void close() throws IOException { _pulsarClient.close(); diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConsumer.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConsumer.java index 36a27d7e96ac..c206574bc924 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConsumer.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarPartitionLevelConsumer.java @@ -21,17 +21,12 @@ 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 java.util.Objects; import org.apache.pinot.spi.stream.BytesStreamMessage; import org.apache.pinot.spi.stream.PartitionGroupConsumer; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.stream.StreamMessageMetadata; 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.apache.pulsar.client.api.Reader; @@ -45,74 +40,60 @@ public class PulsarPartitionLevelConsumer extends PulsarPartitionLevelConnectionHandler implements PartitionGroupConsumer { private static final Logger LOGGER = LoggerFactory.getLogger(PulsarPartitionLevelConsumer.class); + private final Reader _reader; + private MessageId _nextMessageId = null; - private final Reader _reader; - - // TODO: Revisit the logic of using a separate executor to manage the request timeout. Currently it is not thread safe - private final ExecutorService _executorService = Executors.newSingleThreadExecutor(); - - public PulsarPartitionLevelConsumer(String clientId, StreamConfig streamConfig, int partitionId) { + public PulsarPartitionLevelConsumer(String clientId, StreamConfig streamConfig, int partition) { super(clientId, streamConfig); + String topicName = _config.getPulsarTopicName(); try { - _reader = createReaderForPartition(partitionId); + List partitions = _pulsarClient.getPartitionsForTopic(topicName).get(); + _reader = _pulsarClient.newReader().topic(partitions.get(partition)).startMessageId(MessageId.earliest) + .startMessageIdInclusive().create(); } catch (Exception e) { - throw new RuntimeException("Caught exception while creating Pulsar reader", e); + throw new RuntimeException( + String.format("Caught exception while creating Pulsar reader for topic: %s, partition: %d", topicName, + partition), e); } - LOGGER.info("Created Pulsar reader with topic: {}, partition: {}, initial message id: {}", - _config.getPulsarTopicName(), partitionId, _config.getInitialMessageId()); + LOGGER.info("Created Pulsar reader for topic: {}, partition: {}", topicName, partition); } - /** - * Fetch records from the Pulsar stream between the start and end StreamPartitionMsgOffset - * 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 PulsarMessageBatch fetchMessages(StreamPartitionMsgOffset startMsgOffset, int timeoutMs) { - MessageIdStreamOffset startOffset = (MessageIdStreamOffset) startMsgOffset; + public synchronized PulsarMessageBatch fetchMessages(StreamPartitionMsgOffset startOffset, int timeoutMs) { + MessageId startMessageId = ((MessageIdStreamOffset) startOffset).getMessageId(); + long endTimeMs = System.currentTimeMillis() + timeoutMs; List messages = new ArrayList<>(); - Future pulsarResultFuture = _executorService.submit(() -> fetchMessages(startOffset, messages)); - try { - return pulsarResultFuture.get(timeoutMs, 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. - pulsarResultFuture.cancel(true); - } catch (Exception e) { - LOGGER.warn("Error while fetching records from Pulsar", e); + + // Seek to the start message id if necessary + // NOTE: Use Objects.equals() to check reference first for performance. + if (!Objects.equals(startMessageId, _nextMessageId)) { + try { + _reader.seek(startMessageId); + } catch (PulsarClientException e) { + throw new RuntimeException("Caught exception while seeking to message id: " + startMessageId, e); + } } - return buildPulsarMessageBatch(startOffset, messages); - } - private PulsarMessageBatch fetchMessages(MessageIdStreamOffset startOffset, List messages) { + // Read messages until all available messages are read, or we run out of time try { - MessageId startMessageId = startOffset.getMessageId(); - _reader.seek(startMessageId); - while (_reader.hasMessageAvailable()) { - Message message = _reader.readNext(); - messages.add(PulsarUtils.buildPulsarStreamMessage(message, _config)); - if (Thread.interrupted()) { - break; - } + while (_reader.hasMessageAvailable() && System.currentTimeMillis() < endTimeMs) { + messages.add(PulsarUtils.buildPulsarStreamMessage(_reader.readNext(), _config)); } } catch (PulsarClientException e) { - LOGGER.warn("Error consuming records from Pulsar topic", e); + throw new RuntimeException("Caught exception while fetching messages from Pulsar", e); } - return buildPulsarMessageBatch(startOffset, messages); - } - private PulsarMessageBatch buildPulsarMessageBatch(MessageIdStreamOffset startOffset, - List messages) { MessageIdStreamOffset offsetOfNextBatch; if (messages.isEmpty()) { - offsetOfNextBatch = startOffset; + offsetOfNextBatch = (MessageIdStreamOffset) startOffset; } else { StreamMessageMetadata lastMessageMetadata = messages.get(messages.size() - 1).getMetadata(); assert lastMessageMetadata != null; offsetOfNextBatch = (MessageIdStreamOffset) lastMessageMetadata.getNextOffset(); } - return new PulsarMessageBatch(messages, offsetOfNextBatch, false); + assert offsetOfNextBatch != null; + _nextMessageId = offsetOfNextBatch.getMessageId(); + return new PulsarMessageBatch(messages, offsetOfNextBatch, _reader.hasReachedEndOfTopic()); } @Override @@ -120,18 +101,5 @@ public void close() throws IOException { _reader.close(); super.close(); - shutdownAndAwaitTermination(); - } - - void shutdownAndAwaitTermination() { - _executorService.shutdown(); - try { - if (!_executorService.awaitTermination(60, TimeUnit.SECONDS)) { - _executorService.shutdownNow(); - } - } catch (InterruptedException ie) { - _executorService.shutdownNow(); - Thread.currentThread().interrupt(); - } } } diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtils.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtils.java index 301beb71fc2c..0ccacc304704 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtils.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/main/java/org/apache/pinot/plugin/stream/pulsar/PulsarUtils.java @@ -53,16 +53,6 @@ public static SubscriptionInitialPosition offsetCriteriaToSubscription(OffsetCri throw new IllegalArgumentException("Unsupported offset criteria: " + offsetCriteria); } - public static MessageId offsetCriteriaToMessageId(OffsetCriteria offsetCriteria) { - if (offsetCriteria.isLargest()) { - return MessageId.latest; - } - if (offsetCriteria.isSmallest()) { - return MessageId.earliest; - } - throw new IllegalArgumentException("Unsupported offset criteria: " + offsetCriteria); - } - /** * Stitch key and value bytes together using a simple format: * 4 bytes for key length + key bytes + 4 bytes for value length + value bytes diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerTest.java b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerTest.java index 5b7fbe4ae168..01cd5cd26e0f 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerTest.java +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/src/test/java/org/apache/pinot/plugin/stream/pulsar/PulsarConsumerTest.java @@ -18,24 +18,18 @@ */ package org.apache.pinot.plugin.stream.pulsar; -import com.google.common.base.Function; import java.time.Duration; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import javax.annotation.Nullable; -import org.apache.pinot.spi.stream.MessageBatch; -import org.apache.pinot.spi.stream.PartitionGroupConsumer; +import org.apache.pinot.spi.stream.BytesStreamMessage; import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; import org.apache.pinot.spi.stream.StreamConfig; -import org.apache.pinot.spi.stream.StreamConfigProperties; import org.apache.pinot.spi.stream.StreamConsumerFactory; import org.apache.pinot.spi.stream.StreamConsumerFactoryProvider; +import org.apache.pinot.spi.stream.StreamMessageMetadata; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Message; @@ -45,12 +39,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TopicMetadata; -import org.apache.pulsar.client.impl.BatchMessageIdImpl; -import org.apache.pulsar.client.impl.MessageIdImpl; -import org.apache.pulsar.client.internal.DefaultImplementation; import org.apache.pulsar.common.policies.data.InactiveTopicPolicies; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.testcontainers.containers.PulsarContainer; import org.testcontainers.utility.DockerImageName; import org.testng.annotations.AfterClass; @@ -58,61 +47,46 @@ import org.testng.annotations.Test; import static org.testng.Assert.assertEquals; -import static org.testng.Assert.fail; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; public class PulsarConsumerTest { - private static final Logger LOGGER = LoggerFactory.getLogger(PulsarConsumerTest.class); - - private static final DockerImageName PULSAR_IMAGE = DockerImageName.parse("apachepulsar/pulsar:2.7.2"); + private static final DockerImageName PULSAR_IMAGE = DockerImageName.parse("apachepulsar/pulsar:2.11.4"); public static final String TABLE_NAME_WITH_TYPE = "tableName_REALTIME"; public static final String TEST_TOPIC = "test-topic"; public static final String TEST_TOPIC_BATCH = "test-topic-batch"; - public static final String MESSAGE_PREFIX = "sample_msg"; + public static final String MESSAGE_PREFIX = "sample_msg_"; public static final String CLIENT_ID = "clientId"; - public static final int NUM_PARTITION = 1; + public static final int NUM_PARTITIONS = 2; public static final int NUM_RECORDS_PER_PARTITION = 1000; public static final int BATCH_SIZE = 10; - public static final int CONSUMER_FETCH_TIMEOUT_MILLIS = (int) Duration.ofMinutes(5).toMillis(); + public static final int CONSUMER_FETCH_TIMEOUT_MILLIS = (int) TimeUnit.MINUTES.toMillis(1); + + private final List> _partitionToMessageIdMapping = new ArrayList<>(NUM_PARTITIONS); + private final List> _partitionToMessageIdMappingBatch = new ArrayList<>(NUM_PARTITIONS); + private PulsarContainer _pulsar; private PulsarClient _pulsarClient; - private PulsarContainer _pulsar = null; - private HashMap _partitionToFirstMessageIdMap = new HashMap<>(); - private HashMap _partitionToFirstMessageIdMapBatch = new HashMap<>(); - private ConcurrentHashMap> _partitionToMessageIdMapping = new ConcurrentHashMap<>(); @BeforeClass public void setUp() throws Exception { + _pulsar = new PulsarContainer(PULSAR_IMAGE).withStartupTimeout(Duration.ofMinutes(5)); try { - _pulsar = new PulsarContainer(PULSAR_IMAGE).withStartupTimeout(Duration.ofMinutes(5)); _pulsar.start(); - - // Waiting for namespace to be created. - // There should be a better approach. - Thread.sleep(20 * 1000L); - - PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(_pulsar.getHttpServiceUrl()).build(); - - String bootstrapServer = _pulsar.getPulsarBrokerUrl(); - - _pulsarClient = PulsarClient.builder().serviceUrl(bootstrapServer).build(); - - createTopics(admin); - - publishRecords(); - publishRecordsBatch(); - - waitForMessagesToPublish(admin, TEST_TOPIC); - waitForMessagesToPublish(admin, TEST_TOPIC_BATCH); - - admin.close(); - } catch (Exception e) { - if (_pulsar != null) { - _pulsar.stop(); - _pulsar = null; + _pulsarClient = PulsarClient.builder().serviceUrl(_pulsar.getPulsarBrokerUrl()).build(); + + try (PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(_pulsar.getHttpServiceUrl()).build()) { + createTopics(admin); + publishRecords(); + publishRecordsBatch(); + waitForMessagesToPublish(admin, TEST_TOPIC); + waitForMessagesToPublish(admin, TEST_TOPIC_BATCH); } + } catch (Exception e) { + _pulsar.stop(); throw new RuntimeException("Failed to setUp test environment", e); } } @@ -123,50 +97,36 @@ private void createTopics(PulsarAdmin admin) inactiveTopicPolicies.setDeleteWhileInactive(false); admin.namespaces().setInactiveTopicPolicies("public/default", inactiveTopicPolicies); - admin.topics().createPartitionedTopic(TEST_TOPIC, NUM_PARTITION); - admin.topics().createPartitionedTopic(TEST_TOPIC_BATCH, NUM_PARTITION); + admin.topics().createPartitionedTopic(TEST_TOPIC, NUM_PARTITIONS); + admin.topics().createPartitionedTopic(TEST_TOPIC_BATCH, NUM_PARTITIONS); } - private void waitForMessagesToPublish(PulsarAdmin admin, String topicName) { - waitForCondition(new Function() { - @Nullable - @Override - public Boolean apply(@Nullable Void aVoid) { - try { - return getNumberOfEntries(admin, topicName) == NUM_RECORDS_PER_PARTITION * NUM_PARTITION; - } catch (Exception e) { - LOGGER.warn("Could not fetch number of messages in pulsar topic " + topicName, e); - return null; - } + private void waitForMessagesToPublish(PulsarAdmin admin, String topicName) + throws Exception { + long endTimeMs = System.currentTimeMillis() + TimeUnit.MINUTES.toMillis(5); + while (System.currentTimeMillis() < endTimeMs) { + if (admin.topics().getPartitionedStats(topicName, false).getMsgInCounter() + == NUM_RECORDS_PER_PARTITION * NUM_PARTITIONS) { + return; } - }, 2000L, 60 * 1000L, "Failed to produce " + NUM_RECORDS_PER_PARTITION * NUM_PARTITION + " messages", true); - } - - private long getNumberOfEntries(PulsarAdmin admin, String topicName) { - try { - return admin.topics().getPartitionedStats(topicName, false).getMsgInCounter(); - } catch (Exception e) { - e.printStackTrace(); - LOGGER.warn("Could not fetch number of rows in pulsar topic " + topicName, e); + Thread.sleep(1000); } - return -1; + throw new RuntimeException("Failed to publish messages to topic: " + topicName); } @AfterClass public void tearDown() throws Exception { - if (_pulsar != null) { - _pulsar.stop(); - _pulsarClient.close(); - _partitionToMessageIdMapping.clear(); - _pulsar = null; - } + _pulsarClient.close(); + _pulsar.stop(); } public void publishRecords() throws Exception { - for (int p = 0; p < NUM_PARTITION; p++) { - final int partition = p; + for (int p = 0; p < NUM_PARTITIONS; p++) { + List messageIds = new ArrayList<>(NUM_RECORDS_PER_PARTITION); + _partitionToMessageIdMapping.add(messageIds); + int partition = p; try (Producer producer = _pulsarClient.newProducer(Schema.STRING).topic(TEST_TOPIC) .messageRouter(new MessageRouter() { @Override @@ -175,23 +135,19 @@ public int choosePartition(Message msg, TopicMetadata metadata) { } }).create()) { for (int i = 0; i < NUM_RECORDS_PER_PARTITION; i++) { - MessageId messageId = producer.send(MESSAGE_PREFIX + "_" + i); - if (!_partitionToFirstMessageIdMap.containsKey(partition)) { - _partitionToFirstMessageIdMap.put(partition, messageId); - } + messageIds.add(producer.send(MESSAGE_PREFIX + i)); } producer.flush(); } - waitForCondition(input -> validatePartitionMessageCount(partition, NUM_RECORDS_PER_PARTITION, TEST_TOPIC), - 1 * 1000L, 5 * 60 * 1000L, - "Failed to consume " + NUM_RECORDS_PER_PARTITION + " messages from partition " + partition, true); } } public void publishRecordsBatch() throws Exception { - for (int p = 0; p < NUM_PARTITION; p++) { - final int partition = p; + for (int p = 0; p < NUM_PARTITIONS; p++) { + List messageIds = new ArrayList<>(NUM_RECORDS_PER_PARTITION); + _partitionToMessageIdMappingBatch.add(messageIds); + int partition = p; try (Producer producer = _pulsarClient.newProducer(Schema.STRING).topic(TEST_TOPIC_BATCH) .messageRouter(new MessageRouter() { @Override @@ -200,71 +156,23 @@ public int choosePartition(Message msg, TopicMetadata metadata) { } }).batchingMaxMessages(BATCH_SIZE).batchingMaxPublishDelay(1, TimeUnit.SECONDS).create()) { for (int i = 0; i < NUM_RECORDS_PER_PARTITION; i++) { - CompletableFuture messageIdCompletableFuture = producer.sendAsync(MESSAGE_PREFIX + "_" + i); - messageIdCompletableFuture.thenAccept(messageId -> { - - _partitionToMessageIdMapping.compute(partition, (partitionId, messageIds) -> { - if (messageIds == null) { - List messageIdList = new ArrayList<>(); - messageIdList.add((BatchMessageIdImpl) messageId); - if (!_partitionToFirstMessageIdMapBatch.containsKey(partition)) { - _partitionToFirstMessageIdMapBatch.put(partition, messageId); - } - return messageIdList; - } else { - messageIds.add((BatchMessageIdImpl) messageId); - return messageIds; - } - }); - }); + messageIds.add(producer.send(MESSAGE_PREFIX + i)); } producer.flush(); } - waitForCondition(input -> validatePartitionMessageCount(partition, NUM_RECORDS_PER_PARTITION, TEST_TOPIC_BATCH), - 1 * 1000L, 5 * 60 * 1000L, - "Failed to consume " + NUM_RECORDS_PER_PARTITION + " messages from partition " + partition, true); - } - } - - private boolean validatePartitionMessageCount(int partition, int expectedMsgCount, String topicName) { - final PartitionGroupConsumer consumer = StreamConsumerFactoryProvider.create(getStreamConfig(topicName)) - .createPartitionGroupConsumer(CLIENT_ID, - new PartitionGroupConsumptionStatus(partition, 1, new MessageIdStreamOffset(MessageId.earliest), null, - "CONSUMING")); - try { - MessageBatch messageBatch = - consumer.fetchMessages(new MessageIdStreamOffset(MessageId.earliest), CONSUMER_FETCH_TIMEOUT_MILLIS); - LOGGER.info("Partition: " + partition + ", Consumed messageBatch count = " + messageBatch.getMessageCount()); - return messageBatch.getMessageCount() == expectedMsgCount; - } catch (TimeoutException e) { - return false; } } public StreamConfig getStreamConfig(String topicName) { - String streamType = "pulsar"; - String streamPulsarBrokerList = _pulsar.getPulsarBrokerUrl(); - String streamPulsarConsumerType = "simple"; - String tableNameWithType = TABLE_NAME_WITH_TYPE; - Map streamConfigMap = new HashMap<>(); - streamConfigMap.put("streamType", streamType); - streamConfigMap.put("stream.pulsar.consumer.type", streamPulsarConsumerType); + streamConfigMap.put("streamType", "pulsar"); + streamConfigMap.put("stream.pulsar.consumer.type", "simple"); streamConfigMap.put("stream.pulsar.topic.name", topicName); - streamConfigMap.put("stream.pulsar.bootstrap.servers", streamPulsarBrokerList); + streamConfigMap.put("stream.pulsar.bootstrap.servers", _pulsar.getPulsarBrokerUrl()); streamConfigMap.put("stream.pulsar.consumer.prop.auto.offset.reset", "smallest"); - streamConfigMap.put("stream.pulsar.consumer.factory.class.name", getPulsarConsumerFactoryName()); - streamConfigMap.put( - StreamConfigProperties.constructStreamProperty("pulsar", StreamConfigProperties.STREAM_FETCH_TIMEOUT_MILLIS), - "1000"); - streamConfigMap.put("stream.pulsar.decoder.class.name", "decoderClass"); - StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); - - return streamConfig; - } - - protected String getPulsarConsumerFactoryName() { - return PulsarConsumerFactory.class.getName(); + streamConfigMap.put("stream.pulsar.consumer.factory.class.name", PulsarConsumerFactory.class.getName()); + streamConfigMap.put("stream.pulsar.decoder.class.name", "dummy"); + return new StreamConfig(TABLE_NAME_WITH_TYPE, streamConfigMap); } @Test @@ -288,23 +196,39 @@ public void testPartitionLevelConsumer() PulsarMessageBatch messageBatch = consumer.fetchMessages(new MessageIdStreamOffset(MessageId.earliest), CONSUMER_FETCH_TIMEOUT_MILLIS); assertEquals(messageBatch.getMessageCount(), 1000); + assertFalse(messageBatch.isEndOfPartitionGroup()); for (int i = 0; i < 1000; i++) { - byte[] msg = messageBatch.getStreamMessage(i).getValue(); - assertEquals(new String(msg), "sample_msg_" + i); + verifyMessage(messageBatch.getStreamMessage(i), partition, i, false); } messageBatch = - consumer.fetchMessages(new MessageIdStreamOffset(getMessageIdForPartitionAndIndex(partition, 500)), + consumer.fetchMessages(new MessageIdStreamOffset(_partitionToMessageIdMapping.get(partition).get(500)), CONSUMER_FETCH_TIMEOUT_MILLIS); assertEquals(messageBatch.getMessageCount(), 500); + assertFalse(messageBatch.isEndOfPartitionGroup()); for (int i = 0; i < 500; i++) { - byte[] msg = messageBatch.getStreamMessage(i).getValue(); - assertEquals(new String(msg), "sample_msg_" + (500 + i)); + verifyMessage(messageBatch.getStreamMessage(i), partition, 500 + i, false); } } } } + private void verifyMessage(BytesStreamMessage streamMessage, int partition, int index, boolean batch) { + assertEquals(new String(streamMessage.getValue()), MESSAGE_PREFIX + index); + StreamMessageMetadata messageMetadata = streamMessage.getMetadata(); + assertNotNull(messageMetadata); + MessageIdStreamOffset offset = (MessageIdStreamOffset) messageMetadata.getOffset(); + assertNotNull(offset); + MessageIdStreamOffset nextOffset = (MessageIdStreamOffset) messageMetadata.getNextOffset(); + assertNotNull(nextOffset); + List messageIds = + batch ? _partitionToMessageIdMappingBatch.get(partition) : _partitionToMessageIdMapping.get(partition); + assertEquals(offset.getMessageId(), messageIds.get(index)); + if (index < NUM_RECORDS_PER_PARTITION - 1) { + assertEquals(nextOffset.getMessageId(), messageIds.get(index + 1)); + } + } + @Test public void testPartitionLevelConsumerBatchMessages() throws Exception { @@ -327,50 +251,20 @@ public void testPartitionLevelConsumerBatchMessages() PulsarMessageBatch messageBatch = consumer.fetchMessages(new MessageIdStreamOffset(MessageId.earliest), CONSUMER_FETCH_TIMEOUT_MILLIS); assertEquals(messageBatch.getMessageCount(), 1000); + assertFalse(messageBatch.isEndOfPartitionGroup()); for (int i = 0; i < 1000; i++) { - byte[] msg = messageBatch.getStreamMessage(i).getValue(); - assertEquals(new String(msg), "sample_msg_" + i); + verifyMessage(messageBatch.getStreamMessage(i), partition, i, true); } messageBatch = - consumer.fetchMessages(new MessageIdStreamOffset(getBatchMessageIdForPartitionAndIndex(partition, 500)), + consumer.fetchMessages(new MessageIdStreamOffset(_partitionToMessageIdMappingBatch.get(partition).get(500)), CONSUMER_FETCH_TIMEOUT_MILLIS); assertEquals(messageBatch.getMessageCount(), 500); - for (int i = 0; i < messageBatch.getMessageCount(); i++) { - byte[] msg = messageBatch.getStreamMessage(i).getValue(); - assertEquals(new String(msg), "sample_msg_" + (500 + i)); - } - } - } - } - - private MessageId getMessageIdForPartitionAndIndex(int partitionNum, int index) { - MessageId startMessageIdRaw = _partitionToFirstMessageIdMap.get(partitionNum); - MessageIdImpl startMessageId = MessageIdImpl.convertToMessageIdImpl(startMessageIdRaw); - return DefaultImplementation.getDefaultImplementation() - .newMessageId(startMessageId.getLedgerId(), index, partitionNum); - } - - private MessageId getBatchMessageIdForPartitionAndIndex(int partitionNum, int index) { - return _partitionToMessageIdMapping.get(partitionNum).get(index); - } - - private void waitForCondition(Function condition, long checkIntervalMs, long timeoutMs, - @Nullable String errorMessage, boolean raiseError) { - long endTime = System.currentTimeMillis() + timeoutMs; - String errorMessageSuffix = errorMessage != null ? ", error message: " + errorMessage : ""; - while (System.currentTimeMillis() < endTime) { - try { - if (Boolean.TRUE.equals(condition.apply(null))) { - return; + assertFalse(messageBatch.isEndOfPartitionGroup()); + for (int i = 0; i < 500; i++) { + verifyMessage(messageBatch.getStreamMessage(i), partition, 500 + i, true); } - Thread.sleep(checkIntervalMs); - } catch (Exception e) { - fail("Caught exception while checking the condition" + errorMessageSuffix, e); } } - if (raiseError) { - fail("Failed to meet condition in " + timeoutMs + "ms" + errorMessageSuffix); - } } }