diff --git a/dev-tools/scripts/smokeTestRelease.py b/dev-tools/scripts/smokeTestRelease.py index 84978bbede3..1083a09d425 100644 --- a/dev-tools/scripts/smokeTestRelease.py +++ b/dev-tools/scripts/smokeTestRelease.py @@ -644,10 +644,14 @@ def verifyUnpacked(java, project, artifact, unpackPath, gitRevision, version, te textFiles.append('BUILD') for fileName in textFiles: - fileName += '.txt' - if fileName not in l: - raise RuntimeError('file "%s" is missing from artifact %s' % (fileName, artifact)) - l.remove(fileName) + fileNameTxt = fileName + '.txt' + fileNameMd = fileName + '.md' + if fileNameTxt in l: + l.remove(fileNameTxt) + elif fileNameMd in l: + l.remove(fileNameMd) + else: + raise RuntimeError('file "%s".[txt|md] is missing from artifact %s' % (fileName, artifact)) if project == 'lucene': if LUCENE_NOTICE is None: diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java index 27e28c08734..0e2a06708b4 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java @@ -544,11 +544,11 @@ final class DocumentsWriter implements Closeable, Accountable { dwptSuccess = true; } finally { subtractFlushedNumDocs(flushingDocsInRam); - if (!flushingDWPT.pendingFilesToDelete().isEmpty()) { + if (flushingDWPT.pendingFilesToDelete().isEmpty() == false) { putEvent(new DeleteNewFilesEvent(flushingDWPT.pendingFilesToDelete())); hasEvents = true; } - if (!dwptSuccess) { + if (dwptSuccess == false) { putEvent(new FlushFailedEvent(flushingDWPT.getSegmentInfo())); hasEvents = true; } @@ -582,6 +582,10 @@ final class DocumentsWriter implements Closeable, Accountable { flushingDWPT = flushControl.nextPendingFlush(); } + if (hasEvents) { + writer.doAfterSegmentFlushed(false, false); + } + // If deletes alone are consuming > 1/2 our RAM // buffer, force them all to apply now. This is to // prevent too-frequent flushing of a long tail of @@ -605,7 +609,7 @@ final class DocumentsWriter implements Closeable, Accountable { void subtractFlushedNumDocs(int numFlushed) { int oldValue = numDocsInRAM.get(); - while (!numDocsInRAM.compareAndSet(oldValue, oldValue - numFlushed)) { + while (numDocsInRAM.compareAndSet(oldValue, oldValue - numFlushed) == false) { oldValue = numDocsInRAM.get(); } assert numDocsInRAM.get() >= 0; @@ -726,10 +730,9 @@ final class DocumentsWriter implements Closeable, Accountable { static final class ApplyDeletesEvent implements Event { static final Event INSTANCE = new ApplyDeletesEvent(); - private int instCount = 0; + private ApplyDeletesEvent() { - assert instCount == 0; - instCount++; + // only one instance } @Override @@ -740,10 +743,9 @@ final class DocumentsWriter implements Closeable, Accountable { static final class ForcedPurgeEvent implements Event { static final Event INSTANCE = new ForcedPurgeEvent(); - private int instCount = 0; + private ForcedPurgeEvent() { - assert instCount == 0; - instCount++; + // only one instance } @Override diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java index df1b38c45fc..b051545841a 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java @@ -119,6 +119,10 @@ class DocumentsWriterFlushQueue { synchronized (this) { // finally remove the published ticket from the queue final FlushTicket poll = queue.poll(); + + // we hold the purgeLock so no other thread should have polled: + assert poll == head; + ticketCount.decrementAndGet(); assert poll == head; } diff --git a/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java b/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java index 82488416c38..c4e60db76dd 100644 --- a/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java +++ b/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java @@ -262,6 +262,8 @@ class FrozenBufferedUpdates { int totalSegmentCount = 0; long totalDelCount = 0; + boolean finished = false; + // Optimistic concurrency: assume we are free to resolve the deletes against all current segments in the index, despite that // concurrent merges are running. Once we are done, we check to see if a merge completed while we were running. If so, we must retry // resolving against the newly merged segment(s). Eventually no merge finishes while we were running and we are done. @@ -334,7 +336,7 @@ class FrozenBufferedUpdates { if (infoStream.isEnabled("BD")) { infoStream.message("BD", String.format(Locale.ROOT, - messagePrefix + "done apply del packet (%s) to %d segments; %d new deletes/updates; took %.3f sec", + messagePrefix + "done inner apply del packet (%s) to %d segments; %d new deletes/updates; took %.3f sec", this, segStates.length, delCount, (System.nanoTime() - iterStartNS) / 1000000000.)); } @@ -352,6 +354,13 @@ class FrozenBufferedUpdates { if (mergeGenCur == mergeGenStart) { + // Must do this while still holding IW lock else a merge could finish and skip carrying over our updates: + + // Record that this packet is finished: + writer.bufferedUpdatesStream.finished(this); + + finished = true; + // No merge finished while we were applying, so we are done! break; } @@ -367,9 +376,11 @@ class FrozenBufferedUpdates { iter++; } - // Record that this packet is finished: - writer.bufferedUpdatesStream.finished(this); - + if (finished == false) { + // Record that this packet is finished: + writer.bufferedUpdatesStream.finished(this); + } + if (infoStream.isEnabled("BD")) { String message = String.format(Locale.ROOT, "done apply del packet (%s) to %d segments; %d new deletes/updates; took %.3f sec", diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java index 2ef928bb8a7..26a25a59b08 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java @@ -642,12 +642,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { } } - void writeDocValuesUpdates(List infos) throws IOException { + void writeDocValuesUpdatesForMerge(List infos) throws IOException { boolean any = false; for (SegmentCommitInfo info : infos) { ReadersAndUpdates rld = get(info, false); if (rld != null) { any |= rld.writeFieldUpdates(directory, bufferedUpdatesStream.getCompletedDelGen(), infoStream); + rld.setIsMerging(); } } if (any) { @@ -4216,7 +4217,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { // Must move the pending doc values updates to disk now, else the newly merged segment will not see them: // TODO: we could fix merging to pull the merged DV iterator so we don't have to move these updates to disk first, i.e. just carry them // in memory: - readerPool.writeDocValuesUpdates(merge.segments); + readerPool.writeDocValuesUpdatesForMerge(merge.segments); // Bind a new segment name here so even with // ConcurrentMergePolicy we keep deterministic segment diff --git a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java index 630131ef5e7..dd49fcb9437 100644 --- a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java +++ b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java @@ -808,14 +808,16 @@ class ReadersAndUpdates { return true; } - /** Returns a reader for merge, with the latest doc values updates and deletions. */ - synchronized SegmentReader getReaderForMerge(IOContext context) throws IOException { - + synchronized public void setIsMerging() { // This ensures any newly resolved doc value updates while we are merging are // saved for re-applying after this segment is done merging: isMerging = true; assert mergingDVUpdates.isEmpty(); + } + + /** Returns a reader for merge, with the latest doc values updates and deletions. */ + synchronized SegmentReader getReaderForMerge(IOContext context) throws IOException { // We must carry over any still-pending DV updates because they were not // successfully written, e.g. because there was a hole in the delGens, diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexManyDocuments.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexManyDocuments.java new file mode 100644 index 00000000000..eb31e738856 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexManyDocuments.java @@ -0,0 +1,71 @@ +/* + * 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 java.util.concurrent.atomic.AtomicInteger; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.TestUtil; + +public class TestIndexManyDocuments extends LuceneTestCase { + + public void test() throws Exception { + Directory dir = newFSDirectory(createTempDir()); + IndexWriterConfig iwc = new IndexWriterConfig(); + iwc.setMaxBufferedDocs(TestUtil.nextInt(random(), 100, 2000)); + + int numDocs = atLeast(10000); + + final IndexWriter w = new IndexWriter(dir, iwc); + final AtomicInteger count = new AtomicInteger(); + Thread[] threads = new Thread[2]; + for(int i=0;i - - + + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-binaryfield.xml b/solr/core/src/test-files/solr/collection1/conf/schema-binaryfield.xml index 26a428d599d..f0f216b2b12 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-binaryfield.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-binaryfield.xml @@ -27,46 +27,10 @@ - - - - - - - - - - - - - - - - - - - diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml b/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml index 96adf2dd623..adea3c19a64 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml @@ -27,34 +27,10 @@ - - - - - - - - - - - - - + + + + @@ -94,7 +70,7 @@ - + - - - - - - - - - - - + + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-non-stored-docvalues.xml b/solr/core/src/test-files/solr/collection1/conf/schema-non-stored-docvalues.xml index f700e60a416..4e631becdf2 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-non-stored-docvalues.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-non-stored-docvalues.xml @@ -17,52 +17,15 @@ --> - - - - - - - - - - - - - - - + + + + - - - - - + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-not-required-unique-key.xml b/solr/core/src/test-files/solr/collection1/conf/schema-not-required-unique-key.xml index d38952f3183..ba8bbd5e801 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-not-required-unique-key.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-not-required-unique-key.xml @@ -23,7 +23,6 @@ --> - diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-replication1.xml b/solr/core/src/test-files/solr/collection1/conf/schema-replication1.xml index eeb51bdccd7..1a4e32e2645 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-replication1.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-replication1.xml @@ -27,7 +27,6 @@ - diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-replication2.xml b/solr/core/src/test-files/solr/collection1/conf/schema-replication2.xml index ade7bf54517..8b831272719 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-replication2.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-replication2.xml @@ -27,7 +27,6 @@ - diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-required-fields.xml b/solr/core/src/test-files/solr/collection1/conf/schema-required-fields.xml index 551ac1d9357..66add46c7b7 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-required-fields.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-required-fields.xml @@ -27,34 +27,10 @@ - - - - - - - - - - - - - + + + + @@ -77,7 +53,7 @@ - + - - - - + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-tokenizer-test.xml b/solr/core/src/test-files/solr/collection1/conf/schema-tokenizer-test.xml index 174787ef989..6c33504427f 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-tokenizer-test.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-tokenizer-test.xml @@ -27,27 +27,8 @@ more concise example. - - - - - - - - - + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-version-dv.xml b/solr/core/src/test-files/solr/collection1/conf/schema-version-dv.xml index 593b2a27881..375311a9b69 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-version-dv.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-version-dv.xml @@ -16,15 +16,14 @@ limitations under the License. --> - + id - - + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-version-indexed.xml b/solr/core/src/test-files/solr/collection1/conf/schema-version-indexed.xml index 06d6b656abe..4523d507ece 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-version-indexed.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-version-indexed.xml @@ -16,15 +16,14 @@ limitations under the License. --> - + id - - + diff --git a/solr/core/src/test/org/apache/solr/TestRandomDVFaceting.java b/solr/core/src/test/org/apache/solr/TestRandomDVFaceting.java index 46762ea006a..aaeab54d5fc 100644 --- a/solr/core/src/test/org/apache/solr/TestRandomDVFaceting.java +++ b/solr/core/src/test/org/apache/solr/TestRandomDVFaceting.java @@ -28,6 +28,8 @@ import org.apache.lucene.util.LuceneTestCase.Slow; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.schema.SchemaField; +import org.apache.solr.schema.TrieIntField; +import org.apache.solr.schema.IntPointField; import org.junit.BeforeClass; import org.junit.Test; import org.slf4j.Logger; @@ -39,18 +41,36 @@ import org.slf4j.LoggerFactory; * to the indexed facet results as if it were just another faceting method. */ @Slow +@SolrTestCaseJ4.SuppressPointFields(bugUrl="Test explicitly compares Trie to Points, randomization defeats the point") public class TestRandomDVFaceting extends SolrTestCaseJ4 { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); @BeforeClass public static void beforeTests() throws Exception { - // we need DVs on point fields to compute stats & facets - // but test also has hard coded assumptions about these field types *NOT* having DV when *NOT* points - // so use docvalue if and only if we are using points... - System.setProperty(NUMERIC_DOCVALUES_SYSPROP, System.getProperty(NUMERIC_POINTS_SYSPROP)); + // This tests explicitly compares Trie DV with non-DV Trie with DV Points + // so we don't want randomized DocValues on all Trie fields + System.setProperty(NUMERIC_DOCVALUES_SYSPROP, "false"); initCore("solrconfig-basic.xml","schema-docValuesFaceting.xml"); + + assertEquals("DocValues: Schema assumptions are broken", + false, h.getCore().getLatestSchema().getField("foo_i").hasDocValues()); + assertEquals("DocValues: Schema assumptions are broken", + true, h.getCore().getLatestSchema().getField("foo_i_dv").hasDocValues()); + assertEquals("DocValues: Schema assumptions are broken", + true, h.getCore().getLatestSchema().getField("foo_i_p").hasDocValues()); + + assertEquals("Type: Schema assumptions are broken", + TrieIntField.class, + h.getCore().getLatestSchema().getField("foo_i").getType().getClass()); + assertEquals("Type: Schema assumptions are broken", + TrieIntField.class, + h.getCore().getLatestSchema().getField("foo_i_dv").getType().getClass()); + assertEquals("Type: Schema assumptions are broken", + IntPointField.class, + h.getCore().getLatestSchema().getField("foo_i_p").getType().getClass()); + } int indexSize; diff --git a/solr/core/src/test/org/apache/solr/core/DirectoryFactoryTest.java b/solr/core/src/test/org/apache/solr/core/DirectoryFactoryTest.java index 35198b7786f..aa00342116d 100755 --- a/solr/core/src/test/org/apache/solr/core/DirectoryFactoryTest.java +++ b/solr/core/src/test/org/apache/solr/core/DirectoryFactoryTest.java @@ -55,20 +55,25 @@ public class DirectoryFactoryTest extends LuceneTestCase { rdf.init(new NamedList()); // No solr.data.home property set. Absolute instanceDir - assertEquals("/tmp/inst1/data", rdf.getDataHome(new CoreDescriptor("core_name", Paths.get("/tmp/inst1"), cp, zkAware))); + assertDataHome("/tmp/inst1/data", "/tmp/inst1", rdf, cc); // Simulate solr.data.home set in solrconfig.xml tag NamedList args = new NamedList(); args.add("solr.data.home", "/solrdata/"); rdf.init(args); - assertEquals("/solrdata/inst_dir/data", rdf.getDataHome(new CoreDescriptor("core_name", Paths.get("inst_dir"), cp, zkAware))); - + assertDataHome("/solrdata/inst_dir/data", "inst_dir", rdf, cc); + // solr.data.home set with System property, and relative path System.setProperty("solr.data.home", "solrdata"); rdf.init(new NamedList()); - assertEquals("/solr/home/solrdata/inst_dir/data", rdf.getDataHome(new CoreDescriptor("core_name", Paths.get("inst_dir"), cp, zkAware))); + assertDataHome("/solr/home/solrdata/inst_dir/data", "inst_dir", rdf, cc); // Test parsing last component of instanceDir, and using custom dataDir - assertEquals("/solr/home/solrdata/myinst/mydata", rdf.getDataHome(new CoreDescriptor("core_name", Paths.get("/path/to/myinst"), cp, zkAware, "dataDir", "mydata"))); + assertDataHome("/solr/home/solrdata/myinst/mydata", "/path/to/myinst", rdf, cc, "dataDir", "mydata"); + } + + private void assertDataHome(String expected, String instanceDir, RAMDirectoryFactory rdf, MockCoreContainer cc, String... properties) throws IOException { + String dataHome = rdf.getDataHome(new CoreDescriptor("core_name", Paths.get(instanceDir), cc.containerProperties, cc.isZooKeeperAware(), properties)); + assertEquals(Paths.get(expected).toAbsolutePath(), Paths.get(dataHome).toAbsolutePath()); } diff --git a/solr/core/src/java/org/apache/solr/common/cloud/ReplicaPosition.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ReplicaPosition.java similarity index 100% rename from solr/core/src/java/org/apache/solr/common/cloud/ReplicaPosition.java rename to solr/solrj/src/java/org/apache/solr/common/cloud/ReplicaPosition.java