Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Improve realtime Lucene text index freshness/cpu/disk io usage #13503

Merged
merged 3 commits into from
Jul 26, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -51,8 +51,6 @@
import org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentConfig;
import org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentStatsHistory;
import org.apache.pinot.segment.local.realtime.impl.dictionary.BaseOffHeapMutableDictionary;
import org.apache.pinot.segment.local.realtime.impl.invertedindex.RealtimeLuceneIndexRefreshState;
import org.apache.pinot.segment.local.realtime.impl.invertedindex.RealtimeLuceneTextIndex;
import org.apache.pinot.segment.local.realtime.impl.nullvalue.MutableNullValueVector;
import org.apache.pinot.segment.local.segment.index.datasource.ImmutableDataSource;
import org.apache.pinot.segment.local.segment.index.datasource.MutableDataSource;
Expand Down Expand Up @@ -159,8 +157,6 @@ public class MutableSegmentImpl implements MutableSegment {
private volatile long _lastIndexedTimeMs = Long.MIN_VALUE;
private volatile long _latestIngestionTimeMs = Long.MIN_VALUE;

private RealtimeLuceneIndexRefreshState.RealtimeLuceneReaders _realtimeLuceneReaders;

private final PartitionDedupMetadataManager _partitionDedupMetadataManager;

private final PartitionUpsertMetadataManager _partitionUpsertMetadataManager;
Expand Down Expand Up @@ -347,17 +343,6 @@ public boolean isMutableSegment() {
}
}

// TODO - this logic is in the wrong place and belongs in a Lucene-specific submodule,
// it is beyond the scope of realtime index pluggability to do this refactoring, so realtime
// text indexes remain statically defined. Revisit this after this refactoring has been done.
MutableIndex textIndex = mutableIndexes.get(StandardIndexes.text());
if (textIndex instanceof RealtimeLuceneTextIndex) {
if (_realtimeLuceneReaders == null) {
_realtimeLuceneReaders = new RealtimeLuceneIndexRefreshState.RealtimeLuceneReaders(_segmentName);
}
_realtimeLuceneReaders.addReader((RealtimeLuceneTextIndex) textIndex);
}

Pair<String, ValueAggregator> columnAggregatorPair =
metricsAggregators.getOrDefault(column, Pair.of(column, null));
String sourceColumn = columnAggregatorPair.getLeft();
Expand All @@ -368,13 +353,6 @@ public boolean isMutableSegment() {
nullValueVector, sourceColumn, valueAggregator));
}

// TODO separate concerns: this logic does not belong here
if (_realtimeLuceneReaders != null) {
// add the realtime lucene index readers to the global queue for refresh task to pick up
RealtimeLuceneIndexRefreshState realtimeLuceneIndexRefreshState = RealtimeLuceneIndexRefreshState.getInstance();
realtimeLuceneIndexRefreshState.addRealtimeReadersToQueue(_realtimeLuceneReaders);
}

_partitionDedupMetadataManager = config.getPartitionDedupMetadataManager();

_partitionUpsertMetadataManager = config.getPartitionUpsertMetadataManager();
Expand Down Expand Up @@ -993,19 +971,6 @@ public void destroy() {
}
}

// Stop the text index refresh before closing the indexes
if (_realtimeLuceneReaders != null) {
// set this to true as a way of signalling the refresh task thread to
// not attempt refresh on this segment here onwards
_realtimeLuceneReaders.getLock().lock();
try {
_realtimeLuceneReaders.setSegmentDestroyed();
_realtimeLuceneReaders.clearRealtimeReaderList();
} finally {
_realtimeLuceneReaders.getLock().unlock();
}
}

// Close the indexes
for (IndexContainer indexContainer : _indexContainerMap.values()) {
indexContainer.close();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
/**
* 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.realtime.impl.invertedindex;

import java.io.IOException;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.lucene.index.MergeTrigger;
import org.apache.lucene.index.SegmentCommitInfo;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.index.TieredMergePolicy;
import org.apache.lucene.store.NRTCachingDirectory;


/**
* LuceneNRTCachingMergePolicy is a best-effort policy to generate merges for segments that are fully in memory,
* at the time of SegmentInfo selection. It does not consider segments that have been flushed to disk eligible
* for merging.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add comments on why we want to introduce NRTCaching. For performance? reduced I/O?

* <p>
* Each refresh creates a small Lucene segment. Increasing the frequency of refreshes to reduce indexing lag results
* in a large number of small segments, and high disk IO ops for merging them. By using this best-effort merge policy
* the small ops can be avoided since the segments are in memory when merged.
*/
public class LuceneNRTCachingMergePolicy extends TieredMergePolicy {
private final NRTCachingDirectory _nrtCachingDirectory;

public LuceneNRTCachingMergePolicy(NRTCachingDirectory nrtCachingDirectory) {
_nrtCachingDirectory = nrtCachingDirectory;
}

@Override
public MergeSpecification findMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext)
throws IOException {
SegmentInfos inMemorySegmentInfos = new SegmentInfos(segmentInfos.getIndexCreatedVersionMajor());
// Collect all segment commit infos that still have all files in memory
Set<String> cachedFiles = new HashSet<>(List.of(_nrtCachingDirectory.listCachedFiles()));
for (SegmentCommitInfo info : segmentInfos) {
for (String file : info.files()) {
if (!cachedFiles.contains(file)) {
break;
}
}
inMemorySegmentInfos.add(info);
}
return super.findMerges(mergeTrigger, inMemorySegmentInfos, mergeContext);
}
}

This file was deleted.

Loading
Loading