corecursion commented on code in PR #15015: URL: https://github.com/apache/lucene/pull/15015#discussion_r2249017404
########## lucene/core/src/java/org/apache/lucene/index/MultiIndexMergeScheduler.java: ########## @@ -0,0 +1,203 @@ +/* + * 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.lucene.index; + +import java.io.IOException; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.InfoStream; + +class MultiIndexMergeScheduler extends MergeScheduler { + private final Directory directory; + private final CombinedMergeScheduler combinedMergeScheduler; + + /** The main MultiIndexMergeScheduler constructor -- use this one. */ + public MultiIndexMergeScheduler(Directory directory) { + this.directory = directory; + this.combinedMergeScheduler = CombinedMergeScheduler.singleton; + } + + /** Alternate MultiIndexMergeScheduler constructor for unit testing or tenant grouping. */ + public MultiIndexMergeScheduler( + Directory directory, CombinedMergeScheduler combinedMergeScheduler) { + this.directory = directory; + this.combinedMergeScheduler = combinedMergeScheduler; + } + + public Directory getDirectory() { + return directory; + } + + public CombinedMergeScheduler getCombinedMergeScheduler() { + return combinedMergeScheduler; + } + + @Override + public void merge(MergeSource mergeSource, MergeTrigger trigger) throws IOException { + var taggedMergeSource = + new CombinedMergeScheduler.TaggedMergeSource(mergeSource, this.directory); + this.combinedMergeScheduler.merge(taggedMergeSource, trigger); + } + + @Override + public Directory wrapForMerge(MergePolicy.OneMerge merge, Directory in) { + return this.combinedMergeScheduler.wrapForMerge(merge, in); + } + + /** Close this scheduler for one directory/index. Called automatically by IndexWriter. */ + @Override + public void close() throws IOException { + this.combinedMergeScheduler.close(this.directory); + } + + // We created this method because we cannot easily override the initialize() method + // in ConcurrentMergeScheduler. We don't need the initDynamicDefaults() part in the + // initialize() method, and only need the setInfoStream(). + public void setInfoStream(InfoStream infoStream) { + this.combinedMergeScheduler.setInfoStream(infoStream); + } + + /** + * CombinedMergeScheduler is used internally by MultiIndexMergeScheduler to balance resources + * across multiple indices. Normally you don't need to use this. + * + * <p>For testing purposes, or if partitioning of tenants into groups is needed for some reason, a + * CombinedMergeScheduler can be provided to the MultiIndexMergeScheduler constructor. + * + * <p>CombinedMergeScheduler should <b><i>not</i></b> be passed directly to IndexWriter. + */ + static class CombinedMergeScheduler extends ConcurrentMergeScheduler { + static final CombinedMergeScheduler singleton = new CombinedMergeScheduler(); + + // A filter pattern + static class TaggedMergeSource implements MergeScheduler.MergeSource { + private final MergeScheduler.MergeSource in; + private final Directory directory; + + TaggedMergeSource(MergeScheduler.MergeSource in, Directory directory) { + this.in = in; + this.directory = directory; + } + + public Directory getDirectory() { + return this.directory; + } + + @Override + public MergePolicy.OneMerge getNextMerge() { + return this.in.getNextMerge(); + } + + @Override + public void onMergeFinished(MergePolicy.OneMerge merge) { + this.in.onMergeFinished(merge); + } + + @Override + public boolean hasPendingMerges() { + return this.in.hasPendingMerges(); + } + + @Override + public void merge(MergePolicy.OneMerge merge) throws IOException { + this.in.merge(merge); + } + } + + // Same functionality as calling initialize() in MergeScheduler, used by + // MultiIndexMergeScheduler to + // initialize the wrapped per index CombinedMergeScheduler. + public void setInfoStream(InfoStream infoStream) { + this.infoStream = infoStream; + } + + // The implementation is copied from sync() in ConcurrentMergeScheduler in Lucene code, with + // only one additional check: + // ((InstrumentedMergeThread) t).getDirectory().equals(directory). + public void close(Directory directory) { + boolean interrupted = false; + try { + while (true) { + MergeThread toSync = null; + synchronized (this) { + for (MergeThread t : this.mergeThreads) { + // In case a merge thread is calling us, don't try to sync on + // itself, since that will never finish! + if (t.isAlive() + && t != Thread.currentThread() + // Only wait for merge threads for the current index to finish + && ((InstrumentedMergeThread) t).getDirectory().equals(directory)) { + toSync = t; + break; + } + } + } + if (toSync != null) { + try { + toSync.join(); + } catch ( + @SuppressWarnings("unused") + InterruptedException ie) { + // ignore this Exception, we will retry until all threads are dead + interrupted = true; + } + } else { + break; + } + } + } finally { + // finally, restore interrupt status: + if (interrupted) { + Thread.currentThread().interrupt(); + } + } + } + + @Override + protected synchronized ConcurrentMergeScheduler.MergeThread getMergeThread( Review Comment: removed `getMergeThread()` in [1f6e301](https://github.com/apache/lucene/pull/15015/commits/1f6e3016991f9b475f09ddcf165248a3945188fb) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org