From f03043a2caa1b5f31ee5e425c8ea295b154acef7 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Sat, 16 Nov 2024 00:51:31 +0530 Subject: [PATCH 01/47] initial refactoring for handling threshold logic --- .../realtime/RealtimeSegmentDataManager.java | 4 +++- .../mutable/MutableSegmentImpl.java | 18 ++++++++++++++++++ .../realtime/impl/RealtimeSegmentConfig.java | 16 ++++++++++++++-- .../SegmentsValidationAndRetentionConfig.java | 9 +++++++++ .../spi/utils/builder/TableConfigBuilder.java | 2 ++ 5 files changed, 46 insertions(+), 3 deletions(-) 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 2fbee173afa5..5a51be5309f8 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 @@ -309,6 +309,7 @@ public void deleteSegmentFile() { private String _stopReason = null; private final Semaphore _segBuildSemaphore; private final boolean _isOffHeap; + private final boolean _enableThresholdForNumOfValues; /** * Whether null handling is enabled by default. This value is only used if * {@link Schema#isEnableColumnBasedNullHandling()} is false. @@ -1529,6 +1530,7 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf _isOffHeap = indexLoadingConfig.isRealtimeOffHeapAllocation(); _defaultNullHandlingEnabled = indexingConfig.isNullHandlingEnabled(); + _enableThresholdForNumOfValues = tableConfig.getValidationConfig().isEnableThresholdForNumOfValues(); // Start new realtime segment String consumerDir = realtimeTableDataManager.getConsumerDir(); @@ -1552,7 +1554,7 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf .setUpsertDropOutOfOrderRecord(tableConfig.isDropOutOfOrderRecord()) .setPartitionDedupMetadataManager(partitionDedupMetadataManager) .setDedupTimeColumn(tableConfig.getDedupTimeColumn()) - .setFieldConfigList(tableConfig.getFieldConfigList()); + .setFieldConfigList(tableConfig.getFieldConfigList()).setEnableThresholdForNumOfValues(_enableThresholdForNumOfValues); // Create message decoder Set fieldsToRead = IngestionUtils.getFieldsForRecordExtractor(_tableConfig.getIngestionConfig(), _schema); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index 06fb22ec0e2c..72e96f35db41 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -147,8 +147,10 @@ public class MutableSegmentImpl implements MutableSegment { private final int _mainPartitionId; // partition id designated for this consuming segment private final boolean _defaultNullHandlingEnabled; private final File _consumerDir; + private final boolean _enableThresholdForNumOfValues; private final Map _indexContainerMap = new HashMap<>(); + private final Map _columnVsNumValues = new HashMap<>(); private final IdMap _recordIdMap; @@ -225,6 +227,7 @@ public boolean isMutableSegment() { _mainPartitionId = config.getPartitionId(); _defaultNullHandlingEnabled = config.isNullHandlingEnabled(); _consumerDir = new File(config.getConsumerDir()); + _enableThresholdForNumOfValues = config.isEnableThresholdForNumOfValues(); Collection allFieldSpecs = _schema.getAllFieldSpecs(); List physicalFieldSpecs = new ArrayList<>(allFieldSpecs.size()); @@ -586,9 +589,24 @@ public boolean index(GenericRow row, @Nullable RowMetadata rowMetadata) _latestIngestionTimeMs = Math.max(_latestIngestionTimeMs, rowMetadata.getRecordIngestionTimeMs()); } + if (_enableThresholdForNumOfValues) { + canTakeMore = canTakeMore && isNumOfValuesBelowThreshold(row); + } + return canTakeMore; } + private boolean isNumOfValuesBelowThreshold(GenericRow row) { + for (Map.Entry entry : _indexContainerMap.entrySet()) { + String column = entry.getKey(); + Object value = row.getValue(column); + Object[] values = (Object[]) value; + int prevCount = _columnVsNumValues.getOrDefault(column, 0); + _columnVsNumValues.put(column, prevCount + values.length); + } + return true; + } + private boolean isUpsertEnabled() { return _partitionUpsertMetadataManager != null; } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java index 5b3aeb26d533..dffec834aa78 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java @@ -73,6 +73,7 @@ public class RealtimeSegmentConfig { private final String _consumerDir; private final List _fieldConfigList; private final List _ingestionAggregationConfigs; + private final boolean _enableThresholdForNumOfValues; // TODO: Clean up this constructor. Most of these things can be extracted from tableConfig. @@ -89,7 +90,7 @@ private RealtimeSegmentConfig(String tableNameWithType, String segmentName, Stri List upsertComparisonColumns, String upsertDeleteRecordColumn, String upsertOutOfOrderRecordColumn, boolean upsertDropOutOfOrderRecord, PartitionUpsertMetadataManager partitionUpsertMetadataManager, String dedupTimeColumn, PartitionDedupMetadataManager partitionDedupMetadataManager, - List fieldConfigList, List ingestionAggregationConfigs) { + List fieldConfigList, List ingestionAggregationConfigs, boolean enableThresholdForNumOfValues) { _tableNameWithType = tableNameWithType; _segmentName = segmentName; _streamName = streamName; @@ -119,6 +120,7 @@ private RealtimeSegmentConfig(String tableNameWithType, String segmentName, Stri _partitionDedupMetadataManager = partitionDedupMetadataManager; _fieldConfigList = fieldConfigList; _ingestionAggregationConfigs = ingestionAggregationConfigs; + _enableThresholdForNumOfValues = enableThresholdForNumOfValues; } public String getTableNameWithType() { @@ -241,6 +243,10 @@ public List getIngestionAggregationConfigs() { return _ingestionAggregationConfigs; } + public boolean isEnableThresholdForNumOfValues() { + return _enableThresholdForNumOfValues; + } + public static class Builder { private String _tableNameWithType; private String _segmentName; @@ -275,6 +281,7 @@ public static class Builder { private PartitionDedupMetadataManager _partitionDedupMetadataManager; private List _fieldConfigList; private List _ingestionAggregationConfigs; + private boolean _enableThresholdForNumOfValues = false; public Builder() { _indexConfigByCol = new HashMap<>(); @@ -475,6 +482,11 @@ public Builder setIngestionAggregationConfigs(List ingestionA return this; } + public Builder setEnableThresholdForNumOfValues(boolean enableThresholdForNumOfValues) { + _enableThresholdForNumOfValues = enableThresholdForNumOfValues; + return this; + } + public RealtimeSegmentConfig build() { Map indexConfigByCol = Maps.newHashMapWithExpectedSize(_indexConfigByCol.size()); for (Map.Entry entry : _indexConfigByCol.entrySet()) { @@ -487,7 +499,7 @@ public RealtimeSegmentConfig build() { _defaultNullHandlingEnabled, _consumerDir, _upsertMode, _upsertConsistencyMode, _upsertComparisonColumns, _upsertDeleteRecordColumn, _upsertOutOfOrderRecordColumn, _upsertDropOutOfOrderRecord, _partitionUpsertMetadataManager, _dedupTimeColumn, _partitionDedupMetadataManager, _fieldConfigList, - _ingestionAggregationConfigs); + _ingestionAggregationConfigs, _enableThresholdForNumOfValues); } } } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentsValidationAndRetentionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentsValidationAndRetentionConfig.java index 0b8a403041ab..cfbc25f7671c 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentsValidationAndRetentionConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentsValidationAndRetentionConfig.java @@ -51,6 +51,7 @@ public class SegmentsValidationAndRetentionConfig extends BaseJsonConfig { // using the specified download scheme. Both realtime tables and offline tables can set this field. // For more usage of this field, please refer to this design doc: https://tinyurl.com/f63ru4sb private String _peerSegmentDownloadScheme; + private boolean _enableThresholdForNumOfValues; @Deprecated public String getSegmentAssignmentStrategy() { @@ -218,6 +219,14 @@ public void setPeerSegmentDownloadScheme(String peerSegmentDownloadScheme) { _peerSegmentDownloadScheme = peerSegmentDownloadScheme; } + public boolean isEnableThresholdForNumOfValues() { + return _enableThresholdForNumOfValues; + } + + public void setEnableThresholdForNumOfValues(boolean enableThresholdForNumOfValues) { + _enableThresholdForNumOfValues = enableThresholdForNumOfValues; + } + public String getCrypterClassName() { return _crypterClassName; } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java index dc8fb2ae8a11..ba1970fb359c 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java @@ -81,6 +81,7 @@ public class TableConfigBuilder { private ReplicaGroupStrategyConfig _replicaGroupStrategyConfig; private CompletionConfig _completionConfig; private String _crypterClassName; + private boolean _enableThresholdForNumOfValues = false; // Tenant config related private String _brokerTenant; @@ -439,6 +440,7 @@ public TableConfig build() { validationConfig.setReplication(_numReplicas); validationConfig.setPeerSegmentDownloadScheme(_peerSegmentDownloadScheme); validationConfig.setCrypterClassName(_crypterClassName); + validationConfig.setEnableThresholdForNumOfValues(_enableThresholdForNumOfValues); // Tenant config TenantConfig tenantConfig = new TenantConfig(_brokerTenant, _serverTenant, _tagOverrideConfig); From 09d43d780ae1277e9107f873d611595d02e000e2 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Mon, 18 Nov 2024 12:05:41 +0530 Subject: [PATCH 02/47] Adds logic for endCriteriaReached based upon numValues --- .../realtime/RealtimeSegmentDataManager.java | 5 ++++ .../mutable/MutableSegmentImpl.java | 28 +++++++++++++++---- 2 files changed, 28 insertions(+), 5 deletions(-) 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 5a51be5309f8..00e8b4845f86 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 @@ -558,6 +558,11 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee for (int index = 0; index < messageCount; index++) { prematureExit = _shouldStop || endCriteriaReached(); + + if (_enableThresholdForNumOfValues) { + prematureExit = prematureExit || _realtimeSegment.isNumOfValuesAboveThreshold(); + } + if (prematureExit) { if (_segmentLogger.isDebugEnabled()) { _segmentLogger.debug("stop processing message batch early shouldStop: {}", _shouldStop); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index 72e96f35db41..0ba9e3fcc6c8 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -128,6 +128,7 @@ public class MutableSegmentImpl implements MutableSegment { private static final int EXPECTED_COMPRESSION = 1000; private static final int MIN_ROWS_TO_INDEX = 1000_000; // Min size of recordIdMap for updatable metrics. private static final int MIN_RECORD_ID_MAP_CACHE_SIZE = 10000; // Min overflow map size for updatable metrics. + private final static int DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN = 2_000_000_000; private final Logger _logger; private final long _startTimeMillis = System.currentTimeMillis(); @@ -227,7 +228,7 @@ public boolean isMutableSegment() { _mainPartitionId = config.getPartitionId(); _defaultNullHandlingEnabled = config.isNullHandlingEnabled(); _consumerDir = new File(config.getConsumerDir()); - _enableThresholdForNumOfValues = config.isEnableThresholdForNumOfValues(); + _enableThresholdForNumOfValues = config.isEnableThresholdForNumOfValues(); Collection allFieldSpecs = _schema.getAllFieldSpecs(); List physicalFieldSpecs = new ArrayList<>(allFieldSpecs.size()); @@ -590,21 +591,26 @@ public boolean index(GenericRow row, @Nullable RowMetadata rowMetadata) } if (_enableThresholdForNumOfValues) { - canTakeMore = canTakeMore && isNumOfValuesBelowThreshold(row); + updateNumOfValues(row); } return canTakeMore; } - private boolean isNumOfValuesBelowThreshold(GenericRow row) { + private void updateNumOfValues(GenericRow row) { for (Map.Entry entry : _indexContainerMap.entrySet()) { + IndexContainer indexContainer = entry.getValue(); + var fieldSpec = indexContainer._fieldSpec; + if (fieldSpec.isSingleValueField()) { + continue; + } String column = entry.getKey(); Object value = row.getValue(column); Object[] values = (Object[]) value; int prevCount = _columnVsNumValues.getOrDefault(column, 0); - _columnVsNumValues.put(column, prevCount + values.length); + int newCount = prevCount + values.length; + _columnVsNumValues.put(column, newCount); } - return true; } private boolean isUpsertEnabled() { @@ -1247,6 +1253,18 @@ private boolean isAggregateMetricsEnabled() { return _recordIdMap != null; } + public boolean isNumOfValuesAboveThreshold() { + for (String col : _columnVsNumValues.keySet()) { + int numValues = _columnVsNumValues.get(col); + if (numValues > DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN) { + _logger.info("Num of values for col:{} is above threshold for segment:{}", col, + _segmentName); + return true; + } + } + return false; + } + // NOTE: Okay for single-writer @SuppressWarnings("NonAtomicOperationOnVolatileField") private static class ValuesInfo { From 121d23ae040799f8963bde878b7da227742a0074 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Mon, 18 Nov 2024 16:15:20 +0530 Subject: [PATCH 03/47] add logic inside endCriteriaReached --- .../protocols/SegmentCompletionProtocol.java | 1 + .../realtime/RealtimeSegmentDataManager.java | 11 ++++++--- .../mutable/MutableSegmentImpl.java | 24 +++++++++---------- 3 files changed, 20 insertions(+), 16 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/protocols/SegmentCompletionProtocol.java b/pinot-common/src/main/java/org/apache/pinot/common/protocols/SegmentCompletionProtocol.java index 1cad2d09f5ad..eb26a4f3eed6 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/protocols/SegmentCompletionProtocol.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/protocols/SegmentCompletionProtocol.java @@ -149,6 +149,7 @@ public enum ControllerResponseStatus { public static final String REASON_END_OF_PARTITION_GROUP = "endOfPartitionGroup"; // Stop reason sent by server as force commit message received public static final String REASON_FORCE_COMMIT_MESSAGE_RECEIVED = "forceCommitMessageReceived"; + public static final String REASON_NUM_VALUES_ABOVE_THRESHOLD = "numValuesAboveThreshold"; // Canned responses public static final Response RESP_NOT_LEADER = 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 00e8b4845f86..dd391759d394 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 @@ -363,6 +363,11 @@ private boolean endCriteriaReached() { _numRowsConsumed, _numRowsIndexed); _stopReason = SegmentCompletionProtocol.REASON_FORCE_COMMIT_MESSAGE_RECEIVED; return true; + } else if (_enableThresholdForNumOfValues && _realtimeSegment.isNumOfValuesAboveThreshold()) { + _segmentLogger.info("Stopping consumption as num of values for a column is above threshold - numRowsConsumed={} numRowsIndexed={}", + _numRowsConsumed, _numRowsIndexed); + _stopReason = SegmentCompletionProtocol.REASON_NUM_VALUES_ABOVE_THRESHOLD; + return true; } return false; @@ -559,9 +564,9 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee for (int index = 0; index < messageCount; index++) { prematureExit = _shouldStop || endCriteriaReached(); - if (_enableThresholdForNumOfValues) { - prematureExit = prematureExit || _realtimeSegment.isNumOfValuesAboveThreshold(); - } +// if (_enableThresholdForNumOfValues) { +// prematureExit = prematureExit || _realtimeSegment.isNumOfValuesAboveThreshold(); +// } if (prematureExit) { if (_segmentLogger.isDebugEnabled()) { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index 0ba9e3fcc6c8..dc7bfeab8a6e 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -151,7 +151,8 @@ public class MutableSegmentImpl implements MutableSegment { private final boolean _enableThresholdForNumOfValues; private final Map _indexContainerMap = new HashMap<>(); - private final Map _columnVsNumValues = new HashMap<>(); + private final Map _multiColumnNameVsNumValues = new HashMap<>(); + private boolean _numOfColValuesLimitBreached = false; private final IdMap _recordIdMap; @@ -607,9 +608,14 @@ private void updateNumOfValues(GenericRow row) { String column = entry.getKey(); Object value = row.getValue(column); Object[] values = (Object[]) value; - int prevCount = _columnVsNumValues.getOrDefault(column, 0); - int newCount = prevCount + values.length; - _columnVsNumValues.put(column, newCount); + int prevCount = _multiColumnNameVsNumValues.getOrDefault(column, 0); + long newCount = prevCount + values.length; + if (newCount > DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN) { + _logger.warn("Number of total values for column {} is {} and has breached the threshold limit {}", + column, newCount, DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN); + _numOfColValuesLimitBreached = true; + } + _multiColumnNameVsNumValues.put(column, newCount); } } @@ -1254,15 +1260,7 @@ private boolean isAggregateMetricsEnabled() { } public boolean isNumOfValuesAboveThreshold() { - for (String col : _columnVsNumValues.keySet()) { - int numValues = _columnVsNumValues.get(col); - if (numValues > DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN) { - _logger.info("Num of values for col:{} is above threshold for segment:{}", col, - _segmentName); - return true; - } - } - return false; + return _numOfColValuesLimitBreached; } // NOTE: Okay for single-writer From 06523b6c492f100d5e35824bfad5209870923989 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Mon, 18 Nov 2024 16:19:05 +0530 Subject: [PATCH 04/47] renames var --- .../realtime/RealtimeSegmentDataManager.java | 11 ++++++----- .../indexsegment/mutable/MutableSegmentImpl.java | 6 +++--- .../realtime/impl/RealtimeSegmentConfig.java | 16 ++++++++-------- .../SegmentsValidationAndRetentionConfig.java | 10 +++++----- .../spi/utils/builder/TableConfigBuilder.java | 4 ++-- 5 files changed, 24 insertions(+), 23 deletions(-) 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 dd391759d394..9e2daaf2f325 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 @@ -309,7 +309,7 @@ public void deleteSegmentFile() { private String _stopReason = null; private final Semaphore _segBuildSemaphore; private final boolean _isOffHeap; - private final boolean _enableThresholdForNumOfValues; + private final boolean _thresholdForNumOfColValuesEnabled; /** * Whether null handling is enabled by default. This value is only used if * {@link Schema#isEnableColumnBasedNullHandling()} is false. @@ -363,7 +363,7 @@ private boolean endCriteriaReached() { _numRowsConsumed, _numRowsIndexed); _stopReason = SegmentCompletionProtocol.REASON_FORCE_COMMIT_MESSAGE_RECEIVED; return true; - } else if (_enableThresholdForNumOfValues && _realtimeSegment.isNumOfValuesAboveThreshold()) { + } else if (_thresholdForNumOfColValuesEnabled && _realtimeSegment.isNumOfValuesAboveThreshold()) { _segmentLogger.info("Stopping consumption as num of values for a column is above threshold - numRowsConsumed={} numRowsIndexed={}", _numRowsConsumed, _numRowsIndexed); _stopReason = SegmentCompletionProtocol.REASON_NUM_VALUES_ABOVE_THRESHOLD; @@ -564,7 +564,7 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee for (int index = 0; index < messageCount; index++) { prematureExit = _shouldStop || endCriteriaReached(); -// if (_enableThresholdForNumOfValues) { +// if (_thresholdForNumOfColValuesEnabled) { // prematureExit = prematureExit || _realtimeSegment.isNumOfValuesAboveThreshold(); // } @@ -1540,7 +1540,7 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf _isOffHeap = indexLoadingConfig.isRealtimeOffHeapAllocation(); _defaultNullHandlingEnabled = indexingConfig.isNullHandlingEnabled(); - _enableThresholdForNumOfValues = tableConfig.getValidationConfig().isEnableThresholdForNumOfValues(); + _thresholdForNumOfColValuesEnabled = tableConfig.getValidationConfig().isThresholdForNumOfColValuesEnabled(); // Start new realtime segment String consumerDir = realtimeTableDataManager.getConsumerDir(); @@ -1564,7 +1564,8 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf .setUpsertDropOutOfOrderRecord(tableConfig.isDropOutOfOrderRecord()) .setPartitionDedupMetadataManager(partitionDedupMetadataManager) .setDedupTimeColumn(tableConfig.getDedupTimeColumn()) - .setFieldConfigList(tableConfig.getFieldConfigList()).setEnableThresholdForNumOfValues(_enableThresholdForNumOfValues); + .setFieldConfigList(tableConfig.getFieldConfigList()).setThresholdForNumOfColValuesEnabled( + _thresholdForNumOfColValuesEnabled); // Create message decoder Set fieldsToRead = IngestionUtils.getFieldsForRecordExtractor(_tableConfig.getIngestionConfig(), _schema); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index dc7bfeab8a6e..d3e390cb80bf 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -148,7 +148,7 @@ public class MutableSegmentImpl implements MutableSegment { private final int _mainPartitionId; // partition id designated for this consuming segment private final boolean _defaultNullHandlingEnabled; private final File _consumerDir; - private final boolean _enableThresholdForNumOfValues; + private final boolean _thresholdForNumOfColValuesEnabled; private final Map _indexContainerMap = new HashMap<>(); private final Map _multiColumnNameVsNumValues = new HashMap<>(); @@ -229,7 +229,7 @@ public boolean isMutableSegment() { _mainPartitionId = config.getPartitionId(); _defaultNullHandlingEnabled = config.isNullHandlingEnabled(); _consumerDir = new File(config.getConsumerDir()); - _enableThresholdForNumOfValues = config.isEnableThresholdForNumOfValues(); + _thresholdForNumOfColValuesEnabled = config.isThresholdForNumOfColValuesEnabled(); Collection allFieldSpecs = _schema.getAllFieldSpecs(); List physicalFieldSpecs = new ArrayList<>(allFieldSpecs.size()); @@ -591,7 +591,7 @@ public boolean index(GenericRow row, @Nullable RowMetadata rowMetadata) _latestIngestionTimeMs = Math.max(_latestIngestionTimeMs, rowMetadata.getRecordIngestionTimeMs()); } - if (_enableThresholdForNumOfValues) { + if (_thresholdForNumOfColValuesEnabled) { updateNumOfValues(row); } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java index dffec834aa78..efa690c06e53 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java @@ -73,7 +73,7 @@ public class RealtimeSegmentConfig { private final String _consumerDir; private final List _fieldConfigList; private final List _ingestionAggregationConfigs; - private final boolean _enableThresholdForNumOfValues; + private final boolean _thresholdForNumOfColValuesEnabled; // TODO: Clean up this constructor. Most of these things can be extracted from tableConfig. @@ -120,7 +120,7 @@ private RealtimeSegmentConfig(String tableNameWithType, String segmentName, Stri _partitionDedupMetadataManager = partitionDedupMetadataManager; _fieldConfigList = fieldConfigList; _ingestionAggregationConfigs = ingestionAggregationConfigs; - _enableThresholdForNumOfValues = enableThresholdForNumOfValues; + _thresholdForNumOfColValuesEnabled = enableThresholdForNumOfValues; } public String getTableNameWithType() { @@ -243,8 +243,8 @@ public List getIngestionAggregationConfigs() { return _ingestionAggregationConfigs; } - public boolean isEnableThresholdForNumOfValues() { - return _enableThresholdForNumOfValues; + public boolean isThresholdForNumOfColValuesEnabled() { + return _thresholdForNumOfColValuesEnabled; } public static class Builder { @@ -281,7 +281,7 @@ public static class Builder { private PartitionDedupMetadataManager _partitionDedupMetadataManager; private List _fieldConfigList; private List _ingestionAggregationConfigs; - private boolean _enableThresholdForNumOfValues = false; + private boolean _thresholdForNumOfColValuesEnabled = false; public Builder() { _indexConfigByCol = new HashMap<>(); @@ -482,8 +482,8 @@ public Builder setIngestionAggregationConfigs(List ingestionA return this; } - public Builder setEnableThresholdForNumOfValues(boolean enableThresholdForNumOfValues) { - _enableThresholdForNumOfValues = enableThresholdForNumOfValues; + public Builder setThresholdForNumOfColValuesEnabled(boolean thresholdForNumOfColValuesEnabled) { + _thresholdForNumOfColValuesEnabled = thresholdForNumOfColValuesEnabled; return this; } @@ -499,7 +499,7 @@ public RealtimeSegmentConfig build() { _defaultNullHandlingEnabled, _consumerDir, _upsertMode, _upsertConsistencyMode, _upsertComparisonColumns, _upsertDeleteRecordColumn, _upsertOutOfOrderRecordColumn, _upsertDropOutOfOrderRecord, _partitionUpsertMetadataManager, _dedupTimeColumn, _partitionDedupMetadataManager, _fieldConfigList, - _ingestionAggregationConfigs, _enableThresholdForNumOfValues); + _ingestionAggregationConfigs, _thresholdForNumOfColValuesEnabled); } } } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentsValidationAndRetentionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentsValidationAndRetentionConfig.java index cfbc25f7671c..e410d69b9703 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentsValidationAndRetentionConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentsValidationAndRetentionConfig.java @@ -51,7 +51,7 @@ public class SegmentsValidationAndRetentionConfig extends BaseJsonConfig { // using the specified download scheme. Both realtime tables and offline tables can set this field. // For more usage of this field, please refer to this design doc: https://tinyurl.com/f63ru4sb private String _peerSegmentDownloadScheme; - private boolean _enableThresholdForNumOfValues; + private boolean _thresholdForNumOfColValuesEnabled; @Deprecated public String getSegmentAssignmentStrategy() { @@ -219,12 +219,12 @@ public void setPeerSegmentDownloadScheme(String peerSegmentDownloadScheme) { _peerSegmentDownloadScheme = peerSegmentDownloadScheme; } - public boolean isEnableThresholdForNumOfValues() { - return _enableThresholdForNumOfValues; + public boolean isThresholdForNumOfColValuesEnabled() { + return _thresholdForNumOfColValuesEnabled; } - public void setEnableThresholdForNumOfValues(boolean enableThresholdForNumOfValues) { - _enableThresholdForNumOfValues = enableThresholdForNumOfValues; + public void setThresholdForNumOfColValuesEnabled(boolean thresholdForNumOfColValuesEnabled) { + _thresholdForNumOfColValuesEnabled = thresholdForNumOfColValuesEnabled; } public String getCrypterClassName() { diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java index ba1970fb359c..0e5a996450d7 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java @@ -81,7 +81,7 @@ public class TableConfigBuilder { private ReplicaGroupStrategyConfig _replicaGroupStrategyConfig; private CompletionConfig _completionConfig; private String _crypterClassName; - private boolean _enableThresholdForNumOfValues = false; + private boolean _thresholdForNumOfColValuesEnabled = false; // Tenant config related private String _brokerTenant; @@ -440,7 +440,7 @@ public TableConfig build() { validationConfig.setReplication(_numReplicas); validationConfig.setPeerSegmentDownloadScheme(_peerSegmentDownloadScheme); validationConfig.setCrypterClassName(_crypterClassName); - validationConfig.setEnableThresholdForNumOfValues(_enableThresholdForNumOfValues); + validationConfig.setThresholdForNumOfColValuesEnabled(_thresholdForNumOfColValuesEnabled); // Tenant config TenantConfig tenantConfig = new TenantConfig(_brokerTenant, _serverTenant, _tagOverrideConfig); From f47cb082cfcf425d4dcf11d806f09070f44b9646 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Mon, 18 Nov 2024 16:20:02 +0530 Subject: [PATCH 05/47] renames constant --- .../pinot/common/protocols/SegmentCompletionProtocol.java | 2 +- .../core/data/manager/realtime/RealtimeSegmentDataManager.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/protocols/SegmentCompletionProtocol.java b/pinot-common/src/main/java/org/apache/pinot/common/protocols/SegmentCompletionProtocol.java index eb26a4f3eed6..ade16eb63de1 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/protocols/SegmentCompletionProtocol.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/protocols/SegmentCompletionProtocol.java @@ -149,7 +149,7 @@ public enum ControllerResponseStatus { public static final String REASON_END_OF_PARTITION_GROUP = "endOfPartitionGroup"; // Stop reason sent by server as force commit message received public static final String REASON_FORCE_COMMIT_MESSAGE_RECEIVED = "forceCommitMessageReceived"; - public static final String REASON_NUM_VALUES_ABOVE_THRESHOLD = "numValuesAboveThreshold"; + public static final String REASON_NUM_OF_COL_VALUES_ABOVE_THRESHOLD = "numColValuesAboveThreshold"; // Canned responses public static final Response RESP_NOT_LEADER = 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 9e2daaf2f325..b2833d5ceaf5 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 @@ -366,7 +366,7 @@ private boolean endCriteriaReached() { } else if (_thresholdForNumOfColValuesEnabled && _realtimeSegment.isNumOfValuesAboveThreshold()) { _segmentLogger.info("Stopping consumption as num of values for a column is above threshold - numRowsConsumed={} numRowsIndexed={}", _numRowsConsumed, _numRowsIndexed); - _stopReason = SegmentCompletionProtocol.REASON_NUM_VALUES_ABOVE_THRESHOLD; + _stopReason = SegmentCompletionProtocol.REASON_NUM_OF_COL_VALUES_ABOVE_THRESHOLD; return true; } return false; From da8d691f1355da824643c23cddfe20151cb5c050 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Mon, 18 Nov 2024 16:22:30 +0530 Subject: [PATCH 06/47] nit --- .../core/data/manager/realtime/RealtimeSegmentDataManager.java | 2 +- .../segment/local/indexsegment/mutable/MutableSegmentImpl.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 b2833d5ceaf5..ba05129dd653 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 @@ -363,7 +363,7 @@ private boolean endCriteriaReached() { _numRowsConsumed, _numRowsIndexed); _stopReason = SegmentCompletionProtocol.REASON_FORCE_COMMIT_MESSAGE_RECEIVED; return true; - } else if (_thresholdForNumOfColValuesEnabled && _realtimeSegment.isNumOfValuesAboveThreshold()) { + } else if (_thresholdForNumOfColValuesEnabled && _realtimeSegment.isNumOfColValuesAboveThreshold()) { _segmentLogger.info("Stopping consumption as num of values for a column is above threshold - numRowsConsumed={} numRowsIndexed={}", _numRowsConsumed, _numRowsIndexed); _stopReason = SegmentCompletionProtocol.REASON_NUM_OF_COL_VALUES_ABOVE_THRESHOLD; diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index d3e390cb80bf..39c933dfea23 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -1259,7 +1259,7 @@ private boolean isAggregateMetricsEnabled() { return _recordIdMap != null; } - public boolean isNumOfValuesAboveThreshold() { + public boolean isNumOfColValuesAboveThreshold() { return _numOfColValuesLimitBreached; } From 2cff520c72776c73a98f44ccec304df88f46bf17 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Mon, 18 Nov 2024 16:23:52 +0530 Subject: [PATCH 07/47] nit --- .../segment/local/indexsegment/mutable/MutableSegmentImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index 39c933dfea23..4c812c7e5280 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -608,7 +608,7 @@ private void updateNumOfValues(GenericRow row) { String column = entry.getKey(); Object value = row.getValue(column); Object[] values = (Object[]) value; - int prevCount = _multiColumnNameVsNumValues.getOrDefault(column, 0); + long prevCount = _multiColumnNameVsNumValues.getOrDefault(column, 0L); long newCount = prevCount + values.length; if (newCount > DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN) { _logger.warn("Number of total values for column {} is {} and has breached the threshold limit {}", From 55bf033352882bd7c4b5cf5efde07818552bced2 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Mon, 18 Nov 2024 18:09:26 +0530 Subject: [PATCH 08/47] refactors logic to inside of addNewRow --- .../mutable/MutableSegmentImpl.java | 22 +++++-------------- 1 file changed, 6 insertions(+), 16 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index 4c812c7e5280..56f0c4ced13f 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -591,32 +591,18 @@ public boolean index(GenericRow row, @Nullable RowMetadata rowMetadata) _latestIngestionTimeMs = Math.max(_latestIngestionTimeMs, rowMetadata.getRecordIngestionTimeMs()); } - if (_thresholdForNumOfColValuesEnabled) { - updateNumOfValues(row); - } - return canTakeMore; } - private void updateNumOfValues(GenericRow row) { - for (Map.Entry entry : _indexContainerMap.entrySet()) { - IndexContainer indexContainer = entry.getValue(); - var fieldSpec = indexContainer._fieldSpec; - if (fieldSpec.isSingleValueField()) { - continue; - } - String column = entry.getKey(); - Object value = row.getValue(column); - Object[] values = (Object[]) value; + private void updateNumOfValuesMap(String column, int valuesLen) { long prevCount = _multiColumnNameVsNumValues.getOrDefault(column, 0L); - long newCount = prevCount + values.length; + long newCount = prevCount + valuesLen; if (newCount > DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN) { _logger.warn("Number of total values for column {} is {} and has breached the threshold limit {}", column, newCount, DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN); _numOfColValuesLimitBreached = true; } _multiColumnNameVsNumValues.put(column, newCount); - } } private boolean isUpsertEnabled() { @@ -828,6 +814,10 @@ private void addNewRow(int docId, GenericRow row) { } } indexContainer._valuesInfo.updateMVNumValues(values.length); + + if (_thresholdForNumOfColValuesEnabled) { + updateNumOfValuesMap(column, values.length); + } } } } From 4549e5c42ffcc85d83eb80760f904d26b23d408d Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Mon, 18 Nov 2024 18:14:05 +0530 Subject: [PATCH 09/47] remove comments --- .../data/manager/realtime/RealtimeSegmentDataManager.java | 4 ---- 1 file changed, 4 deletions(-) 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 ba05129dd653..af1ba43a1112 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 @@ -564,10 +564,6 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee for (int index = 0; index < messageCount; index++) { prematureExit = _shouldStop || endCriteriaReached(); -// if (_thresholdForNumOfColValuesEnabled) { -// prematureExit = prematureExit || _realtimeSegment.isNumOfValuesAboveThreshold(); -// } - if (prematureExit) { if (_segmentLogger.isDebugEnabled()) { _segmentLogger.debug("stop processing message batch early shouldStop: {}", _shouldStop); From 36c5f0da5da330223b5d18db5c0a5db575b01d95 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Mon, 18 Nov 2024 18:24:26 +0530 Subject: [PATCH 10/47] removes usage of map --- .../mutable/MutableSegmentImpl.java | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index 56f0c4ced13f..8a39550f9fbb 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -151,7 +151,6 @@ public class MutableSegmentImpl implements MutableSegment { private final boolean _thresholdForNumOfColValuesEnabled; private final Map _indexContainerMap = new HashMap<>(); - private final Map _multiColumnNameVsNumValues = new HashMap<>(); private boolean _numOfColValuesLimitBreached = false; private final IdMap _recordIdMap; @@ -594,17 +593,6 @@ public boolean index(GenericRow row, @Nullable RowMetadata rowMetadata) return canTakeMore; } - private void updateNumOfValuesMap(String column, int valuesLen) { - long prevCount = _multiColumnNameVsNumValues.getOrDefault(column, 0L); - long newCount = prevCount + valuesLen; - if (newCount > DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN) { - _logger.warn("Number of total values for column {} is {} and has breached the threshold limit {}", - column, newCount, DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN); - _numOfColValuesLimitBreached = true; - } - _multiColumnNameVsNumValues.put(column, newCount); - } - private boolean isUpsertEnabled() { return _partitionUpsertMetadataManager != null; } @@ -813,11 +801,19 @@ private void addNewRow(int docId, GenericRow row) { recordIndexingError(indexEntry.getKey(), e); } } - indexContainer._valuesInfo.updateMVNumValues(values.length); if (_thresholdForNumOfColValuesEnabled) { - updateNumOfValuesMap(column, values.length); + int prevCount = indexContainer._valuesInfo.getNumValues(); + long newCount = prevCount + 1L + values.length; + + if (newCount > DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN) { + _logger.warn("Number of total values for column {} is {} and has breached the threshold limit {}", + column, newCount, DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN); + _numOfColValuesLimitBreached = true; + } } + + indexContainer._valuesInfo.updateMVNumValues(values.length); } } } @@ -1309,6 +1305,10 @@ void updateVarByteMVMaxRowLengthInBytes(Object entry, DataType dataType) { throw new IllegalStateException("Invalid type=" + dataType); } } + + int getNumValues() { + return _numValues; + } } private static Map> getMetricsAggregators(RealtimeSegmentConfig segmentConfig) { From c2f78cb358d9be6de94ef7b1c8269a9fc01908ab Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Mon, 18 Nov 2024 18:28:53 +0530 Subject: [PATCH 11/47] nit --- .../manager/realtime/RealtimeSegmentDataManager.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) 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 af1ba43a1112..d701ad34abd4 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 @@ -364,7 +364,9 @@ private boolean endCriteriaReached() { _stopReason = SegmentCompletionProtocol.REASON_FORCE_COMMIT_MESSAGE_RECEIVED; return true; } else if (_thresholdForNumOfColValuesEnabled && _realtimeSegment.isNumOfColValuesAboveThreshold()) { - _segmentLogger.info("Stopping consumption as num of values for a column is above threshold - numRowsConsumed={} numRowsIndexed={}", + _segmentLogger.info( + "Stopping consumption as num of values for a column is above threshold - numRowsConsumed={} " + + "numRowsIndexed={}", _numRowsConsumed, _numRowsIndexed); _stopReason = SegmentCompletionProtocol.REASON_NUM_OF_COL_VALUES_ABOVE_THRESHOLD; return true; @@ -563,7 +565,6 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee for (int index = 0; index < messageCount; index++) { prematureExit = _shouldStop || endCriteriaReached(); - if (prematureExit) { if (_segmentLogger.isDebugEnabled()) { _segmentLogger.debug("stop processing message batch early shouldStop: {}", _shouldStop); @@ -1240,6 +1241,7 @@ private static class ConsumptionStopIndicator { final Logger _logger; final ServerSegmentCompletionProtocolHandler _protocolHandler; final String _reason; + private ConsumptionStopIndicator(StreamPartitionMsgOffset offset, String segmentName, String instanceId, ServerSegmentCompletionProtocolHandler protocolHandler, String reason, Logger logger) { _offset = offset; @@ -1560,8 +1562,8 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf .setUpsertDropOutOfOrderRecord(tableConfig.isDropOutOfOrderRecord()) .setPartitionDedupMetadataManager(partitionDedupMetadataManager) .setDedupTimeColumn(tableConfig.getDedupTimeColumn()) - .setFieldConfigList(tableConfig.getFieldConfigList()).setThresholdForNumOfColValuesEnabled( - _thresholdForNumOfColValuesEnabled); + .setFieldConfigList(tableConfig.getFieldConfigList()) + .setThresholdForNumOfColValuesEnabled(_thresholdForNumOfColValuesEnabled); // Create message decoder Set fieldsToRead = IngestionUtils.getFieldsForRecordExtractor(_tableConfig.getIngestionConfig(), _schema); @@ -1634,7 +1636,7 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf "Failed to initialize segment data manager", e)); _segmentLogger.warn( "Scheduling task to call controller to mark the segment as OFFLINE in Ideal State due" - + " to initialization error: '{}'", + + " to initialization error: '{}'", e.getMessage()); // Since we are going to throw exception from this thread (helix execution thread), the externalview // entry for this segment will be ERROR. We allow time for Helix to make this transition, and then From 0acb3f60a1ae006ef7351623c3459516289372b2 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Mon, 18 Nov 2024 22:49:47 +0530 Subject: [PATCH 12/47] fixes lint --- .../segment/local/realtime/impl/RealtimeSegmentConfig.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java index efa690c06e53..c1401b9b4238 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java @@ -90,7 +90,8 @@ private RealtimeSegmentConfig(String tableNameWithType, String segmentName, Stri List upsertComparisonColumns, String upsertDeleteRecordColumn, String upsertOutOfOrderRecordColumn, boolean upsertDropOutOfOrderRecord, PartitionUpsertMetadataManager partitionUpsertMetadataManager, String dedupTimeColumn, PartitionDedupMetadataManager partitionDedupMetadataManager, - List fieldConfigList, List ingestionAggregationConfigs, boolean enableThresholdForNumOfValues) { + List fieldConfigList, List ingestionAggregationConfigs, + boolean enableThresholdForNumOfValues) { _tableNameWithType = tableNameWithType; _segmentName = segmentName; _streamName = streamName; @@ -120,7 +121,7 @@ private RealtimeSegmentConfig(String tableNameWithType, String segmentName, Stri _partitionDedupMetadataManager = partitionDedupMetadataManager; _fieldConfigList = fieldConfigList; _ingestionAggregationConfigs = ingestionAggregationConfigs; - _thresholdForNumOfColValuesEnabled = enableThresholdForNumOfValues; + _thresholdForNumOfColValuesEnabled = enableThresholdForNumOfValues; } public String getTableNameWithType() { From 96e8663d6df421399f8fd6fabecd720f9b77bd39 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Tue, 19 Nov 2024 15:13:00 +0530 Subject: [PATCH 13/47] nit --- .../segment/local/indexsegment/mutable/MutableSegmentImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index 8a39550f9fbb..12f91e849859 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -804,7 +804,7 @@ private void addNewRow(int docId, GenericRow row) { if (_thresholdForNumOfColValuesEnabled) { int prevCount = indexContainer._valuesInfo.getNumValues(); - long newCount = prevCount + 1L + values.length; + long newCount = (long) prevCount + values.length; if (newCount > DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN) { _logger.warn("Number of total values for column {} is {} and has breached the threshold limit {}", From ea48ba0b456a7224b707a520a48218253bf3a481 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Wed, 20 Nov 2024 17:29:06 +0530 Subject: [PATCH 14/47] refactoring --- .../mutable/MutableSegmentImpl.java | 29 +++++++------------ .../FixedByteMVMutableForwardIndex.java | 10 +++++++ .../spi/index/mutable/MutableIndex.java | 5 ++++ 3 files changed, 26 insertions(+), 18 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index 12f91e849859..885b871ad9e7 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -128,7 +128,6 @@ public class MutableSegmentImpl implements MutableSegment { private static final int EXPECTED_COMPRESSION = 1000; private static final int MIN_ROWS_TO_INDEX = 1000_000; // Min size of recordIdMap for updatable metrics. private static final int MIN_RECORD_ID_MAP_CACHE_SIZE = 10000; // Min overflow map size for updatable metrics. - private final static int DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN = 2_000_000_000; private final Logger _logger; private final long _startTimeMillis = System.currentTimeMillis(); @@ -796,23 +795,21 @@ private void addNewRow(int docId, GenericRow row) { Object[] values = (Object[]) value; for (Map.Entry indexEntry : indexContainer._mutableIndexes.entrySet()) { try { - indexEntry.getValue().add(values, dictIds, docId); + MutableIndex mutableIndex = indexEntry.getValue(); + if (_thresholdForNumOfColValuesEnabled) { + if (mutableIndex.canAdd(values.length)) { + mutableIndex.add(values, dictIds, docId); + } else { + _logger.warn("Cannot add new row for column {} due to num of col value threshold limit", column); + _numOfColValuesLimitBreached = true; + } + } else { + mutableIndex.add(values, dictIds, docId); + } } catch (Exception e) { recordIndexingError(indexEntry.getKey(), e); } } - - if (_thresholdForNumOfColValuesEnabled) { - int prevCount = indexContainer._valuesInfo.getNumValues(); - long newCount = (long) prevCount + values.length; - - if (newCount > DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN) { - _logger.warn("Number of total values for column {} is {} and has breached the threshold limit {}", - column, newCount, DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN); - _numOfColValuesLimitBreached = true; - } - } - indexContainer._valuesInfo.updateMVNumValues(values.length); } } @@ -1305,10 +1302,6 @@ void updateVarByteMVMaxRowLengthInBytes(Object entry, DataType dataType) { throw new IllegalStateException("Invalid type=" + dataType); } } - - int getNumValues() { - return _numValues; - } } private static Map> getMetricsAggregators(RealtimeSegmentConfig segmentConfig) { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/FixedByteMVMutableForwardIndex.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/FixedByteMVMutableForwardIndex.java index 16c877c0217e..6e0f09275c9b 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/FixedByteMVMutableForwardIndex.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/FixedByteMVMutableForwardIndex.java @@ -104,6 +104,8 @@ public class FixedByteMVMutableForwardIndex implements MutableForwardIndex { private static final int INCREMENT_PERCENTAGE = 100; //Increments the Initial size by 100% of initial capacity every time we runs out of capacity + private final static int DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN = 400_000_000; + // For single writer multiple readers setup, use ArrayList for writer and CopyOnWriteArrayList for reader private final List _headerWriters = new ArrayList<>(); private final List _headerReaders = new CopyOnWriteArrayList<>(); @@ -124,6 +126,7 @@ public class FixedByteMVMutableForwardIndex implements MutableForwardIndex { private int _currentCapacity = 0; private int _prevRowStartIndex = 0; // Offset in the data-buffer for the last row added. private int _prevRowLength = 0; // Number of values in the column for the last row added. + private int _numValues = 0; public FixedByteMVMutableForwardIndex(int maxNumberOfMultiValuesPerRow, int avgMultiValueCount, int rowCountPerChunk, int columnSizeInBytes, PinotDataBufferMemoryManager memoryManager, String context, boolean isDictionaryEncoded, @@ -200,6 +203,7 @@ private int getRowInCurrentHeader(int row) { private int updateHeader(int row, int numValues) { assert (numValues <= _maxNumberOfMultiValuesPerRow); + _numValues += numValues; int newStartIndex = _prevRowStartIndex + _prevRowLength; if (newStartIndex + numValues > _currentCapacity) { addDataBuffer(_incrementalCapacity); @@ -408,6 +412,12 @@ public void setDoubleMV(int docId, double[] values) { } } + @Override + public boolean canAdd(int valuesLen) { + long newNumValues = (long) _numValues + valuesLen; + return newNumValues < DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN; + } + @Override public void close() throws IOException { diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableIndex.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableIndex.java index dc3bdc98691d..33fb5964cf2a 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableIndex.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableIndex.java @@ -68,4 +68,9 @@ public interface MutableIndex extends IndexReader { */ default void commit() { } + + default boolean canAdd(int valuesLen) { + return true; + } + } From 58370ca252a7cdc3cb0e31f17353708cb0c48a69 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Wed, 20 Nov 2024 17:31:14 +0530 Subject: [PATCH 15/47] nit --- .../core/data/manager/realtime/RealtimeSegmentDataManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 d701ad34abd4..b0fd1ca2ba15 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 @@ -1636,7 +1636,7 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf "Failed to initialize segment data manager", e)); _segmentLogger.warn( "Scheduling task to call controller to mark the segment as OFFLINE in Ideal State due" - + " to initialization error: '{}'", + + " to initialization error: '{}'", e.getMessage()); // Since we are going to throw exception from this thread (helix execution thread), the externalview // entry for this segment will be ERROR. We allow time for Helix to make this transition, and then From fbc9c93fe0bfa023d3419d7b999ffe3f099151a1 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Thu, 21 Nov 2024 21:38:31 +0530 Subject: [PATCH 16/47] Adds test --- .../mutable/MutableSegmentImpl.java | 16 ++-- .../FixedByteMVMutableForwardIndex.java | 5 +- ...tableSegmentEntriesAboveThresholdTest.java | 79 +++++++++++++++++++ .../mutable/MutableSegmentImplTestUtils.java | 11 +-- .../spi/index/mutable/MutableIndex.java | 2 +- 5 files changed, 95 insertions(+), 18 deletions(-) create mode 100644 pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index 885b871ad9e7..eb43e488f15b 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -796,15 +796,13 @@ private void addNewRow(int docId, GenericRow row) { for (Map.Entry indexEntry : indexContainer._mutableIndexes.entrySet()) { try { MutableIndex mutableIndex = indexEntry.getValue(); - if (_thresholdForNumOfColValuesEnabled) { - if (mutableIndex.canAdd(values.length)) { - mutableIndex.add(values, dictIds, docId); - } else { - _logger.warn("Cannot add new row for column {} due to num of col value threshold limit", column); - _numOfColValuesLimitBreached = true; - } - } else { - mutableIndex.add(values, dictIds, docId); + mutableIndex.add(values, dictIds, docId); + if (_thresholdForNumOfColValuesEnabled && !mutableIndex.canAddMore()) { + _logger.warn( + "failed to index value with {} for column {} due to num of col value threshold limit", + indexEntry.getKey(), column + ); + _numOfColValuesLimitBreached = true; } } catch (Exception e) { recordIndexingError(indexEntry.getKey(), e); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/FixedByteMVMutableForwardIndex.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/FixedByteMVMutableForwardIndex.java index 6e0f09275c9b..4a671b0cfbf1 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/FixedByteMVMutableForwardIndex.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/FixedByteMVMutableForwardIndex.java @@ -413,9 +413,8 @@ public void setDoubleMV(int docId, double[] values) { } @Override - public boolean canAdd(int valuesLen) { - long newNumValues = (long) _numValues + valuesLen; - return newNumValues < DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN; + public boolean canAddMore() { + return _numValues < DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN; } @Override diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java new file mode 100644 index 000000000000..16bb5bb18225 --- /dev/null +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java @@ -0,0 +1,79 @@ +package org.apache.pinot.segment.local.indexsegment.mutable; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.util.Collections; +import javax.annotation.Nullable; +import org.apache.commons.io.FileUtils; +import org.apache.pinot.common.metrics.ServerMetrics; +import org.apache.pinot.segment.local.dedup.PartitionDedupMetadataManager; +import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader; +import org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentConfig; +import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer; +import org.apache.pinot.segment.local.segment.creator.SegmentTestUtils; +import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; +import org.apache.pinot.segment.local.segment.virtualcolumn.VirtualColumnProviderFactory; +import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; +import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver; +import org.apache.pinot.spi.config.table.DedupConfig; +import org.apache.pinot.spi.config.table.HashFunction; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.data.readers.FileFormat; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.data.readers.RecordReader; +import org.apache.pinot.spi.data.readers.RecordReaderFactory; +import org.apache.pinot.spi.stream.RowMetadata; +import org.apache.pinot.spi.stream.StreamMessageMetadata; +import org.apache.pinot.spi.utils.ReadMode; +import org.apache.pinot.spi.utils.builder.TableConfigBuilder; +import org.testng.Assert; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + + +public class MutableSegmentEntriesAboveThresholdTest { + private static final File TEMP_DIR = + new File(FileUtils.getTempDirectory(), MutableSegmentEntriesAboveThresholdTest.class.getSimpleName()); + private static final String AVRO_FILE = "data/test_data-mv.avro"; + private Schema _schema; + + @Test + public void testIfLimitBreached() + throws Exception { + FileUtils.deleteQuietly(TEMP_DIR); + MutableSegmentImpl _mutableSegmentImpl; + + URL resourceUrl = MutableSegmentImplTest.class.getClassLoader().getResource(AVRO_FILE); + Assert.assertNotNull(resourceUrl); + File avroFile = new File(resourceUrl.getFile()); + + SegmentGeneratorConfig config = + SegmentTestUtils.getSegmentGeneratorConfigWithoutTimeColumn(avroFile, TEMP_DIR, "testTable"); + SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl(); + driver.init(config); + driver.build(); + + _schema = config.getSchema(); + VirtualColumnProviderFactory.addBuiltInVirtualColumnsToSegmentSchema(_schema, "testSegment"); + _mutableSegmentImpl = MutableSegmentImplTestUtils + .createMutableSegmentImpl(_schema, Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), + Collections.emptyMap(), + false, false, null, null, null, null, null, null, Collections.emptyList(), true); + var _lastIngestionTimeMs = System.currentTimeMillis(); + StreamMessageMetadata defaultMetadata = new StreamMessageMetadata(_lastIngestionTimeMs, new GenericRow()); + + try (RecordReader recordReader = RecordReaderFactory + .getRecordReader(FileFormat.AVRO, avroFile, _schema.getColumnNames(), null)) { + GenericRow reuse = new GenericRow(); + while (recordReader.hasNext()) { + _mutableSegmentImpl.index(recordReader.next(reuse), defaultMetadata); + } + } + + assert !_mutableSegmentImpl.isNumOfColValuesAboveThreshold(); + + } +} diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java index b23f203ec7a1..d71e63d018a5 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java @@ -59,7 +59,7 @@ public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set varLengthDictionaryColumns, Set invertedIndexColumns, List preAggregationConfigs) { return createMutableSegmentImpl(schema, noDictionaryColumns, varLengthDictionaryColumns, invertedIndexColumns, - Collections.emptyMap(), false, false, null, null, null, null, null, null, preAggregationConfigs); + Collections.emptyMap(), false, false, null, null, null, null, null, null, preAggregationConfigs, false); } public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set noDictionaryColumns, @@ -82,14 +82,14 @@ public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set noDictionaryColumns, Set varLengthDictionaryColumns, Set invertedIndexColumns, Map jsonIndexConfigs, ServerMetrics serverMetrics) { return createMutableSegmentImpl(schema, noDictionaryColumns, varLengthDictionaryColumns, invertedIndexColumns, - jsonIndexConfigs, false, true, null, null, null, null, null, serverMetrics, Collections.emptyList()); + jsonIndexConfigs, false, true, null, null, null, null, null, serverMetrics, Collections.emptyList(), false); } public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set noDictionaryColumns, @@ -97,7 +97,7 @@ public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set jsonIndexConfigs, boolean aggregateMetrics, boolean nullHandlingEnabled, UpsertConfig upsertConfig, String timeColumnName, PartitionUpsertMetadataManager partitionUpsertMetadataManager, DedupConfig dedupConfig, PartitionDedupMetadataManager partitionDedupMetadataManager, ServerMetrics serverMetrics, - List aggregationConfigs) { + List aggregationConfigs, boolean thresholdForColEnabled) { RealtimeSegmentStatsHistory statsHistory = mock(RealtimeSegmentStatsHistory.class); when(statsHistory.getEstimatedCardinality(anyString())).thenReturn(200); @@ -126,7 +126,8 @@ public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set entry : jsonIndexConfigs.entrySet()) { segmentConfBuilder.setIndex(entry.getKey(), StandardIndexes.json(), entry.getValue()); } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableIndex.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableIndex.java index 33fb5964cf2a..83deb4df28c0 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableIndex.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableIndex.java @@ -69,7 +69,7 @@ public interface MutableIndex extends IndexReader { default void commit() { } - default boolean canAdd(int valuesLen) { + default boolean canAddMore() { return true; } From 8722efcfbdb4d236eefe1eb9ae281e6842ffa6f7 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Sat, 23 Nov 2024 05:20:31 +0530 Subject: [PATCH 17/47] Adds test --- ...tableSegmentEntriesAboveThresholdTest.java | 142 ++++++++++++++++-- 1 file changed, 127 insertions(+), 15 deletions(-) diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java index 16bb5bb18225..f29d78924f61 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java @@ -2,35 +2,29 @@ import java.io.File; import java.io.IOException; +import java.lang.reflect.Field; import java.net.URL; import java.util.Collections; -import javax.annotation.Nullable; +import java.util.Map; import org.apache.commons.io.FileUtils; -import org.apache.pinot.common.metrics.ServerMetrics; -import org.apache.pinot.segment.local.dedup.PartitionDedupMetadataManager; -import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader; -import org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentConfig; -import org.apache.pinot.segment.local.recordtransformer.CompositeTransformer; import org.apache.pinot.segment.local.segment.creator.SegmentTestUtils; import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; +import org.apache.pinot.segment.local.segment.index.forward.ForwardIndexPlugin; import org.apache.pinot.segment.local.segment.virtualcolumn.VirtualColumnProviderFactory; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver; -import org.apache.pinot.spi.config.table.DedupConfig; -import org.apache.pinot.spi.config.table.HashFunction; -import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.segment.spi.index.IndexType; +import org.apache.pinot.segment.spi.index.StandardIndexes; +import org.apache.pinot.segment.spi.index.mutable.MutableForwardIndex; +import org.apache.pinot.segment.spi.index.mutable.MutableIndex; +import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.data.readers.FileFormat; import org.apache.pinot.spi.data.readers.GenericRow; import org.apache.pinot.spi.data.readers.RecordReader; import org.apache.pinot.spi.data.readers.RecordReaderFactory; -import org.apache.pinot.spi.stream.RowMetadata; import org.apache.pinot.spi.stream.StreamMessageMetadata; -import org.apache.pinot.spi.utils.ReadMode; -import org.apache.pinot.spi.utils.builder.TableConfigBuilder; import org.testng.Assert; -import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -40,8 +34,67 @@ public class MutableSegmentEntriesAboveThresholdTest { private static final String AVRO_FILE = "data/test_data-mv.avro"; private Schema _schema; + private class FakeMutableForwardIndex implements MutableForwardIndex { + + private final MutableForwardIndex _mutableForwardIndex; + private static final int threshold = 2; + private int _numValues; + + FakeMutableForwardIndex(MutableForwardIndex mutableForwardIndex) { + this._mutableForwardIndex = mutableForwardIndex; + this._numValues = 0; + } + + @Override + public boolean canAddMore() { + return _numValues < threshold; + } + + @Override + public void setDictIdMV(int docId, int[] dictIds) { + _numValues += dictIds.length; + this._mutableForwardIndex.setDictIdMV(docId, dictIds); + } + + @Override + public int getLengthOfShortestElement() { + return 0; + } + + @Override + public int getLengthOfLongestElement() { + return 0; + } + + @Override + public void setDictId(int docId, int dictId) { + this._mutableForwardIndex.setDictId(docId, dictId); + } + + @Override + public boolean isDictionaryEncoded() { + return false; + } + + @Override + public boolean isSingleValue() { + return false; + } + + @Override + public FieldSpec.DataType getStoredType() { + return null; + } + + @Override + public void close() + throws IOException { + + } + } + @Test - public void testIfLimitBreached() + public void testNoLimitBreached() throws Exception { FileUtils.deleteQuietly(TEMP_DIR); MutableSegmentImpl _mutableSegmentImpl; @@ -74,6 +127,65 @@ public void testIfLimitBreached() } assert !_mutableSegmentImpl.isNumOfColValuesAboveThreshold(); + } + + @Test + public void testLimitBreached1() + throws Exception { + FileUtils.deleteQuietly(TEMP_DIR); + MutableSegmentImpl _mutableSegmentImpl; + + URL resourceUrl = MutableSegmentImplTest.class.getClassLoader().getResource(AVRO_FILE); + Assert.assertNotNull(resourceUrl); + File avroFile = new File(resourceUrl.getFile()); + + SegmentGeneratorConfig config = + SegmentTestUtils.getSegmentGeneratorConfigWithoutTimeColumn(avroFile, TEMP_DIR, "testTable"); + SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl(); + driver.init(config); + driver.build(); + + _schema = config.getSchema(); + VirtualColumnProviderFactory.addBuiltInVirtualColumnsToSegmentSchema(_schema, "testSegment"); + _mutableSegmentImpl = MutableSegmentImplTestUtils + .createMutableSegmentImpl(_schema, Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), + Collections.emptyMap(), + false, false, null, null, null, null, null, null, Collections.emptyList(), true); + var _lastIngestionTimeMs = System.currentTimeMillis(); + StreamMessageMetadata defaultMetadata = new StreamMessageMetadata(_lastIngestionTimeMs, new GenericRow()); + + Field indexContainerMapField = MutableSegmentImpl.class.getDeclaredField("_indexContainerMap"); + indexContainerMapField.setAccessible(true); + Map colVsIndexContainer = (Map) indexContainerMapField.get(_mutableSegmentImpl); + + for (Map.Entry entry : colVsIndexContainer.entrySet()) { + Object indexContainer = entry.getValue(); + Field mutableIndexesField = indexContainer.getClass().getDeclaredField("_mutableIndexes"); + mutableIndexesField.setAccessible(true); + Map indexTypeVsMutableIndex = + (Map) mutableIndexesField.get(indexContainer); + + MutableForwardIndex mutableForwardIndex = null; + for (IndexType indexType : indexTypeVsMutableIndex.keySet()) { + if (indexType.getId().equals(StandardIndexes.FORWARD_ID)) { + mutableForwardIndex = (MutableForwardIndex) indexTypeVsMutableIndex.get(indexType); + } + } + + assert mutableForwardIndex != null; + + indexTypeVsMutableIndex.put(new ForwardIndexPlugin().getIndexType(), + new FakeMutableForwardIndex(mutableForwardIndex)); + } + + try (RecordReader recordReader = RecordReaderFactory + .getRecordReader(FileFormat.AVRO, avroFile, _schema.getColumnNames(), null)) { + GenericRow reuse = new GenericRow(); + while (recordReader.hasNext()) { + _mutableSegmentImpl.index(recordReader.next(reuse), defaultMetadata); + } + } + assert _mutableSegmentImpl.isNumOfColValuesAboveThreshold(); } } From a1467b5e8d381b65940756174c0de8b39a1e9bdd Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Sat, 23 Nov 2024 22:10:19 +0530 Subject: [PATCH 18/47] Adds test for realtimesegmentDataManager --- .../RealtimeSegmentDataManagerTest.java | 32 ++++++-- ...tableSegmentEntriesAboveThresholdTest.java | 75 ++++++++----------- 2 files changed, 57 insertions(+), 50 deletions(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java index 910edf7d9bc8..57fab70a6297 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java @@ -48,10 +48,12 @@ import org.apache.pinot.core.realtime.impl.fakestream.FakeStreamConsumerFactory; import org.apache.pinot.core.realtime.impl.fakestream.FakeStreamMessageDecoder; import org.apache.pinot.segment.local.data.manager.TableDataManager; +import org.apache.pinot.segment.local.indexsegment.mutable.MutableSegmentImpl; import org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentStatsHistory; import org.apache.pinot.segment.local.segment.creator.Fixtures; import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.segment.local.utils.SegmentLocks; +import org.apache.pinot.segment.spi.MutableSegment; import org.apache.pinot.spi.config.instance.InstanceDataManagerConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.data.Schema; @@ -124,11 +126,12 @@ private SegmentZKMetadata createZkMetadata() { private FakeRealtimeSegmentDataManager createFakeSegmentManager() throws Exception { - return createFakeSegmentManager(false, new TimeSupplier(), null, null, null); + return createFakeSegmentManager(false, new TimeSupplier(), null, null, null, false); } private FakeRealtimeSegmentDataManager createFakeSegmentManager(boolean noUpsert, TimeSupplier timeSupplier, - @Nullable String maxRows, @Nullable String maxDuration, @Nullable TableConfig tableConfig) + @Nullable String maxRows, @Nullable String maxDuration, @Nullable TableConfig tableConfig, + boolean setThresholdForNumOfColValuesEnabled) throws Exception { SegmentZKMetadata segmentZKMetadata = createZkMetadata(); if (tableConfig == null) { @@ -145,6 +148,7 @@ private FakeRealtimeSegmentDataManager createFakeSegmentManager(boolean noUpsert tableConfig.getIndexingConfig().getStreamConfigs() .put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_TIME, maxDuration); } + tableConfig.getValidationConfig().setThresholdForNumOfColValuesEnabled(setThresholdForNumOfColValuesEnabled); RealtimeTableDataManager tableDataManager = createTableDataManager(tableConfig); LLCSegmentName llcSegmentName = new LLCSegmentName(SEGMENT_NAME_STR); _partitionGroupIdToSemaphoreMap.putIfAbsent(PARTITION_GROUP_ID, new Semaphore(1)); @@ -302,7 +306,7 @@ public void testCommitAfterCatchupWithPeriodOffset() StreamConfigProperties.constructStreamProperty(StreamConfigProperties.STREAM_CONSUMER_OFFSET_CRITERIA, "fakeStream"), "2d"); FakeRealtimeSegmentDataManager segmentDataManager = - createFakeSegmentManager(false, new TimeSupplier(), null, null, tableConfig); + createFakeSegmentManager(false, new TimeSupplier(), null, null, tableConfig, false); RealtimeSegmentDataManager.PartitionConsumer consumer = segmentDataManager.createPartitionConsumer(); final LongMsgOffset firstOffset = new LongMsgOffset(START_OFFSET_VALUE + 500); final LongMsgOffset catchupOffset = new LongMsgOffset(firstOffset.getOffset() + 10); @@ -349,7 +353,7 @@ public void testCommitAfterCatchupWithTimestampOffset() StreamConfigProperties.constructStreamProperty(StreamConfigProperties.STREAM_CONSUMER_OFFSET_CRITERIA, "fakeStream"), Instant.now().toString()); FakeRealtimeSegmentDataManager segmentDataManager = - createFakeSegmentManager(false, new TimeSupplier(), null, null, tableConfig); + createFakeSegmentManager(false, new TimeSupplier(), null, null, tableConfig, false); RealtimeSegmentDataManager.PartitionConsumer consumer = segmentDataManager.createPartitionConsumer(); final LongMsgOffset firstOffset = new LongMsgOffset(START_OFFSET_VALUE + 500); final LongMsgOffset catchupOffset = new LongMsgOffset(firstOffset.getOffset() + 10); @@ -636,6 +640,22 @@ public void testEndCriteriaChecking() segmentDataManager._timeSupplier.set(endTime); Assert.assertTrue(segmentDataManager.invokeEndCriteriaReached()); } + + try (FakeRealtimeSegmentDataManager segmentDataManager = createFakeSegmentManager(false, new TimeSupplier(), null, null, null, true)) { + segmentDataManager._state.set(segmentDataManager, RealtimeSegmentDataManager.State.INITIAL_CONSUMING); + Assert.assertFalse(segmentDataManager.invokeEndCriteriaReached()); + + Field realtimeSegmentField = RealtimeSegmentDataManager.class.getDeclaredField("_realtimeSegment"); + realtimeSegmentField.setAccessible(true); + MutableSegmentImpl mutableSegment = (MutableSegmentImpl)realtimeSegmentField.get(segmentDataManager); + + Field numOfColValuesLimitBreachedField = MutableSegmentImpl.class.getDeclaredField("_numOfColValuesLimitBreached"); + numOfColValuesLimitBreachedField.setAccessible(true); + numOfColValuesLimitBreachedField.set(mutableSegment, true); + + Assert.assertTrue(segmentDataManager.invokeEndCriteriaReached()); + Assert.assertEquals(segmentDataManager.getStopReason(), SegmentCompletionProtocol.REASON_NUM_OF_COL_VALUES_ABOVE_THRESHOLD); + } } private void setHasMessagesFetched(FakeRealtimeSegmentDataManager segmentDataManager, boolean hasMessagesFetched) @@ -804,7 +824,7 @@ public Long get() { } }; try (FakeRealtimeSegmentDataManager segmentDataManager = createFakeSegmentManager(true, timeSupplier, - String.valueOf(FakeStreamConfigUtils.SEGMENT_FLUSH_THRESHOLD_ROWS * 2), segmentTimeThresholdMins + "m", null)) { + String.valueOf(FakeStreamConfigUtils.SEGMENT_FLUSH_THRESHOLD_ROWS * 2), segmentTimeThresholdMins + "m", null, false)) { segmentDataManager._stubConsumeLoop = false; segmentDataManager._state.set(segmentDataManager, RealtimeSegmentDataManager.State.INITIAL_CONSUMING); @@ -839,7 +859,7 @@ public void testShouldNotSkipUnfilteredMessagesIfNotIndexedAndRowCountThresholdI final int segmentTimeThresholdMins = 10; TimeSupplier timeSupplier = new TimeSupplier(); try (FakeRealtimeSegmentDataManager segmentDataManager = createFakeSegmentManager(true, timeSupplier, - String.valueOf(FakeStreamConfigUtils.SEGMENT_FLUSH_THRESHOLD_ROWS), segmentTimeThresholdMins + "m", null)) { + String.valueOf(FakeStreamConfigUtils.SEGMENT_FLUSH_THRESHOLD_ROWS), segmentTimeThresholdMins + "m", null, false)) { segmentDataManager._stubConsumeLoop = false; segmentDataManager._state.set(segmentDataManager, RealtimeSegmentDataManager.State.INITIAL_CONSUMING); diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java index f29d78924f61..ddc0af0daad8 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java @@ -34,7 +34,7 @@ public class MutableSegmentEntriesAboveThresholdTest { private static final String AVRO_FILE = "data/test_data-mv.avro"; private Schema _schema; - private class FakeMutableForwardIndex implements MutableForwardIndex { + private static class FakeMutableForwardIndex implements MutableForwardIndex { private final MutableForwardIndex _mutableForwardIndex; private static final int threshold = 2; @@ -58,12 +58,12 @@ public void setDictIdMV(int docId, int[] dictIds) { @Override public int getLengthOfShortestElement() { - return 0; + return _mutableForwardIndex.getLengthOfShortestElement(); } @Override public int getLengthOfLongestElement() { - return 0; + return _mutableForwardIndex.getLengthOfLongestElement(); } @Override @@ -73,36 +73,37 @@ public void setDictId(int docId, int dictId) { @Override public boolean isDictionaryEncoded() { - return false; + return _mutableForwardIndex.isDictionaryEncoded(); } @Override public boolean isSingleValue() { - return false; + return _mutableForwardIndex.isSingleValue(); } @Override public FieldSpec.DataType getStoredType() { - return null; + return _mutableForwardIndex.getStoredType(); } @Override public void close() throws IOException { - + _mutableForwardIndex.close(); } } - @Test - public void testNoLimitBreached() + private File getAvroFile() { + URL resourceUrl = MutableSegmentImplTest.class.getClassLoader().getResource(AVRO_FILE); + Assert.assertNotNull(resourceUrl); + return new File(resourceUrl.getFile()); + } + + private MutableSegmentImpl getMutableSegment(File avroFile) throws Exception { FileUtils.deleteQuietly(TEMP_DIR); MutableSegmentImpl _mutableSegmentImpl; - URL resourceUrl = MutableSegmentImplTest.class.getClassLoader().getResource(AVRO_FILE); - Assert.assertNotNull(resourceUrl); - File avroFile = new File(resourceUrl.getFile()); - SegmentGeneratorConfig config = SegmentTestUtils.getSegmentGeneratorConfigWithoutTimeColumn(avroFile, TEMP_DIR, "testTable"); SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl(); @@ -115,48 +116,34 @@ public void testNoLimitBreached() .createMutableSegmentImpl(_schema, Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), Collections.emptyMap(), false, false, null, null, null, null, null, null, Collections.emptyList(), true); - var _lastIngestionTimeMs = System.currentTimeMillis(); - StreamMessageMetadata defaultMetadata = new StreamMessageMetadata(_lastIngestionTimeMs, new GenericRow()); + return _mutableSegmentImpl; + } + @Test + public void testNoLimitBreached() + throws Exception { + File avroFile = getAvroFile(); + MutableSegmentImpl mutableSegment = getMutableSegment(avroFile); + StreamMessageMetadata defaultMetadata = new StreamMessageMetadata(System.currentTimeMillis(), new GenericRow()); try (RecordReader recordReader = RecordReaderFactory .getRecordReader(FileFormat.AVRO, avroFile, _schema.getColumnNames(), null)) { GenericRow reuse = new GenericRow(); while (recordReader.hasNext()) { - _mutableSegmentImpl.index(recordReader.next(reuse), defaultMetadata); + mutableSegment.index(recordReader.next(reuse), defaultMetadata); } } - - assert !_mutableSegmentImpl.isNumOfColValuesAboveThreshold(); + assert !mutableSegment.isNumOfColValuesAboveThreshold(); } @Test - public void testLimitBreached1() + public void testLimitBreached() throws Exception { - FileUtils.deleteQuietly(TEMP_DIR); - MutableSegmentImpl _mutableSegmentImpl; - - URL resourceUrl = MutableSegmentImplTest.class.getClassLoader().getResource(AVRO_FILE); - Assert.assertNotNull(resourceUrl); - File avroFile = new File(resourceUrl.getFile()); - - SegmentGeneratorConfig config = - SegmentTestUtils.getSegmentGeneratorConfigWithoutTimeColumn(avroFile, TEMP_DIR, "testTable"); - SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl(); - driver.init(config); - driver.build(); - - _schema = config.getSchema(); - VirtualColumnProviderFactory.addBuiltInVirtualColumnsToSegmentSchema(_schema, "testSegment"); - _mutableSegmentImpl = MutableSegmentImplTestUtils - .createMutableSegmentImpl(_schema, Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), - Collections.emptyMap(), - false, false, null, null, null, null, null, null, Collections.emptyList(), true); - var _lastIngestionTimeMs = System.currentTimeMillis(); - StreamMessageMetadata defaultMetadata = new StreamMessageMetadata(_lastIngestionTimeMs, new GenericRow()); + File avroFile = getAvroFile(); + MutableSegmentImpl mutableSegment = getMutableSegment(avroFile); Field indexContainerMapField = MutableSegmentImpl.class.getDeclaredField("_indexContainerMap"); indexContainerMapField.setAccessible(true); - Map colVsIndexContainer = (Map) indexContainerMapField.get(_mutableSegmentImpl); + Map colVsIndexContainer = (Map) indexContainerMapField.get(mutableSegment); for (Map.Entry entry : colVsIndexContainer.entrySet()) { Object indexContainer = entry.getValue(); @@ -177,15 +164,15 @@ public void testLimitBreached1() indexTypeVsMutableIndex.put(new ForwardIndexPlugin().getIndexType(), new FakeMutableForwardIndex(mutableForwardIndex)); } - + StreamMessageMetadata defaultMetadata = new StreamMessageMetadata(System.currentTimeMillis(), new GenericRow()); try (RecordReader recordReader = RecordReaderFactory .getRecordReader(FileFormat.AVRO, avroFile, _schema.getColumnNames(), null)) { GenericRow reuse = new GenericRow(); while (recordReader.hasNext()) { - _mutableSegmentImpl.index(recordReader.next(reuse), defaultMetadata); + mutableSegment.index(recordReader.next(reuse), defaultMetadata); } } - assert _mutableSegmentImpl.isNumOfColValuesAboveThreshold(); + assert mutableSegment.isNumOfColValuesAboveThreshold(); } } From 27c444306dcc0eeab9a3b766ff85fc526d15b9eb Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Sat, 23 Nov 2024 22:13:26 +0530 Subject: [PATCH 19/47] lint --- .../RealtimeSegmentDataManagerTest.java | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java index 57fab70a6297..fe34733511e8 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java @@ -641,20 +641,24 @@ public void testEndCriteriaChecking() Assert.assertTrue(segmentDataManager.invokeEndCriteriaReached()); } - try (FakeRealtimeSegmentDataManager segmentDataManager = createFakeSegmentManager(false, new TimeSupplier(), null, null, null, true)) { + // test end criteria reached if any of the index cannot take more rows + try (FakeRealtimeSegmentDataManager segmentDataManager = createFakeSegmentManager(false, new TimeSupplier(), null, + null, null, true)) { segmentDataManager._state.set(segmentDataManager, RealtimeSegmentDataManager.State.INITIAL_CONSUMING); Assert.assertFalse(segmentDataManager.invokeEndCriteriaReached()); Field realtimeSegmentField = RealtimeSegmentDataManager.class.getDeclaredField("_realtimeSegment"); realtimeSegmentField.setAccessible(true); - MutableSegmentImpl mutableSegment = (MutableSegmentImpl)realtimeSegmentField.get(segmentDataManager); + MutableSegmentImpl mutableSegment = (MutableSegmentImpl) realtimeSegmentField.get(segmentDataManager); - Field numOfColValuesLimitBreachedField = MutableSegmentImpl.class.getDeclaredField("_numOfColValuesLimitBreached"); + Field numOfColValuesLimitBreachedField = + MutableSegmentImpl.class.getDeclaredField("_numOfColValuesLimitBreached"); numOfColValuesLimitBreachedField.setAccessible(true); numOfColValuesLimitBreachedField.set(mutableSegment, true); Assert.assertTrue(segmentDataManager.invokeEndCriteriaReached()); - Assert.assertEquals(segmentDataManager.getStopReason(), SegmentCompletionProtocol.REASON_NUM_OF_COL_VALUES_ABOVE_THRESHOLD); + Assert.assertEquals(segmentDataManager.getStopReason(), + SegmentCompletionProtocol.REASON_NUM_OF_COL_VALUES_ABOVE_THRESHOLD); } } @@ -824,7 +828,8 @@ public Long get() { } }; try (FakeRealtimeSegmentDataManager segmentDataManager = createFakeSegmentManager(true, timeSupplier, - String.valueOf(FakeStreamConfigUtils.SEGMENT_FLUSH_THRESHOLD_ROWS * 2), segmentTimeThresholdMins + "m", null, false)) { + String.valueOf(FakeStreamConfigUtils.SEGMENT_FLUSH_THRESHOLD_ROWS * 2), segmentTimeThresholdMins + "m", null, + false)) { segmentDataManager._stubConsumeLoop = false; segmentDataManager._state.set(segmentDataManager, RealtimeSegmentDataManager.State.INITIAL_CONSUMING); @@ -859,7 +864,8 @@ public void testShouldNotSkipUnfilteredMessagesIfNotIndexedAndRowCountThresholdI final int segmentTimeThresholdMins = 10; TimeSupplier timeSupplier = new TimeSupplier(); try (FakeRealtimeSegmentDataManager segmentDataManager = createFakeSegmentManager(true, timeSupplier, - String.valueOf(FakeStreamConfigUtils.SEGMENT_FLUSH_THRESHOLD_ROWS), segmentTimeThresholdMins + "m", null, false)) { + String.valueOf(FakeStreamConfigUtils.SEGMENT_FLUSH_THRESHOLD_ROWS), segmentTimeThresholdMins + "m", null, + false)) { segmentDataManager._stubConsumeLoop = false; segmentDataManager._state.set(segmentDataManager, RealtimeSegmentDataManager.State.INITIAL_CONSUMING); From a2b95b66e9f271982a2c78e7aef6722a4afb18f6 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Sat, 23 Nov 2024 22:57:33 +0530 Subject: [PATCH 20/47] refactors config --- .../realtime/RealtimeSegmentDataManager.java | 10 +++++----- .../RealtimeSegmentDataManagerTest.java | 6 +++--- .../mutable/MutableSegmentImpl.java | 14 +++++++------- .../realtime/impl/RealtimeSegmentConfig.java | 18 +++++++++--------- ...utableSegmentEntriesAboveThresholdTest.java | 4 ++-- .../mutable/MutableSegmentImplTestUtils.java | 2 +- .../pinot/spi/config/table/IndexingConfig.java | 9 +++++++++ .../SegmentsValidationAndRetentionConfig.java | 9 --------- .../spi/utils/builder/TableConfigBuilder.java | 4 ++-- 9 files changed, 38 insertions(+), 38 deletions(-) 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 b0fd1ca2ba15..b045e5ce0de0 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 @@ -309,7 +309,7 @@ public void deleteSegmentFile() { private String _stopReason = null; private final Semaphore _segBuildSemaphore; private final boolean _isOffHeap; - private final boolean _thresholdForNumOfColValuesEnabled; + private final boolean _indexCapacityThresholdCheckEnabled; /** * Whether null handling is enabled by default. This value is only used if * {@link Schema#isEnableColumnBasedNullHandling()} is false. @@ -363,7 +363,7 @@ private boolean endCriteriaReached() { _numRowsConsumed, _numRowsIndexed); _stopReason = SegmentCompletionProtocol.REASON_FORCE_COMMIT_MESSAGE_RECEIVED; return true; - } else if (_thresholdForNumOfColValuesEnabled && _realtimeSegment.isNumOfColValuesAboveThreshold()) { + } else if (_indexCapacityThresholdCheckEnabled && _realtimeSegment.isIndexCapacityThresholdBreached()) { _segmentLogger.info( "Stopping consumption as num of values for a column is above threshold - numRowsConsumed={} " + "numRowsIndexed={}", @@ -1538,7 +1538,7 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf _isOffHeap = indexLoadingConfig.isRealtimeOffHeapAllocation(); _defaultNullHandlingEnabled = indexingConfig.isNullHandlingEnabled(); - _thresholdForNumOfColValuesEnabled = tableConfig.getValidationConfig().isThresholdForNumOfColValuesEnabled(); + _indexCapacityThresholdCheckEnabled = tableConfig.getIndexingConfig().isIndexCapacityThresholdCheckEnabled(); // Start new realtime segment String consumerDir = realtimeTableDataManager.getConsumerDir(); @@ -1563,7 +1563,7 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf .setPartitionDedupMetadataManager(partitionDedupMetadataManager) .setDedupTimeColumn(tableConfig.getDedupTimeColumn()) .setFieldConfigList(tableConfig.getFieldConfigList()) - .setThresholdForNumOfColValuesEnabled(_thresholdForNumOfColValuesEnabled); + .setIndexCapacityThresholdCheckEnabled(_indexCapacityThresholdCheckEnabled); // Create message decoder Set fieldsToRead = IngestionUtils.getFieldsForRecordExtractor(_tableConfig.getIngestionConfig(), _schema); @@ -1636,7 +1636,7 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf "Failed to initialize segment data manager", e)); _segmentLogger.warn( "Scheduling task to call controller to mark the segment as OFFLINE in Ideal State due" - + " to initialization error: '{}'", + + " to initialization error: '{}'", e.getMessage()); // Since we are going to throw exception from this thread (helix execution thread), the externalview // entry for this segment will be ERROR. We allow time for Helix to make this transition, and then diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java index fe34733511e8..6cf39a697261 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java @@ -131,7 +131,7 @@ private FakeRealtimeSegmentDataManager createFakeSegmentManager() private FakeRealtimeSegmentDataManager createFakeSegmentManager(boolean noUpsert, TimeSupplier timeSupplier, @Nullable String maxRows, @Nullable String maxDuration, @Nullable TableConfig tableConfig, - boolean setThresholdForNumOfColValuesEnabled) + boolean setIndexCapacityThresholdCheck) throws Exception { SegmentZKMetadata segmentZKMetadata = createZkMetadata(); if (tableConfig == null) { @@ -148,7 +148,7 @@ private FakeRealtimeSegmentDataManager createFakeSegmentManager(boolean noUpsert tableConfig.getIndexingConfig().getStreamConfigs() .put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_TIME, maxDuration); } - tableConfig.getValidationConfig().setThresholdForNumOfColValuesEnabled(setThresholdForNumOfColValuesEnabled); + tableConfig.getIndexingConfig().setIndexCapacityThresholdCheckEnabled(setIndexCapacityThresholdCheck); RealtimeTableDataManager tableDataManager = createTableDataManager(tableConfig); LLCSegmentName llcSegmentName = new LLCSegmentName(SEGMENT_NAME_STR); _partitionGroupIdToSemaphoreMap.putIfAbsent(PARTITION_GROUP_ID, new Semaphore(1)); @@ -652,7 +652,7 @@ public void testEndCriteriaChecking() MutableSegmentImpl mutableSegment = (MutableSegmentImpl) realtimeSegmentField.get(segmentDataManager); Field numOfColValuesLimitBreachedField = - MutableSegmentImpl.class.getDeclaredField("_numOfColValuesLimitBreached"); + MutableSegmentImpl.class.getDeclaredField("_indexCapacityThresholdBreached"); numOfColValuesLimitBreachedField.setAccessible(true); numOfColValuesLimitBreachedField.set(mutableSegment, true); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index eb43e488f15b..f40889175bdf 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -147,10 +147,10 @@ public class MutableSegmentImpl implements MutableSegment { private final int _mainPartitionId; // partition id designated for this consuming segment private final boolean _defaultNullHandlingEnabled; private final File _consumerDir; - private final boolean _thresholdForNumOfColValuesEnabled; + private final boolean _indexCapacityThresholdCheckEnabled; private final Map _indexContainerMap = new HashMap<>(); - private boolean _numOfColValuesLimitBreached = false; + private boolean _indexCapacityThresholdBreached = false; private final IdMap _recordIdMap; @@ -227,7 +227,7 @@ public boolean isMutableSegment() { _mainPartitionId = config.getPartitionId(); _defaultNullHandlingEnabled = config.isNullHandlingEnabled(); _consumerDir = new File(config.getConsumerDir()); - _thresholdForNumOfColValuesEnabled = config.isThresholdForNumOfColValuesEnabled(); + _indexCapacityThresholdCheckEnabled = config.isIndexCapacityThresholdCheckEnabled(); Collection allFieldSpecs = _schema.getAllFieldSpecs(); List physicalFieldSpecs = new ArrayList<>(allFieldSpecs.size()); @@ -797,12 +797,12 @@ private void addNewRow(int docId, GenericRow row) { try { MutableIndex mutableIndex = indexEntry.getValue(); mutableIndex.add(values, dictIds, docId); - if (_thresholdForNumOfColValuesEnabled && !mutableIndex.canAddMore()) { + if (_indexCapacityThresholdCheckEnabled && !mutableIndex.canAddMore()) { _logger.warn( "failed to index value with {} for column {} due to num of col value threshold limit", indexEntry.getKey(), column ); - _numOfColValuesLimitBreached = true; + _indexCapacityThresholdBreached = true; } } catch (Exception e) { recordIndexingError(indexEntry.getKey(), e); @@ -1240,8 +1240,8 @@ private boolean isAggregateMetricsEnabled() { return _recordIdMap != null; } - public boolean isNumOfColValuesAboveThreshold() { - return _numOfColValuesLimitBreached; + public boolean isIndexCapacityThresholdBreached() { + return _indexCapacityThresholdBreached; } // NOTE: Okay for single-writer diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java index c1401b9b4238..2e4a4d51fe6d 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java @@ -73,7 +73,7 @@ public class RealtimeSegmentConfig { private final String _consumerDir; private final List _fieldConfigList; private final List _ingestionAggregationConfigs; - private final boolean _thresholdForNumOfColValuesEnabled; + private final boolean _indexCapacityThresholdCheckEnabled; // TODO: Clean up this constructor. Most of these things can be extracted from tableConfig. @@ -91,7 +91,7 @@ private RealtimeSegmentConfig(String tableNameWithType, String segmentName, Stri boolean upsertDropOutOfOrderRecord, PartitionUpsertMetadataManager partitionUpsertMetadataManager, String dedupTimeColumn, PartitionDedupMetadataManager partitionDedupMetadataManager, List fieldConfigList, List ingestionAggregationConfigs, - boolean enableThresholdForNumOfValues) { + boolean indexCapacityThresholdCheckEnabled) { _tableNameWithType = tableNameWithType; _segmentName = segmentName; _streamName = streamName; @@ -121,7 +121,7 @@ private RealtimeSegmentConfig(String tableNameWithType, String segmentName, Stri _partitionDedupMetadataManager = partitionDedupMetadataManager; _fieldConfigList = fieldConfigList; _ingestionAggregationConfigs = ingestionAggregationConfigs; - _thresholdForNumOfColValuesEnabled = enableThresholdForNumOfValues; + _indexCapacityThresholdCheckEnabled = indexCapacityThresholdCheckEnabled; } public String getTableNameWithType() { @@ -244,8 +244,8 @@ public List getIngestionAggregationConfigs() { return _ingestionAggregationConfigs; } - public boolean isThresholdForNumOfColValuesEnabled() { - return _thresholdForNumOfColValuesEnabled; + public boolean isIndexCapacityThresholdCheckEnabled() { + return _indexCapacityThresholdCheckEnabled; } public static class Builder { @@ -282,7 +282,7 @@ public static class Builder { private PartitionDedupMetadataManager _partitionDedupMetadataManager; private List _fieldConfigList; private List _ingestionAggregationConfigs; - private boolean _thresholdForNumOfColValuesEnabled = false; + private boolean _indexCapacityThresholdCheckEnabled = false; public Builder() { _indexConfigByCol = new HashMap<>(); @@ -483,8 +483,8 @@ public Builder setIngestionAggregationConfigs(List ingestionA return this; } - public Builder setThresholdForNumOfColValuesEnabled(boolean thresholdForNumOfColValuesEnabled) { - _thresholdForNumOfColValuesEnabled = thresholdForNumOfColValuesEnabled; + public Builder setIndexCapacityThresholdCheckEnabled(boolean indexCapacityThresholdCheckEnabled) { + _indexCapacityThresholdCheckEnabled = indexCapacityThresholdCheckEnabled; return this; } @@ -500,7 +500,7 @@ public RealtimeSegmentConfig build() { _defaultNullHandlingEnabled, _consumerDir, _upsertMode, _upsertConsistencyMode, _upsertComparisonColumns, _upsertDeleteRecordColumn, _upsertOutOfOrderRecordColumn, _upsertDropOutOfOrderRecord, _partitionUpsertMetadataManager, _dedupTimeColumn, _partitionDedupMetadataManager, _fieldConfigList, - _ingestionAggregationConfigs, _thresholdForNumOfColValuesEnabled); + _ingestionAggregationConfigs, _indexCapacityThresholdCheckEnabled); } } } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java index ddc0af0daad8..dccc4ca20f60 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java @@ -132,7 +132,7 @@ public void testNoLimitBreached() mutableSegment.index(recordReader.next(reuse), defaultMetadata); } } - assert !mutableSegment.isNumOfColValuesAboveThreshold(); + assert !mutableSegment.isIndexCapacityThresholdBreached(); } @Test @@ -173,6 +173,6 @@ public void testLimitBreached() } } - assert mutableSegment.isNumOfColValuesAboveThreshold(); + assert mutableSegment.isIndexCapacityThresholdBreached(); } } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java index d71e63d018a5..ee58ce88a311 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java @@ -127,7 +127,7 @@ public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set entry : jsonIndexConfigs.entrySet()) { segmentConfBuilder.setIndex(entry.getKey(), StandardIndexes.json(), entry.getValue()); } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java index 4b86dfa9b3d2..2a964a93a43d 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java @@ -66,6 +66,7 @@ public class IndexingConfig extends BaseJsonConfig { private boolean _aggregateMetrics; private boolean _nullHandlingEnabled; private boolean _columnMajorSegmentBuilderEnabled = true; + private boolean _indexCapacityThresholdCheckEnabled = false; /** * If `optimizeDictionary` enabled, dictionary is not created for the high-cardinality @@ -396,6 +397,14 @@ public void setSegmentNameGeneratorType(String segmentNameGeneratorType) { _segmentNameGeneratorType = segmentNameGeneratorType; } + public boolean isIndexCapacityThresholdCheckEnabled() { + return _indexCapacityThresholdCheckEnabled; + } + + public void setIndexCapacityThresholdCheckEnabled(boolean indexCapacityThresholdCheckEnabled) { + _indexCapacityThresholdCheckEnabled = indexCapacityThresholdCheckEnabled; + } + /** * Returns all columns referenced in the indexing config. This is useful to construct FieldIndexConfigs in * IndexLoadingConfig when schema is not provided. Only including the columns referenced by indexes supported in diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentsValidationAndRetentionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentsValidationAndRetentionConfig.java index e410d69b9703..0b8a403041ab 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentsValidationAndRetentionConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentsValidationAndRetentionConfig.java @@ -51,7 +51,6 @@ public class SegmentsValidationAndRetentionConfig extends BaseJsonConfig { // using the specified download scheme. Both realtime tables and offline tables can set this field. // For more usage of this field, please refer to this design doc: https://tinyurl.com/f63ru4sb private String _peerSegmentDownloadScheme; - private boolean _thresholdForNumOfColValuesEnabled; @Deprecated public String getSegmentAssignmentStrategy() { @@ -219,14 +218,6 @@ public void setPeerSegmentDownloadScheme(String peerSegmentDownloadScheme) { _peerSegmentDownloadScheme = peerSegmentDownloadScheme; } - public boolean isThresholdForNumOfColValuesEnabled() { - return _thresholdForNumOfColValuesEnabled; - } - - public void setThresholdForNumOfColValuesEnabled(boolean thresholdForNumOfColValuesEnabled) { - _thresholdForNumOfColValuesEnabled = thresholdForNumOfColValuesEnabled; - } - public String getCrypterClassName() { return _crypterClassName; } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java index 0e5a996450d7..f982a5223735 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java @@ -81,7 +81,6 @@ public class TableConfigBuilder { private ReplicaGroupStrategyConfig _replicaGroupStrategyConfig; private CompletionConfig _completionConfig; private String _crypterClassName; - private boolean _thresholdForNumOfColValuesEnabled = false; // Tenant config related private String _brokerTenant; @@ -110,6 +109,7 @@ public class TableConfigBuilder { // This threshold determines if dictionary should be enabled or not for a metric column and is relevant // only when _optimizeDictionaryForMetrics is set to true. private double _noDictionarySizeRatioThreshold; + private boolean _indexCapacityThresholdCheckEnabled = false; private TableCustomConfig _customConfig; private QuotaConfig _quotaConfig; @@ -440,7 +440,6 @@ public TableConfig build() { validationConfig.setReplication(_numReplicas); validationConfig.setPeerSegmentDownloadScheme(_peerSegmentDownloadScheme); validationConfig.setCrypterClassName(_crypterClassName); - validationConfig.setThresholdForNumOfColValuesEnabled(_thresholdForNumOfColValuesEnabled); // Tenant config TenantConfig tenantConfig = new TenantConfig(_brokerTenant, _serverTenant, _tagOverrideConfig); @@ -469,6 +468,7 @@ public TableConfig build() { indexingConfig.setOptimizeDictionaryForMetrics(_optimizeDictionaryForMetrics); indexingConfig.setNoDictionarySizeRatioThreshold(_noDictionarySizeRatioThreshold); indexingConfig.setTierOverwrites(_tierOverwrites); + indexingConfig.setIndexCapacityThresholdCheckEnabled(_indexCapacityThresholdCheckEnabled); if (_customConfig == null) { _customConfig = new TableCustomConfig(null); From c8f5ed2b1f60e00324005ed9d8675a7c795a1a39 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Sat, 23 Nov 2024 23:07:27 +0530 Subject: [PATCH 21/47] changes log --- .../core/data/manager/realtime/RealtimeSegmentDataManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 b493aeb59657..6e09de417747 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 @@ -366,7 +366,7 @@ private boolean endCriteriaReached() { return true; } else if (_indexCapacityThresholdCheckEnabled && _realtimeSegment.isIndexCapacityThresholdBreached()) { _segmentLogger.info( - "Stopping consumption as num of values for a column is above threshold - numRowsConsumed={} " + "Stopping consumption as mutable index cannot consume more rows - numRowsConsumed={} " + "numRowsIndexed={}", _numRowsConsumed, _numRowsIndexed); _stopReason = SegmentCompletionProtocol.REASON_NUM_OF_COL_VALUES_ABOVE_THRESHOLD; From ba598308b66d6d1695d8567374cafcbbc0b1a67e Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Sat, 23 Nov 2024 23:08:16 +0530 Subject: [PATCH 22/47] nit --- .../core/data/manager/realtime/RealtimeSegmentDataManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 6e09de417747..06c081338090 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 @@ -1652,7 +1652,7 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf "Failed to initialize segment data manager", e)); _segmentLogger.warn( "Scheduling task to call controller to mark the segment as OFFLINE in Ideal State due" - + " to initialization error: '{}'", + + " to initialization error: '{}'", e.getMessage()); // Since we are going to throw exception from this thread (helix execution thread), the externalview // entry for this segment will be ERROR. We allow time for Helix to make this transition, and then From 9f13ff133ad517fcac5ebab5c4e3fc13a40fe1c0 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Sat, 23 Nov 2024 23:09:04 +0530 Subject: [PATCH 23/47] nit --- .../segment/local/indexsegment/mutable/MutableSegmentImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index 7cbfebf6d9e4..ceaea1236cc1 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -151,7 +151,7 @@ public class MutableSegmentImpl implements MutableSegment { private final boolean _indexCapacityThresholdCheckEnabled; private final Map _indexContainerMap = new HashMap<>(); - private boolean _indexCapacityThresholdBreached = false; + private boolean _indexCapacityThresholdBreached; private final IdMap _recordIdMap; From b1143d5393c1bac34912db5a435720d3c670e265 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Mon, 25 Nov 2024 17:26:54 +0530 Subject: [PATCH 24/47] Adds test and minor refactoring --- .../protocols/SegmentCompletionProtocol.java | 4 +- .../realtime/RealtimeSegmentDataManager.java | 2 +- .../RealtimeSegmentDataManagerTest.java | 3 +- .../mutable/MutableSegmentImpl.java | 11 +++-- .../FixedByteMVMutableForwardIndex.java | 3 +- .../FixedByteMVMutableForwardIndexTest.java | 43 +++++++++++++++++++ .../spi/index/mutable/MutableIndex.java | 5 ++- 7 files changed, 62 insertions(+), 9 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/protocols/SegmentCompletionProtocol.java b/pinot-common/src/main/java/org/apache/pinot/common/protocols/SegmentCompletionProtocol.java index ade16eb63de1..f1347f08f209 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/protocols/SegmentCompletionProtocol.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/protocols/SegmentCompletionProtocol.java @@ -149,7 +149,9 @@ public enum ControllerResponseStatus { public static final String REASON_END_OF_PARTITION_GROUP = "endOfPartitionGroup"; // Stop reason sent by server as force commit message received public static final String REASON_FORCE_COMMIT_MESSAGE_RECEIVED = "forceCommitMessageReceived"; - public static final String REASON_NUM_OF_COL_VALUES_ABOVE_THRESHOLD = "numColValuesAboveThreshold"; + // Stop reason sent by server as mutable index cannot consume more rows + // (like size reaching close to its limit or number of col values for a col is about to overflow int max) + public static final String REASON_INDEX_CAPACITY_THRESHOLD_BREACHED = "indexCapacityThresholdBreached"; // Canned responses public static final Response RESP_NOT_LEADER = 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 06c081338090..3b791994144a 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 @@ -369,7 +369,7 @@ private boolean endCriteriaReached() { "Stopping consumption as mutable index cannot consume more rows - numRowsConsumed={} " + "numRowsIndexed={}", _numRowsConsumed, _numRowsIndexed); - _stopReason = SegmentCompletionProtocol.REASON_NUM_OF_COL_VALUES_ABOVE_THRESHOLD; + _stopReason = SegmentCompletionProtocol.REASON_INDEX_CAPACITY_THRESHOLD_BREACHED; return true; } return false; diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java index fdbab333ae70..c5fb96ba6b4a 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java @@ -54,7 +54,6 @@ import org.apache.pinot.segment.local.segment.creator.Fixtures; import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.segment.local.utils.SegmentLocks; -import org.apache.pinot.segment.spi.MutableSegment; import org.apache.pinot.spi.config.instance.InstanceDataManagerConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.data.Schema; @@ -659,7 +658,7 @@ public void testEndCriteriaChecking() Assert.assertTrue(segmentDataManager.invokeEndCriteriaReached()); Assert.assertEquals(segmentDataManager.getStopReason(), - SegmentCompletionProtocol.REASON_NUM_OF_COL_VALUES_ABOVE_THRESHOLD); + SegmentCompletionProtocol.REASON_INDEX_CAPACITY_THRESHOLD_BREACHED); } } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index ceaea1236cc1..81ac26407dc8 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -661,7 +661,8 @@ private Comparable getComparisonValue(GenericRow row) { * @throws UnsupportedOperationException if the length of an MV column would exceed the * capacity of a chunk in the ForwardIndex */ - private void validateLengthOfMVColumns(GenericRow row) throws UnsupportedOperationException { + private void validateLengthOfMVColumns(GenericRow row) + throws UnsupportedOperationException { for (Map.Entry entry : _indexContainerMap.entrySet()) { IndexContainer indexContainer = entry.getValue(); FieldSpec fieldSpec = indexContainer._fieldSpec; @@ -683,7 +684,6 @@ private void validateLengthOfMVColumns(GenericRow row) throws UnsupportedOperati } } - private void updateDictionary(GenericRow row) { for (Map.Entry entry : _indexContainerMap.entrySet()) { IndexContainer indexContainer = entry.getValue(); @@ -833,9 +833,14 @@ private void addNewRow(int docId, GenericRow row) { try { MutableIndex mutableIndex = indexEntry.getValue(); mutableIndex.add(values, dictIds, docId); + // Few of the Immutable version of the mutable index are bounded by size like FixedBitMVForwardIndex. + // If num of values overflows or size is above limit, A mutable index is unable to convert to + // an immutable index and segment build fails causing the realtime consumption to stop. + // Hence, The below check is a temporary measure to avoid such scenarios until immutable index + // implementations are changed. if (_indexCapacityThresholdCheckEnabled && !mutableIndex.canAddMore()) { _logger.warn( - "failed to index value with {} for column {} due to num of col value threshold limit", + "failed to index value with indexType {} for column {} as index cannot consume more rows", indexEntry.getKey(), column ); _indexCapacityThresholdBreached = true; diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/FixedByteMVMutableForwardIndex.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/FixedByteMVMutableForwardIndex.java index 07afdb50cd28..b1826e08a82e 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/FixedByteMVMutableForwardIndex.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/FixedByteMVMutableForwardIndex.java @@ -104,7 +104,8 @@ public class FixedByteMVMutableForwardIndex implements MutableForwardIndex { private static final int INCREMENT_PERCENTAGE = 100; //Increments the Initial size by 100% of initial capacity every time we runs out of capacity - private final static int DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN = 400_000_000; + // Conservative figure to not breach 2GB size limit for immutable index + private final static int DEFAULT_THRESHOLD_FOR_NUM_OF_VALUES_PER_COLUMN = 450_000_000; // For single writer multiple readers setup, use ArrayList for writer and CopyOnWriteArrayList for reader private final List _headerWriters = new ArrayList<>(); diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/FixedByteMVMutableForwardIndexTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/FixedByteMVMutableForwardIndexTest.java index 326827756257..78d48e534539 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/FixedByteMVMutableForwardIndexTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/FixedByteMVMutableForwardIndexTest.java @@ -19,8 +19,11 @@ package org.apache.pinot.segment.local.segment.index.forward.mutable; import java.io.IOException; +import java.lang.reflect.Field; import java.util.Arrays; +import java.util.Map; import java.util.Random; +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.impl.forward.FixedByteMVMutableForwardIndex; import org.apache.pinot.segment.spi.memory.PinotDataBufferMemoryManager; @@ -78,6 +81,7 @@ public void testIntArray(final long seed, boolean isDictionaryEncoded) readerWriter = new FixedByteMVMutableForwardIndex(maxNumberOfMultiValuesPerRow, 2, rows / 2, columnSizeInBytes, _memoryManager, "IntArray", isDictionaryEncoded, FieldSpec.DataType.INT); + int valuesAdded = 0; Random r = new Random(seed); int[][] data = new int[rows][]; @@ -87,6 +91,7 @@ public void testIntArray(final long seed, boolean isDictionaryEncoded) data[i][j] = r.nextInt(); } readerWriter.setIntMV(i, data[i]); + valuesAdded += data[i].length; } int[] ret = new int[maxNumberOfMultiValuesPerRow]; for (int i = 0; i < rows; i++) { @@ -94,6 +99,7 @@ public void testIntArray(final long seed, boolean isDictionaryEncoded) Assert.assertEquals(data[i].length, length, "Failed with seed=" + seed); Assert.assertTrue(Arrays.equals(data[i], Arrays.copyOf(ret, length)), "Failed with seed=" + seed); } + validateNumOfValues(readerWriter, valuesAdded); readerWriter.close(); } @@ -106,6 +112,7 @@ public void testIntArrayFixedSize(int multiValuesPerRow, long seed, boolean isDi // transition to new ones readerWriter = new FixedByteMVMutableForwardIndex(multiValuesPerRow, multiValuesPerRow, multiValuesPerRow * 2, columnSizeInBytes, _memoryManager, "IntArrayFixedSize", isDictionaryEncoded, FieldSpec.DataType.INT); + int valuesAdded = 0; Random r = new Random(seed); int[][] data = new int[rows][]; @@ -115,6 +122,7 @@ public void testIntArrayFixedSize(int multiValuesPerRow, long seed, boolean isDi data[i][j] = r.nextInt(); } readerWriter.setIntMV(i, data[i]); + valuesAdded += data[i].length; } int[] ret = new int[multiValuesPerRow]; for (int i = 0; i < rows; i++) { @@ -122,6 +130,7 @@ public void testIntArrayFixedSize(int multiValuesPerRow, long seed, boolean isDi Assert.assertEquals(data[i].length, length, "Failed with seed=" + seed); Assert.assertTrue(Arrays.equals(data[i], Arrays.copyOf(ret, length)), "Failed with seed=" + seed); } + validateNumOfValues(readerWriter, valuesAdded); readerWriter.close(); } @@ -135,6 +144,7 @@ public void testWithZeroSize(long seed, boolean isDictionaryEncoded) readerWriter = new FixedByteMVMutableForwardIndex(maxNumberOfMultiValuesPerRow, 3, r.nextInt(rows) + 1, columnSizeInBytes, _memoryManager, "ZeroSize", isDictionaryEncoded, FieldSpec.DataType.INT); + int valuesAdded = 0; int[][] data = new int[rows][]; for (int i = 0; i < rows; i++) { @@ -144,9 +154,11 @@ public void testWithZeroSize(long seed, boolean isDictionaryEncoded) data[i][j] = r.nextInt(); } readerWriter.setIntMV(i, data[i]); + valuesAdded += data[i].length; } else { data[i] = new int[0]; readerWriter.setIntMV(i, data[i]); + valuesAdded += data[i].length; } } int[] ret = new int[maxNumberOfMultiValuesPerRow]; @@ -155,6 +167,7 @@ public void testWithZeroSize(long seed, boolean isDictionaryEncoded) Assert.assertEquals(data[i].length, length, "Failed with seed=" + seed); Assert.assertTrue(Arrays.equals(data[i], Arrays.copyOf(ret, length)), "Failed with seed=" + seed); } + validateNumOfValues(readerWriter, valuesAdded); readerWriter.close(); } @@ -187,6 +200,7 @@ private void testLongArray(boolean isDictionaryEncoded) final int maxNumberOfMultiValuesPerRow = r.nextInt(100) + 1; FixedByteMVMutableForwardIndex readerWriter = createReaderWriter(FieldSpec.DataType.LONG, r, rows, maxNumberOfMultiValuesPerRow, isDictionaryEncoded); + int valuesAdded = 0; long[][] data = new long[rows][]; for (int i = 0; i < rows; i++) { @@ -196,9 +210,11 @@ private void testLongArray(boolean isDictionaryEncoded) data[i][j] = r.nextLong(); } readerWriter.setLongMV(i, data[i]); + valuesAdded += data[i].length; } else { data[i] = new long[0]; readerWriter.setLongMV(i, data[i]); + valuesAdded += data[i].length; } } long[] ret = new long[maxNumberOfMultiValuesPerRow]; @@ -207,6 +223,7 @@ private void testLongArray(boolean isDictionaryEncoded) Assert.assertEquals(data[i].length, length, "Failed with seed=" + seed); Assert.assertTrue(Arrays.equals(data[i], Arrays.copyOf(ret, length)), "Failed with seed=" + seed); } + validateNumOfValues(readerWriter, valuesAdded); readerWriter.close(); } @@ -225,6 +242,7 @@ private void testFloatArray(boolean isDictoinaryEncoded) final int maxNumberOfMultiValuesPerRow = r.nextInt(100) + 1; FixedByteMVMutableForwardIndex readerWriter = createReaderWriter(FieldSpec.DataType.FLOAT, r, rows, maxNumberOfMultiValuesPerRow, isDictoinaryEncoded); + int valuesAdded = 0; float[][] data = new float[rows][]; for (int i = 0; i < rows; i++) { @@ -234,9 +252,11 @@ private void testFloatArray(boolean isDictoinaryEncoded) data[i][j] = r.nextFloat(); } readerWriter.setFloatMV(i, data[i]); + valuesAdded += data[i].length; } else { data[i] = new float[0]; readerWriter.setFloatMV(i, data[i]); + valuesAdded += data[i].length; } } float[] ret = new float[maxNumberOfMultiValuesPerRow]; @@ -245,6 +265,7 @@ private void testFloatArray(boolean isDictoinaryEncoded) Assert.assertEquals(data[i].length, length, "Failed with seed=" + seed); Assert.assertTrue(Arrays.equals(data[i], Arrays.copyOf(ret, length)), "Failed with seed=" + seed); } + validateNumOfValues(readerWriter, valuesAdded); readerWriter.close(); } @@ -263,6 +284,7 @@ private void testDoubleArray(boolean isDictonaryEncoded) final int maxNumberOfMultiValuesPerRow = r.nextInt(100) + 1; FixedByteMVMutableForwardIndex readerWriter = createReaderWriter(FieldSpec.DataType.DOUBLE, r, rows, maxNumberOfMultiValuesPerRow, isDictonaryEncoded); + int valuesAdded = 0; double[][] data = new double[rows][]; for (int i = 0; i < rows; i++) { @@ -272,9 +294,11 @@ private void testDoubleArray(boolean isDictonaryEncoded) data[i][j] = r.nextDouble(); } readerWriter.setDoubleMV(i, data[i]); + valuesAdded += data[i].length; } else { data[i] = new double[0]; readerWriter.setDoubleMV(i, data[i]); + valuesAdded += data[i].length; } } double[] ret = new double[maxNumberOfMultiValuesPerRow]; @@ -283,6 +307,25 @@ private void testDoubleArray(boolean isDictonaryEncoded) Assert.assertEquals(data[i].length, length, "Failed with seed=" + seed); Assert.assertTrue(Arrays.equals(data[i], Arrays.copyOf(ret, length)), "Failed with seed=" + seed); } + validateNumOfValues(readerWriter, valuesAdded); readerWriter.close(); } + + private int getNumValues(FixedByteMVMutableForwardIndex readerWriter) + throws NoSuchFieldException, IllegalAccessException { + Field numValuesField = FixedByteMVMutableForwardIndex.class.getDeclaredField("_numValues"); + numValuesField.setAccessible(true); + return (int) numValuesField.get(readerWriter); + } + + private void validateNumOfValues(FixedByteMVMutableForwardIndex readerWriter, int valuesAdded) { + int numValuesPresentInIndex; + try { + numValuesPresentInIndex = getNumValues(readerWriter); + } catch (Exception e) { + throw new AssertionError("failed to validate the num of values added in the index"); + } + Assert.assertEquals(numValuesPresentInIndex, valuesAdded); + Assert.assertTrue(readerWriter.canAddMore()); + } } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableIndex.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableIndex.java index 83deb4df28c0..320b3f689521 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableIndex.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableIndex.java @@ -23,6 +23,7 @@ import javax.annotation.Nullable; import org.apache.pinot.segment.spi.index.IndexReader; + /** * Implementations of this interface can be used to create indexes in realtime tables and at the same time to read them. * @@ -69,8 +70,10 @@ public interface MutableIndex extends IndexReader { default void commit() { } + /** + * Returns a boolean denoting whether the mutable index can consume any more rows or not. + */ default boolean canAddMore() { return true; } - } From 33d40a5b8df7ca290a80ebb2cf95f0804198add7 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Mon, 25 Nov 2024 17:28:52 +0530 Subject: [PATCH 25/47] nit --- .../forward/mutable/FixedByteSVMutableForwardIndexTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/FixedByteSVMutableForwardIndexTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/FixedByteSVMutableForwardIndexTest.java index 1e0d82779367..cadea9a1fca7 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/FixedByteSVMutableForwardIndexTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/FixedByteSVMutableForwardIndexTest.java @@ -112,6 +112,7 @@ private void testDictId(final Random random, final int rows, final int div) for (int i = 0; i < 2 * rows; i++) { Assert.assertEquals(readerWriter.getDictId(start + i), 0); } + Assert.assertTrue(readerWriter.canAddMore()); readerWriter.close(); } From be8ab39590fc6d2264dc9ec1eac43e74b2ca1a45 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Mon, 25 Nov 2024 17:54:06 +0530 Subject: [PATCH 26/47] nit --- .../apache/pinot/spi/utils/builder/TableConfigBuilder.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java index 3a507c56bd45..1290cf7baef5 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java @@ -288,6 +288,11 @@ public TableConfigBuilder setNoDictionaryCardinalityRatioThreshold(double noDict return this; } + public TableConfigBuilder setIndexCapacityThresholdCheckEnabled(boolean indexCapacityThresholdCheckEnabled) { + _indexCapacityThresholdCheckEnabled = indexCapacityThresholdCheckEnabled; + return this; + } + public TableConfigBuilder setCreateInvertedIndexDuringSegmentGeneration( boolean createInvertedIndexDuringSegmentGeneration) { _createInvertedIndexDuringSegmentGeneration = createInvertedIndexDuringSegmentGeneration; From 3a185dd6929570552c413ce15f7a23415357a179 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Mon, 25 Nov 2024 18:00:43 +0530 Subject: [PATCH 27/47] nit --- .../data/manager/realtime/RealtimeSegmentDataManager.java | 5 ++--- .../local/indexsegment/mutable/MutableSegmentImpl.java | 4 ++-- 2 files changed, 4 insertions(+), 5 deletions(-) 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 3b791994144a..1d7a559a99fb 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 @@ -1257,7 +1257,6 @@ private static class ConsumptionStopIndicator { final Logger _logger; final ServerSegmentCompletionProtocolHandler _protocolHandler; final String _reason; - private ConsumptionStopIndicator(StreamPartitionMsgOffset offset, String segmentName, String instanceId, ServerSegmentCompletionProtocolHandler protocolHandler, String reason, Logger logger) { _offset = offset; @@ -1554,7 +1553,7 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf _isOffHeap = indexLoadingConfig.isRealtimeOffHeapAllocation(); _defaultNullHandlingEnabled = indexingConfig.isNullHandlingEnabled(); - _indexCapacityThresholdCheckEnabled = tableConfig.getIndexingConfig().isIndexCapacityThresholdCheckEnabled(); + _indexCapacityThresholdCheckEnabled = indexingConfig.isIndexCapacityThresholdCheckEnabled(); // Start new realtime segment String consumerDir = realtimeTableDataManager.getConsumerDir(); @@ -1652,7 +1651,7 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf "Failed to initialize segment data manager", e)); _segmentLogger.warn( "Scheduling task to call controller to mark the segment as OFFLINE in Ideal State due" - + " to initialization error: '{}'", + + " to initialization error: '{}'", e.getMessage()); // Since we are going to throw exception from this thread (helix execution thread), the externalview // entry for this segment will be ERROR. We allow time for Helix to make this transition, and then diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index 81ac26407dc8..8ced02b8fc18 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -661,8 +661,7 @@ private Comparable getComparisonValue(GenericRow row) { * @throws UnsupportedOperationException if the length of an MV column would exceed the * capacity of a chunk in the ForwardIndex */ - private void validateLengthOfMVColumns(GenericRow row) - throws UnsupportedOperationException { + private void validateLengthOfMVColumns(GenericRow row) throws UnsupportedOperationException { for (Map.Entry entry : _indexContainerMap.entrySet()) { IndexContainer indexContainer = entry.getValue(); FieldSpec fieldSpec = indexContainer._fieldSpec; @@ -684,6 +683,7 @@ private void validateLengthOfMVColumns(GenericRow row) } } + private void updateDictionary(GenericRow row) { for (Map.Entry entry : _indexContainerMap.entrySet()) { IndexContainer indexContainer = entry.getValue(); From d3a6ea2bc26598b8782f881dd4a1545156bfd7fb Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Mon, 25 Nov 2024 18:19:34 +0530 Subject: [PATCH 28/47] fixes lint --- ...utableSegmentEntriesAboveThresholdTest.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java index dccc4ca20f60..5f135eeb0942 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java @@ -1,3 +1,21 @@ +/** + * 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.indexsegment.mutable; import java.io.File; From ba07fc3d910f11b8fd155672b57250d9103b65b1 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Tue, 26 Nov 2024 16:22:08 +0530 Subject: [PATCH 29/47] Addresses Pr comments --- .../mutable/MutableSegmentImplTestUtils.java | 22 +++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java index ee58ce88a311..d40ec7e6897f 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java @@ -59,13 +59,13 @@ public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set varLengthDictionaryColumns, Set invertedIndexColumns, List preAggregationConfigs) { return createMutableSegmentImpl(schema, noDictionaryColumns, varLengthDictionaryColumns, invertedIndexColumns, - Collections.emptyMap(), false, false, null, null, null, null, null, null, preAggregationConfigs, false); + Collections.emptyMap(), false, false, null, null, null, null, null, null, preAggregationConfigs); } public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set noDictionaryColumns, Set varLengthDictionaryColumns, Set invertedIndexColumns, boolean aggregateMetrics) { return createMutableSegmentImpl(schema, noDictionaryColumns, varLengthDictionaryColumns, invertedIndexColumns, - aggregateMetrics, false); + aggregateMetrics); } public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set noDictionaryColumns, @@ -82,14 +82,14 @@ public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set noDictionaryColumns, Set varLengthDictionaryColumns, Set invertedIndexColumns, Map jsonIndexConfigs, ServerMetrics serverMetrics) { return createMutableSegmentImpl(schema, noDictionaryColumns, varLengthDictionaryColumns, invertedIndexColumns, - jsonIndexConfigs, false, true, null, null, null, null, null, serverMetrics, Collections.emptyList(), false); + jsonIndexConfigs, false, true, null, null, null, null, null, serverMetrics, Collections.emptyList()); } public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set noDictionaryColumns, @@ -134,4 +134,18 @@ public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set noDictionaryColumns, + Set varLengthDictionaryColumns, Set invertedIndexColumns, + Map jsonIndexConfigs, boolean aggregateMetrics, boolean nullHandlingEnabled, + UpsertConfig upsertConfig, String timeColumnName, PartitionUpsertMetadataManager partitionUpsertMetadataManager, + DedupConfig dedupConfig, PartitionDedupMetadataManager partitionDedupMetadataManager, ServerMetrics serverMetrics, + List aggregationConfigs) { + return createMutableSegmentImpl(schema, noDictionaryColumns, + varLengthDictionaryColumns, invertedIndexColumns, + jsonIndexConfigs, aggregateMetrics, nullHandlingEnabled, + upsertConfig, timeColumnName, partitionUpsertMetadataManager, + dedupConfig, partitionDedupMetadataManager, serverMetrics, + aggregationConfigs, false); + } } From 69b7e32e5c30c874c0dd461d5662d25d5565232e Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Tue, 26 Nov 2024 16:27:22 +0530 Subject: [PATCH 30/47] nit --- .../indexsegment/mutable/MutableSegmentImplTestUtils.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java index d40ec7e6897f..f90fa8e27573 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java @@ -59,13 +59,13 @@ public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set varLengthDictionaryColumns, Set invertedIndexColumns, List preAggregationConfigs) { return createMutableSegmentImpl(schema, noDictionaryColumns, varLengthDictionaryColumns, invertedIndexColumns, - Collections.emptyMap(), false, false, null, null, null, null, null, null, preAggregationConfigs); + Collections.emptyMap(), false, false, null, null, null, null, null, null, preAggregationConfigs, false); } public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set noDictionaryColumns, Set varLengthDictionaryColumns, Set invertedIndexColumns, boolean aggregateMetrics) { return createMutableSegmentImpl(schema, noDictionaryColumns, varLengthDictionaryColumns, invertedIndexColumns, - aggregateMetrics); + aggregateMetrics, false); } public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set noDictionaryColumns, From c60f1bd20d1851e199210d98e719fc674d6bf86a Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Tue, 26 Nov 2024 16:28:41 +0530 Subject: [PATCH 31/47] nit --- .../mutable/MutableSegmentImplTestUtils.java | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java index f90fa8e27573..480edc929def 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java @@ -92,6 +92,20 @@ public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set noDictionaryColumns, + Set varLengthDictionaryColumns, Set invertedIndexColumns, + Map jsonIndexConfigs, boolean aggregateMetrics, boolean nullHandlingEnabled, + UpsertConfig upsertConfig, String timeColumnName, PartitionUpsertMetadataManager partitionUpsertMetadataManager, + DedupConfig dedupConfig, PartitionDedupMetadataManager partitionDedupMetadataManager, ServerMetrics serverMetrics, + List aggregationConfigs) { + return createMutableSegmentImpl(schema, noDictionaryColumns, + varLengthDictionaryColumns, invertedIndexColumns, + jsonIndexConfigs, aggregateMetrics, nullHandlingEnabled, + upsertConfig, timeColumnName, partitionUpsertMetadataManager, + dedupConfig, partitionDedupMetadataManager, serverMetrics, + aggregationConfigs, false); + } + public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set noDictionaryColumns, Set varLengthDictionaryColumns, Set invertedIndexColumns, Map jsonIndexConfigs, boolean aggregateMetrics, boolean nullHandlingEnabled, @@ -134,18 +148,4 @@ public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set noDictionaryColumns, - Set varLengthDictionaryColumns, Set invertedIndexColumns, - Map jsonIndexConfigs, boolean aggregateMetrics, boolean nullHandlingEnabled, - UpsertConfig upsertConfig, String timeColumnName, PartitionUpsertMetadataManager partitionUpsertMetadataManager, - DedupConfig dedupConfig, PartitionDedupMetadataManager partitionDedupMetadataManager, ServerMetrics serverMetrics, - List aggregationConfigs) { - return createMutableSegmentImpl(schema, noDictionaryColumns, - varLengthDictionaryColumns, invertedIndexColumns, - jsonIndexConfigs, aggregateMetrics, nullHandlingEnabled, - upsertConfig, timeColumnName, partitionUpsertMetadataManager, - dedupConfig, partitionDedupMetadataManager, serverMetrics, - aggregationConfigs, false); - } } From 5511e1fd27c2d76c85c81b85c3304af30a35d9cb Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Tue, 26 Nov 2024 16:33:46 +0530 Subject: [PATCH 32/47] refactors method name --- .../realtime/RealtimeSegmentDataManagerTest.java | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java index c5fb96ba6b4a..ec54ec7b9ab5 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java @@ -126,7 +126,13 @@ private SegmentZKMetadata createZkMetadata() { private FakeRealtimeSegmentDataManager createFakeSegmentManager() throws Exception { - return createFakeSegmentManager(false, new TimeSupplier(), null, null, null, false); + return createFakeSegmentManager(false, new TimeSupplier(), null, null, null); + } + + private FakeRealtimeSegmentDataManager createFakeSegmentManager(boolean noUpsert, TimeSupplier timeSupplier, + @Nullable String maxRows, @Nullable String maxDuration, @Nullable TableConfig tableConfig) + throws Exception { + return createFakeSegmentManager(noUpsert, timeSupplier, maxRows, maxDuration, tableConfig, false); } private FakeRealtimeSegmentDataManager createFakeSegmentManager(boolean noUpsert, TimeSupplier timeSupplier, @@ -306,7 +312,7 @@ public void testCommitAfterCatchupWithPeriodOffset() StreamConfigProperties.constructStreamProperty(StreamConfigProperties.STREAM_CONSUMER_OFFSET_CRITERIA, "fakeStream"), "2d"); FakeRealtimeSegmentDataManager segmentDataManager = - createFakeSegmentManager(false, new TimeSupplier(), null, null, tableConfig, false); + createFakeSegmentManager(false, new TimeSupplier(), null, null, tableConfig); RealtimeSegmentDataManager.PartitionConsumer consumer = segmentDataManager.createPartitionConsumer(); final LongMsgOffset firstOffset = new LongMsgOffset(START_OFFSET_VALUE + 500); final LongMsgOffset catchupOffset = new LongMsgOffset(firstOffset.getOffset() + 10); @@ -353,7 +359,7 @@ public void testCommitAfterCatchupWithTimestampOffset() StreamConfigProperties.constructStreamProperty(StreamConfigProperties.STREAM_CONSUMER_OFFSET_CRITERIA, "fakeStream"), Instant.now().toString()); FakeRealtimeSegmentDataManager segmentDataManager = - createFakeSegmentManager(false, new TimeSupplier(), null, null, tableConfig, false); + createFakeSegmentManager(false, new TimeSupplier(), null, null, tableConfig); RealtimeSegmentDataManager.PartitionConsumer consumer = segmentDataManager.createPartitionConsumer(); final LongMsgOffset firstOffset = new LongMsgOffset(START_OFFSET_VALUE + 500); final LongMsgOffset catchupOffset = new LongMsgOffset(firstOffset.getOffset() + 10); From ffa984dd87fad9c55e68fd2040f2d97933a6ba68 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Tue, 26 Nov 2024 16:34:34 +0530 Subject: [PATCH 33/47] nit --- .../manager/realtime/RealtimeSegmentDataManagerTest.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java index ec54ec7b9ab5..329af8195fdc 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java @@ -834,8 +834,7 @@ public Long get() { } }; try (FakeRealtimeSegmentDataManager segmentDataManager = createFakeSegmentManager(true, timeSupplier, - String.valueOf(FakeStreamConfigUtils.SEGMENT_FLUSH_THRESHOLD_ROWS * 2), segmentTimeThresholdMins + "m", null, - false)) { + String.valueOf(FakeStreamConfigUtils.SEGMENT_FLUSH_THRESHOLD_ROWS * 2), segmentTimeThresholdMins + "m", null)) { segmentDataManager._stubConsumeLoop = false; segmentDataManager._state.set(segmentDataManager, RealtimeSegmentDataManager.State.INITIAL_CONSUMING); @@ -870,8 +869,7 @@ public void testShouldNotSkipUnfilteredMessagesIfNotIndexedAndRowCountThresholdI final int segmentTimeThresholdMins = 10; TimeSupplier timeSupplier = new TimeSupplier(); try (FakeRealtimeSegmentDataManager segmentDataManager = createFakeSegmentManager(true, timeSupplier, - String.valueOf(FakeStreamConfigUtils.SEGMENT_FLUSH_THRESHOLD_ROWS), segmentTimeThresholdMins + "m", null, - false)) { + String.valueOf(FakeStreamConfigUtils.SEGMENT_FLUSH_THRESHOLD_ROWS), segmentTimeThresholdMins + "m", null)) { segmentDataManager._stubConsumeLoop = false; segmentDataManager._state.set(segmentDataManager, RealtimeSegmentDataManager.State.INITIAL_CONSUMING); From 783f8c095f4bc3c4cac4a533f63ec0436ab1c6d3 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Tue, 26 Nov 2024 16:38:01 +0530 Subject: [PATCH 34/47] nit --- .../org/apache/pinot/segment/spi/index/mutable/MutableIndex.java | 1 - 1 file changed, 1 deletion(-) diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableIndex.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableIndex.java index 320b3f689521..494361947b25 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableIndex.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableIndex.java @@ -23,7 +23,6 @@ import javax.annotation.Nullable; import org.apache.pinot.segment.spi.index.IndexReader; - /** * Implementations of this interface can be used to create indexes in realtime tables and at the same time to read them. * From f9e4b9cb19e9bd3e167a4b75f7ffbaa53789fe68 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Wed, 27 Nov 2024 16:27:30 +0530 Subject: [PATCH 35/47] Fixes lint --- .../forward/mutable/FixedByteMVMutableForwardIndexTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/FixedByteMVMutableForwardIndexTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/FixedByteMVMutableForwardIndexTest.java index 78d48e534539..77fd706f3c64 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/FixedByteMVMutableForwardIndexTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/FixedByteMVMutableForwardIndexTest.java @@ -21,9 +21,7 @@ import java.io.IOException; import java.lang.reflect.Field; import java.util.Arrays; -import java.util.Map; import java.util.Random; -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.impl.forward.FixedByteMVMutableForwardIndex; import org.apache.pinot.segment.spi.memory.PinotDataBufferMemoryManager; From 3e1dde22f531c49e27e0db5af244524463b46cc8 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Thu, 28 Nov 2024 15:53:13 +0530 Subject: [PATCH 36/47] fixes lint --- .../MutableSegmentEntriesAboveThresholdTest.java | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java index 5f135eeb0942..a943003b7f00 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java @@ -55,23 +55,23 @@ public class MutableSegmentEntriesAboveThresholdTest { private static class FakeMutableForwardIndex implements MutableForwardIndex { private final MutableForwardIndex _mutableForwardIndex; - private static final int threshold = 2; + private static final int THRESHOLD = 2; private int _numValues; FakeMutableForwardIndex(MutableForwardIndex mutableForwardIndex) { - this._mutableForwardIndex = mutableForwardIndex; - this._numValues = 0; + _mutableForwardIndex = mutableForwardIndex; + _numValues = 0; } @Override public boolean canAddMore() { - return _numValues < threshold; + return _numValues < THRESHOLD; } @Override public void setDictIdMV(int docId, int[] dictIds) { _numValues += dictIds.length; - this._mutableForwardIndex.setDictIdMV(docId, dictIds); + _mutableForwardIndex.setDictIdMV(docId, dictIds); } @Override @@ -86,7 +86,7 @@ public int getLengthOfLongestElement() { @Override public void setDictId(int docId, int dictId) { - this._mutableForwardIndex.setDictId(docId, dictId); + _mutableForwardIndex.setDictId(docId, dictId); } @Override @@ -120,7 +120,6 @@ private File getAvroFile() { private MutableSegmentImpl getMutableSegment(File avroFile) throws Exception { FileUtils.deleteQuietly(TEMP_DIR); - MutableSegmentImpl _mutableSegmentImpl; SegmentGeneratorConfig config = SegmentTestUtils.getSegmentGeneratorConfigWithoutTimeColumn(avroFile, TEMP_DIR, "testTable"); @@ -130,11 +129,10 @@ private MutableSegmentImpl getMutableSegment(File avroFile) _schema = config.getSchema(); VirtualColumnProviderFactory.addBuiltInVirtualColumnsToSegmentSchema(_schema, "testSegment"); - _mutableSegmentImpl = MutableSegmentImplTestUtils + return MutableSegmentImplTestUtils .createMutableSegmentImpl(_schema, Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), Collections.emptyMap(), false, false, null, null, null, null, null, null, Collections.emptyList(), true); - return _mutableSegmentImpl; } @Test From ed32e3df6696eaf77550660bdf72669c2a710df2 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Wed, 4 Dec 2024 23:44:32 +0530 Subject: [PATCH 37/47] Addresses PR comment --- .../realtime/RealtimeSegmentDataManager.java | 7 ++----- .../RealtimeSegmentDataManagerTest.java | 1 - .../mutable/MutableSegmentImpl.java | 6 ++---- .../realtime/impl/RealtimeSegmentConfig.java | 17 ++--------------- ...MutableSegmentEntriesAboveThresholdTest.java | 4 ++-- .../mutable/MutableSegmentImplTestUtils.java | 3 +-- .../pinot/spi/config/table/IndexingConfig.java | 9 --------- .../spi/utils/builder/TableConfigBuilder.java | 7 ------- 8 files changed, 9 insertions(+), 45 deletions(-) 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 1d7a559a99fb..8e81071ef47c 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 @@ -310,7 +310,6 @@ public void deleteSegmentFile() { private String _stopReason = null; private final Semaphore _segBuildSemaphore; private final boolean _isOffHeap; - private final boolean _indexCapacityThresholdCheckEnabled; /** * Whether null handling is enabled by default. This value is only used if * {@link Schema#isEnableColumnBasedNullHandling()} is false. @@ -364,7 +363,7 @@ private boolean endCriteriaReached() { _numRowsConsumed, _numRowsIndexed); _stopReason = SegmentCompletionProtocol.REASON_FORCE_COMMIT_MESSAGE_RECEIVED; return true; - } else if (_indexCapacityThresholdCheckEnabled && _realtimeSegment.isIndexCapacityThresholdBreached()) { + } else if (_realtimeSegment.canAddMore()) { _segmentLogger.info( "Stopping consumption as mutable index cannot consume more rows - numRowsConsumed={} " + "numRowsIndexed={}", @@ -1553,7 +1552,6 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf _isOffHeap = indexLoadingConfig.isRealtimeOffHeapAllocation(); _defaultNullHandlingEnabled = indexingConfig.isNullHandlingEnabled(); - _indexCapacityThresholdCheckEnabled = indexingConfig.isIndexCapacityThresholdCheckEnabled(); // Start new realtime segment String consumerDir = realtimeTableDataManager.getConsumerDir(); @@ -1577,8 +1575,7 @@ public RealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableConf .setUpsertDropOutOfOrderRecord(tableConfig.isDropOutOfOrderRecord()) .setPartitionDedupMetadataManager(partitionDedupMetadataManager) .setDedupTimeColumn(tableConfig.getDedupTimeColumn()) - .setFieldConfigList(tableConfig.getFieldConfigList()) - .setIndexCapacityThresholdCheckEnabled(_indexCapacityThresholdCheckEnabled); + .setFieldConfigList(tableConfig.getFieldConfigList()); // Create message decoder Set fieldsToRead = IngestionUtils.getFieldsForRecordExtractor(_tableConfig.getIngestionConfig(), _schema); diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java index 329af8195fdc..2473246d9354 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java @@ -154,7 +154,6 @@ private FakeRealtimeSegmentDataManager createFakeSegmentManager(boolean noUpsert tableConfig.getIndexingConfig().getStreamConfigs() .put(StreamConfigProperties.SEGMENT_FLUSH_THRESHOLD_TIME, maxDuration); } - tableConfig.getIndexingConfig().setIndexCapacityThresholdCheckEnabled(setIndexCapacityThresholdCheck); RealtimeTableDataManager tableDataManager = createTableDataManager(tableConfig); LLCSegmentName llcSegmentName = new LLCSegmentName(SEGMENT_NAME_STR); _partitionGroupIdToSemaphoreMap.putIfAbsent(PARTITION_GROUP_ID, new Semaphore(1)); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index 8ced02b8fc18..a34b0c426c10 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -148,7 +148,6 @@ public class MutableSegmentImpl implements MutableSegment { private final int _mainPartitionId; // partition id designated for this consuming segment private final boolean _defaultNullHandlingEnabled; private final File _consumerDir; - private final boolean _indexCapacityThresholdCheckEnabled; private final Map _indexContainerMap = new HashMap<>(); private boolean _indexCapacityThresholdBreached; @@ -228,7 +227,6 @@ public boolean isMutableSegment() { _mainPartitionId = config.getPartitionId(); _defaultNullHandlingEnabled = config.isNullHandlingEnabled(); _consumerDir = new File(config.getConsumerDir()); - _indexCapacityThresholdCheckEnabled = config.isIndexCapacityThresholdCheckEnabled(); Collection allFieldSpecs = _schema.getAllFieldSpecs(); List physicalFieldSpecs = new ArrayList<>(allFieldSpecs.size()); @@ -838,7 +836,7 @@ private void addNewRow(int docId, GenericRow row) { // an immutable index and segment build fails causing the realtime consumption to stop. // Hence, The below check is a temporary measure to avoid such scenarios until immutable index // implementations are changed. - if (_indexCapacityThresholdCheckEnabled && !mutableIndex.canAddMore()) { + if (!mutableIndex.canAddMore()) { _logger.warn( "failed to index value with indexType {} for column {} as index cannot consume more rows", indexEntry.getKey(), column @@ -1281,7 +1279,7 @@ private boolean isAggregateMetricsEnabled() { return _recordIdMap != null; } - public boolean isIndexCapacityThresholdBreached() { + public boolean canAddMore() { return _indexCapacityThresholdBreached; } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java index 2bf771cad2a7..5b3aeb26d533 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/RealtimeSegmentConfig.java @@ -73,7 +73,6 @@ public class RealtimeSegmentConfig { private final String _consumerDir; private final List _fieldConfigList; private final List _ingestionAggregationConfigs; - private final boolean _indexCapacityThresholdCheckEnabled; // TODO: Clean up this constructor. Most of these things can be extracted from tableConfig. @@ -90,8 +89,7 @@ private RealtimeSegmentConfig(String tableNameWithType, String segmentName, Stri List upsertComparisonColumns, String upsertDeleteRecordColumn, String upsertOutOfOrderRecordColumn, boolean upsertDropOutOfOrderRecord, PartitionUpsertMetadataManager partitionUpsertMetadataManager, String dedupTimeColumn, PartitionDedupMetadataManager partitionDedupMetadataManager, - List fieldConfigList, List ingestionAggregationConfigs, - boolean indexCapacityThresholdCheckEnabled) { + List fieldConfigList, List ingestionAggregationConfigs) { _tableNameWithType = tableNameWithType; _segmentName = segmentName; _streamName = streamName; @@ -121,7 +119,6 @@ private RealtimeSegmentConfig(String tableNameWithType, String segmentName, Stri _partitionDedupMetadataManager = partitionDedupMetadataManager; _fieldConfigList = fieldConfigList; _ingestionAggregationConfigs = ingestionAggregationConfigs; - _indexCapacityThresholdCheckEnabled = indexCapacityThresholdCheckEnabled; } public String getTableNameWithType() { @@ -244,10 +241,6 @@ public List getIngestionAggregationConfigs() { return _ingestionAggregationConfigs; } - public boolean isIndexCapacityThresholdCheckEnabled() { - return _indexCapacityThresholdCheckEnabled; - } - public static class Builder { private String _tableNameWithType; private String _segmentName; @@ -282,7 +275,6 @@ public static class Builder { private PartitionDedupMetadataManager _partitionDedupMetadataManager; private List _fieldConfigList; private List _ingestionAggregationConfigs; - private boolean _indexCapacityThresholdCheckEnabled; public Builder() { _indexConfigByCol = new HashMap<>(); @@ -483,11 +475,6 @@ public Builder setIngestionAggregationConfigs(List ingestionA return this; } - public Builder setIndexCapacityThresholdCheckEnabled(boolean indexCapacityThresholdCheckEnabled) { - _indexCapacityThresholdCheckEnabled = indexCapacityThresholdCheckEnabled; - return this; - } - public RealtimeSegmentConfig build() { Map indexConfigByCol = Maps.newHashMapWithExpectedSize(_indexConfigByCol.size()); for (Map.Entry entry : _indexConfigByCol.entrySet()) { @@ -500,7 +487,7 @@ public RealtimeSegmentConfig build() { _defaultNullHandlingEnabled, _consumerDir, _upsertMode, _upsertConsistencyMode, _upsertComparisonColumns, _upsertDeleteRecordColumn, _upsertOutOfOrderRecordColumn, _upsertDropOutOfOrderRecord, _partitionUpsertMetadataManager, _dedupTimeColumn, _partitionDedupMetadataManager, _fieldConfigList, - _ingestionAggregationConfigs, _indexCapacityThresholdCheckEnabled); + _ingestionAggregationConfigs); } } } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java index a943003b7f00..103090ae6929 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java @@ -148,7 +148,7 @@ public void testNoLimitBreached() mutableSegment.index(recordReader.next(reuse), defaultMetadata); } } - assert !mutableSegment.isIndexCapacityThresholdBreached(); + assert !mutableSegment.canAddMore(); } @Test @@ -189,6 +189,6 @@ public void testLimitBreached() } } - assert mutableSegment.isIndexCapacityThresholdBreached(); + assert mutableSegment.canAddMore(); } } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java index 480edc929def..eaf0b9c95ce9 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java @@ -140,8 +140,7 @@ public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set entry : jsonIndexConfigs.entrySet()) { segmentConfBuilder.setIndex(entry.getKey(), StandardIndexes.json(), entry.getValue()); } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java index 8a8efb98547e..5beb126e0d0e 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java @@ -66,7 +66,6 @@ public class IndexingConfig extends BaseJsonConfig { private boolean _aggregateMetrics; private boolean _nullHandlingEnabled; private boolean _columnMajorSegmentBuilderEnabled = true; - private boolean _indexCapacityThresholdCheckEnabled; /** * If `optimizeDictionary` enabled, dictionary is not created for the high-cardinality @@ -411,14 +410,6 @@ public void setSegmentNameGeneratorType(String segmentNameGeneratorType) { _segmentNameGeneratorType = segmentNameGeneratorType; } - public boolean isIndexCapacityThresholdCheckEnabled() { - return _indexCapacityThresholdCheckEnabled; - } - - public void setIndexCapacityThresholdCheckEnabled(boolean indexCapacityThresholdCheckEnabled) { - _indexCapacityThresholdCheckEnabled = indexCapacityThresholdCheckEnabled; - } - /** * Returns all columns referenced in the indexing config. This is useful to construct FieldIndexConfigs in * IndexLoadingConfig when schema is not provided. Only including the columns referenced by indexes supported in diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java index 1290cf7baef5..5e9d915cfc46 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java @@ -112,7 +112,6 @@ public class TableConfigBuilder { private boolean _optimizeDictionaryType; private double _noDictionarySizeRatioThreshold; private double _noDictionaryCardinalityRatioThreshold; - private boolean _indexCapacityThresholdCheckEnabled; private TableCustomConfig _customConfig; private QuotaConfig _quotaConfig; @@ -288,11 +287,6 @@ public TableConfigBuilder setNoDictionaryCardinalityRatioThreshold(double noDict return this; } - public TableConfigBuilder setIndexCapacityThresholdCheckEnabled(boolean indexCapacityThresholdCheckEnabled) { - _indexCapacityThresholdCheckEnabled = indexCapacityThresholdCheckEnabled; - return this; - } - public TableConfigBuilder setCreateInvertedIndexDuringSegmentGeneration( boolean createInvertedIndexDuringSegmentGeneration) { _createInvertedIndexDuringSegmentGeneration = createInvertedIndexDuringSegmentGeneration; @@ -494,7 +488,6 @@ public TableConfig build() { indexingConfig.setNoDictionarySizeRatioThreshold(_noDictionarySizeRatioThreshold); indexingConfig.setNoDictionaryCardinalityRatioThreshold(_noDictionaryCardinalityRatioThreshold); indexingConfig.setTierOverwrites(_tierOverwrites); - indexingConfig.setIndexCapacityThresholdCheckEnabled(_indexCapacityThresholdCheckEnabled); if (_customConfig == null) { _customConfig = new TableCustomConfig(null); From e57d726664eb90a11fc0e8b243d84b201c4b1667 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Wed, 4 Dec 2024 23:46:27 +0530 Subject: [PATCH 38/47] fix bug --- .../core/data/manager/realtime/RealtimeSegmentDataManager.java | 2 +- .../segment/local/indexsegment/mutable/MutableSegmentImpl.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 8e81071ef47c..5249e0068b6c 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 @@ -363,7 +363,7 @@ private boolean endCriteriaReached() { _numRowsConsumed, _numRowsIndexed); _stopReason = SegmentCompletionProtocol.REASON_FORCE_COMMIT_MESSAGE_RECEIVED; return true; - } else if (_realtimeSegment.canAddMore()) { + } else if (!_realtimeSegment.canAddMore()) { _segmentLogger.info( "Stopping consumption as mutable index cannot consume more rows - numRowsConsumed={} " + "numRowsIndexed={}", diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index a34b0c426c10..53d2efcbdf98 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -1280,7 +1280,7 @@ private boolean isAggregateMetricsEnabled() { } public boolean canAddMore() { - return _indexCapacityThresholdBreached; + return !_indexCapacityThresholdBreached; } // NOTE: Okay for single-writer From 24243bda01815436d042fbbdf4cda11ba74f8dcd Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Thu, 5 Dec 2024 00:14:52 +0530 Subject: [PATCH 39/47] removes reflection in test --- .../realtime/RealtimeSegmentDataManager.java | 6 +++- .../RealtimeSegmentDataManagerTest.java | 28 ++++++++----------- 2 files changed, 17 insertions(+), 17 deletions(-) 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 5249e0068b6c..5199bab7e047 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 @@ -363,7 +363,7 @@ private boolean endCriteriaReached() { _numRowsConsumed, _numRowsIndexed); _stopReason = SegmentCompletionProtocol.REASON_FORCE_COMMIT_MESSAGE_RECEIVED; return true; - } else if (!_realtimeSegment.canAddMore()) { + } else if (!canAddMore()) { _segmentLogger.info( "Stopping consumption as mutable index cannot consume more rows - numRowsConsumed={} " + "numRowsIndexed={}", @@ -707,6 +707,10 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee return prematureExit; } + boolean canAddMore() { + return _realtimeSegment.canAddMore(); + } + public class PartitionConsumer implements Runnable { public void run() { long initialConsumptionEnd = 0L; diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java index 2473246d9354..cbf420b97819 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java @@ -132,13 +132,6 @@ private FakeRealtimeSegmentDataManager createFakeSegmentManager() private FakeRealtimeSegmentDataManager createFakeSegmentManager(boolean noUpsert, TimeSupplier timeSupplier, @Nullable String maxRows, @Nullable String maxDuration, @Nullable TableConfig tableConfig) throws Exception { - return createFakeSegmentManager(noUpsert, timeSupplier, maxRows, maxDuration, tableConfig, false); - } - - private FakeRealtimeSegmentDataManager createFakeSegmentManager(boolean noUpsert, TimeSupplier timeSupplier, - @Nullable String maxRows, @Nullable String maxDuration, @Nullable TableConfig tableConfig, - boolean setIndexCapacityThresholdCheck) - throws Exception { SegmentZKMetadata segmentZKMetadata = createZkMetadata(); if (tableConfig == null) { tableConfig = createTableConfig(); @@ -648,18 +641,11 @@ public void testEndCriteriaChecking() // test end criteria reached if any of the index cannot take more rows try (FakeRealtimeSegmentDataManager segmentDataManager = createFakeSegmentManager(false, new TimeSupplier(), null, - null, null, true)) { + null, null)) { segmentDataManager._state.set(segmentDataManager, RealtimeSegmentDataManager.State.INITIAL_CONSUMING); Assert.assertFalse(segmentDataManager.invokeEndCriteriaReached()); - Field realtimeSegmentField = RealtimeSegmentDataManager.class.getDeclaredField("_realtimeSegment"); - realtimeSegmentField.setAccessible(true); - MutableSegmentImpl mutableSegment = (MutableSegmentImpl) realtimeSegmentField.get(segmentDataManager); - - Field numOfColValuesLimitBreachedField = - MutableSegmentImpl.class.getDeclaredField("_indexCapacityThresholdBreached"); - numOfColValuesLimitBreachedField.setAccessible(true); - numOfColValuesLimitBreachedField.set(mutableSegment, true); + segmentDataManager.setIndexCapacityThresholdBreached(true); Assert.assertTrue(segmentDataManager.invokeEndCriteriaReached()); Assert.assertEquals(segmentDataManager.getStopReason(), @@ -935,6 +921,7 @@ public static class FakeRealtimeSegmentDataManager extends RealtimeSegmentDataMa public Map _semaphoreMap; public boolean _stubConsumeLoop = true; private TimeSupplier _timeSupplier; + private boolean indexCapacityThresholdBreached; private static InstanceDataManagerConfig makeInstanceDataManagerConfig() { InstanceDataManagerConfig dataManagerConfig = mock(InstanceDataManagerConfig.class); @@ -1115,6 +1102,15 @@ public void setFinalOffset(long offset) { setOffset(offset, "_finalOffset"); } + @Override + protected boolean canAddMore() { + return !indexCapacityThresholdBreached; + } + + public void setIndexCapacityThresholdBreached(boolean indexCapacityThresholdBreached) { + this.indexCapacityThresholdBreached = indexCapacityThresholdBreached; + } + public boolean invokeEndCriteriaReached() { Method endCriteriaReached = null; try { From 541a7d9d5cee20991b39b2dab88105588dd1f296 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Thu, 5 Dec 2024 00:22:09 +0530 Subject: [PATCH 40/47] fix test --- .../mutable/MutableSegmentEntriesAboveThresholdTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java index 103090ae6929..2b2d996b84b8 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java @@ -189,6 +189,6 @@ public void testLimitBreached() } } - assert mutableSegment.canAddMore(); + assert !mutableSegment.canAddMore(); } } From 9f67ce4f5afc61201da12c60cb7baa69827f8531 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Thu, 5 Dec 2024 00:34:08 +0530 Subject: [PATCH 41/47] fixes lint --- .../manager/realtime/RealtimeSegmentDataManagerTest.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java index cbf420b97819..60a2f1923364 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManagerTest.java @@ -49,7 +49,6 @@ import org.apache.pinot.core.realtime.impl.fakestream.FakeStreamConsumerFactory; import org.apache.pinot.core.realtime.impl.fakestream.FakeStreamMessageDecoder; import org.apache.pinot.segment.local.data.manager.TableDataManager; -import org.apache.pinot.segment.local.indexsegment.mutable.MutableSegmentImpl; import org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentStatsHistory; import org.apache.pinot.segment.local.segment.creator.Fixtures; import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; @@ -921,7 +920,7 @@ public static class FakeRealtimeSegmentDataManager extends RealtimeSegmentDataMa public Map _semaphoreMap; public boolean _stubConsumeLoop = true; private TimeSupplier _timeSupplier; - private boolean indexCapacityThresholdBreached; + private boolean _indexCapacityThresholdBreached; private static InstanceDataManagerConfig makeInstanceDataManagerConfig() { InstanceDataManagerConfig dataManagerConfig = mock(InstanceDataManagerConfig.class); @@ -1104,11 +1103,11 @@ public void setFinalOffset(long offset) { @Override protected boolean canAddMore() { - return !indexCapacityThresholdBreached; + return !_indexCapacityThresholdBreached; } public void setIndexCapacityThresholdBreached(boolean indexCapacityThresholdBreached) { - this.indexCapacityThresholdBreached = indexCapacityThresholdBreached; + _indexCapacityThresholdBreached = indexCapacityThresholdBreached; } public boolean invokeEndCriteriaReached() { From 65fcaab96c15ec0bccce59b2d9ed8b831ac17a30 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Thu, 5 Dec 2024 00:42:24 +0530 Subject: [PATCH 42/47] fix log --- .../segment/local/indexsegment/mutable/MutableSegmentImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index 53d2efcbdf98..57c15df7c496 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -838,7 +838,7 @@ private void addNewRow(int docId, GenericRow row) { // implementations are changed. if (!mutableIndex.canAddMore()) { _logger.warn( - "failed to index value with indexType {} for column {} as index cannot consume more rows", + "index: {} for column: {} cannot consume more rows, marking _indexCapacityThresholdBreached as true", indexEntry.getKey(), column ); _indexCapacityThresholdBreached = true; From e6c1f43b7f19596180c6bc76a66b3784443d701e Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Thu, 5 Dec 2024 01:16:41 +0530 Subject: [PATCH 43/47] move log to debug --- .../segment/local/indexsegment/mutable/MutableSegmentImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index 57c15df7c496..710eea85f78c 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -837,7 +837,7 @@ private void addNewRow(int docId, GenericRow row) { // Hence, The below check is a temporary measure to avoid such scenarios until immutable index // implementations are changed. if (!mutableIndex.canAddMore()) { - _logger.warn( + _logger.debug( "index: {} for column: {} cannot consume more rows, marking _indexCapacityThresholdBreached as true", indexEntry.getKey(), column ); From dbaaf5c5a5234fbe770f6e139b6474df440e6a36 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Thu, 5 Dec 2024 01:48:00 +0530 Subject: [PATCH 44/47] fixes test --- .../mutable/MutableSegmentEntriesAboveThresholdTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java index 2b2d996b84b8..b1e49e81e7e0 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java @@ -148,7 +148,7 @@ public void testNoLimitBreached() mutableSegment.index(recordReader.next(reuse), defaultMetadata); } } - assert !mutableSegment.canAddMore(); + assert mutableSegment.canAddMore(); } @Test From d15a3fd0085e23694f865c558bdee8d7f7513a64 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Thu, 5 Dec 2024 11:00:59 +0530 Subject: [PATCH 45/47] Addresses PR comment --- .../realtime/RealtimeSegmentDataManager.java | 1 + .../indexsegment/mutable/MutableSegmentImpl.java | 4 ++-- .../mutable/MutableSegmentImplTestUtils.java | 16 +--------------- 3 files changed, 4 insertions(+), 17 deletions(-) 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 5199bab7e047..4601abdb24fc 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 @@ -707,6 +707,7 @@ private boolean processStreamEvents(MessageBatch messageBatch, long idlePipeSlee return prematureExit; } + @VisibleForTesting boolean canAddMore() { return _realtimeSegment.canAddMore(); } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index 710eea85f78c..5bd49f256d84 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -837,8 +837,8 @@ private void addNewRow(int docId, GenericRow row) { // Hence, The below check is a temporary measure to avoid such scenarios until immutable index // implementations are changed. if (!mutableIndex.canAddMore()) { - _logger.debug( - "index: {} for column: {} cannot consume more rows, marking _indexCapacityThresholdBreached as true", + _logger.info( + "Index: {} for column: {} cannot consume more rows, marking _indexCapacityThresholdBreached as true", indexEntry.getKey(), column ); _indexCapacityThresholdBreached = true; diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java index eaf0b9c95ce9..b23f203ec7a1 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplTestUtils.java @@ -59,7 +59,7 @@ public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set varLengthDictionaryColumns, Set invertedIndexColumns, List preAggregationConfigs) { return createMutableSegmentImpl(schema, noDictionaryColumns, varLengthDictionaryColumns, invertedIndexColumns, - Collections.emptyMap(), false, false, null, null, null, null, null, null, preAggregationConfigs, false); + Collections.emptyMap(), false, false, null, null, null, null, null, null, preAggregationConfigs); } public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set noDictionaryColumns, @@ -98,20 +98,6 @@ public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set aggregationConfigs) { - return createMutableSegmentImpl(schema, noDictionaryColumns, - varLengthDictionaryColumns, invertedIndexColumns, - jsonIndexConfigs, aggregateMetrics, nullHandlingEnabled, - upsertConfig, timeColumnName, partitionUpsertMetadataManager, - dedupConfig, partitionDedupMetadataManager, serverMetrics, - aggregationConfigs, false); - } - - public static MutableSegmentImpl createMutableSegmentImpl(Schema schema, Set noDictionaryColumns, - Set varLengthDictionaryColumns, Set invertedIndexColumns, - Map jsonIndexConfigs, boolean aggregateMetrics, boolean nullHandlingEnabled, - UpsertConfig upsertConfig, String timeColumnName, PartitionUpsertMetadataManager partitionUpsertMetadataManager, - DedupConfig dedupConfig, PartitionDedupMetadataManager partitionDedupMetadataManager, ServerMetrics serverMetrics, - List aggregationConfigs, boolean thresholdForColEnabled) { RealtimeSegmentStatsHistory statsHistory = mock(RealtimeSegmentStatsHistory.class); when(statsHistory.getEstimatedCardinality(anyString())).thenReturn(200); From 6960b5f19a2e38b0c5fc719b52b862bc28b4a3e4 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Thu, 5 Dec 2024 11:06:17 +0530 Subject: [PATCH 46/47] nit --- .../segment/local/indexsegment/mutable/MutableSegmentImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index 5bd49f256d84..1812334f67b9 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -836,7 +836,7 @@ private void addNewRow(int docId, GenericRow row) { // an immutable index and segment build fails causing the realtime consumption to stop. // Hence, The below check is a temporary measure to avoid such scenarios until immutable index // implementations are changed. - if (!mutableIndex.canAddMore()) { + if (!_indexCapacityThresholdBreached && !mutableIndex.canAddMore()) { _logger.info( "Index: {} for column: {} cannot consume more rows, marking _indexCapacityThresholdBreached as true", indexEntry.getKey(), column From eb3662f6b684192e3cbe5c26423ab0fa4b2dde99 Mon Sep 17 00:00:00 2001 From: Harnoor7 Date: Thu, 5 Dec 2024 11:10:20 +0530 Subject: [PATCH 47/47] nit --- .../mutable/MutableSegmentEntriesAboveThresholdTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java index b1e49e81e7e0..1eaaab657d21 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentEntriesAboveThresholdTest.java @@ -132,7 +132,7 @@ private MutableSegmentImpl getMutableSegment(File avroFile) return MutableSegmentImplTestUtils .createMutableSegmentImpl(_schema, Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), Collections.emptyMap(), - false, false, null, null, null, null, null, null, Collections.emptyList(), true); + false, false, null, null, null, null, null, null, Collections.emptyList()); } @Test