mirror of https://github.com/apache/lucene.git
SOLR-5944: Cleanup comments and logging, use NoMergePolicy instead of LogDocMergePolicy
This commit is contained in:
parent
d5ef026f84
commit
6358afbea6
|
@ -317,7 +317,6 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
}
|
||||
}
|
||||
|
||||
Document luceneDocument = cmd.getLuceneDocument();
|
||||
Term idTerm = getIdTerm(cmd);
|
||||
|
||||
RefCounted<IndexWriter> iw = solrCoreState.getIndexWriter(core);
|
||||
|
|
|
@ -214,7 +214,7 @@ public class AtomicUpdateDocumentMerger {
|
|||
Set<String> segmentSortingFields = null;
|
||||
try {
|
||||
IndexWriter iw = holder.get();
|
||||
fieldNamesFromIndexWriter = iw.getFieldNames();
|
||||
fieldNamesFromIndexWriter = iw.getFieldNames(); // This shouldn't be needed once LUCENE-7659 is resolved
|
||||
segmentSortingFields = iw.getConfig().getIndexSortFields();
|
||||
} finally {
|
||||
holder.decref();
|
||||
|
|
|
@ -1036,13 +1036,13 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
|
||||
VersionBucket bucket = vinfo.bucket(bucketHash);
|
||||
|
||||
long dependentVersionFound = -1; // Last found version for a dependent update; applicable only for in-place updates; useful for logging later
|
||||
// if this is an inplace update, check and wait if we should be waiting for a dependent update, before
|
||||
// entering the synchronized block
|
||||
long dependentVersionFound = -1;
|
||||
// if this is an in-place update, check and wait if we should be waiting for a previous update (on which
|
||||
// this update depends), before entering the synchronized block
|
||||
if (!leaderLogic && cmd.isInPlaceUpdate()) {
|
||||
dependentVersionFound = waitForDependentUpdates(cmd, versionOnUpdate, isReplayOrPeersync, bucket);
|
||||
if (dependentVersionFound == -1) {
|
||||
// it means in leader, the document has been deleted by now. drop this update
|
||||
// it means the document has been deleted by now at the leader. drop this update
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
@ -1135,7 +1135,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
assert fetchedFromLeader instanceof AddUpdateCommand;
|
||||
// Newer document was fetched from the leader. Apply that document instead of this current in-place update.
|
||||
log.info("In-place update of {} failed to find valid lastVersion to apply to, forced to fetch full doc from leader: {}",
|
||||
idBytes.utf8ToString(), (fetchedFromLeader == null? null: ((AddUpdateCommand)fetchedFromLeader).solrDoc));
|
||||
idBytes.utf8ToString(), fetchedFromLeader);
|
||||
|
||||
// Make this update to become a non-inplace update containing the full document obtained from the leader
|
||||
cmd.solrDoc = ((AddUpdateCommand)fetchedFromLeader).solrDoc;
|
||||
|
@ -1273,7 +1273,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
return -1;
|
||||
} else {
|
||||
assert missingUpdate instanceof AddUpdateCommand;
|
||||
log.info("Fetched the document: {}", ((AddUpdateCommand)missingUpdate).getSolrInputDocument());
|
||||
log.debug("Fetched the document: {}", ((AddUpdateCommand)missingUpdate).getSolrInputDocument());
|
||||
versionAdd((AddUpdateCommand)missingUpdate);
|
||||
log.info("Added the fetched document, id="+((AddUpdateCommand)missingUpdate).getPrintableId()+", version="+missingUpdate.getVersion());
|
||||
}
|
||||
|
|
|
@ -30,7 +30,6 @@ import java.util.concurrent.ExecutorService;
|
|||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.lucene.index.LogDocMergePolicy;
|
||||
import org.apache.lucene.util.LuceneTestCase.Slow;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.solr.client.solrj.SolrClient;
|
||||
|
@ -53,7 +52,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
|
|||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.common.util.ExecutorUtil;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.index.LogDocMergePolicyFactory;
|
||||
import org.apache.solr.index.NoMergePolicyFactory;
|
||||
import org.apache.solr.update.processor.DistributedUpdateProcessor;
|
||||
import org.apache.solr.util.DefaultSolrThreadFactory;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
@ -82,8 +81,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
|
|||
|
||||
// we need consistent segments that aren't re-ordered on merge because we're
|
||||
// asserting inplace updates happen by checking the internal [docid]
|
||||
systemSetPropertySolrTestsMergePolicy(LogDocMergePolicy.class.getName());
|
||||
systemSetPropertySolrTestsMergePolicyFactory(LogDocMergePolicyFactory.class.getName());
|
||||
systemSetPropertySolrTestsMergePolicyFactory(NoMergePolicyFactory.class.getName());
|
||||
|
||||
initCore(configString, schemaString);
|
||||
|
||||
|
@ -289,6 +287,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
|
|||
SolrDocument doc = NONLEADERS.get(0).getById(String.valueOf(0), params("distrib", "false"));
|
||||
assertNull("This doc was supposed to have been deleted, but was: " + doc, doc);
|
||||
|
||||
log.info("reorderedDBQIndividualReplicaTest: This test passed fine...");
|
||||
clearIndex();
|
||||
commit();
|
||||
}
|
||||
|
@ -326,7 +325,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
|
|||
for (int id : ids) {
|
||||
// NOTE: in rare cases, this may be setting the value to 0, on a doc that
|
||||
// already had an init value of 0 -- which is an interesting edge case, so we don't exclude it
|
||||
final float multiplier = random().nextBoolean() ? -5.0F : 5.0F;
|
||||
final float multiplier = r.nextBoolean() ? -5.0F : 5.0F;
|
||||
final float value = r.nextFloat() * multiplier;
|
||||
assert -5.0F <= value && value <= 5.0F;
|
||||
valuesList.set(id, value);
|
||||
|
@ -334,7 +333,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
|
|||
log.info("inplace_updatable_float: " + valuesList);
|
||||
|
||||
// update doc w/ set
|
||||
Collections.shuffle(ids, random()); // so updates aren't applied in index order
|
||||
Collections.shuffle(ids, r); // so updates aren't applied in index order
|
||||
for (int id : ids) {
|
||||
index("id", id, "inplace_updatable_float", map("set", valuesList.get(id)));
|
||||
}
|
||||
|
@ -350,12 +349,12 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
|
|||
|
||||
// update doc, w/increment
|
||||
log.info("Updating the documents...");
|
||||
Collections.shuffle(ids, random()); // so updates aren't applied in the same order as our 'set'
|
||||
Collections.shuffle(ids, r); // so updates aren't applied in the same order as our 'set'
|
||||
for (int id : ids) {
|
||||
// all incremements will use some value X such that 20 < abs(X)
|
||||
// thus ensuring that after all incrememnts are done, there should be
|
||||
// 0 test docs matching the query inplace_updatable_float:[-10 TO 10]
|
||||
final float inc = (r.nextBoolean() ? -1.0F : 1.0F) * (random().nextFloat() + (float)atLeast(20));
|
||||
final float inc = (r.nextBoolean() ? -1.0F : 1.0F) * (r.nextFloat() + (float)atLeast(20));
|
||||
assert 20 < Math.abs(inc);
|
||||
final float value = valuesList.get(id) + inc;
|
||||
assert value < -10 || 10 < value;
|
||||
|
@ -371,6 +370,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
|
|||
// existing sort & fl that we want...
|
||||
params),
|
||||
luceneDocids, valuesList);
|
||||
log.info("docValuesUpdateTest: This test passed fine...");
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -458,8 +458,8 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
|
|||
// do an initial (non-inplace) update to ensure both the float & int fields we care about have (any) value
|
||||
// that way all subsequent atomic updates will be inplace
|
||||
currentVersion = addDocAndGetVersion("id", 100,
|
||||
"inplace_updatable_float", map("set", random().nextFloat()),
|
||||
"inplace_updatable_int", map("set", random().nextInt()));
|
||||
"inplace_updatable_float", map("set", r.nextFloat()),
|
||||
"inplace_updatable_int", map("set", r.nextInt()));
|
||||
LEADER.commit();
|
||||
|
||||
// get the internal docids of id=100 document from the three replicas
|
||||
|
@ -477,7 +477,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
|
|||
assertEquals(sdoc.toString(), title, sdoc.get("title_s"));
|
||||
assertEquals(sdoc.toString(), version, sdoc.get("_version_"));
|
||||
|
||||
if(random().nextBoolean()) {
|
||||
if(r.nextBoolean()) {
|
||||
title = "newtitle100";
|
||||
currentVersion = addDocAndGetVersion("id", 100, "title_s", title, "inplace_updatable_float", inplace_updatable_float); // full indexing
|
||||
assertTrue(currentVersion > version);
|
||||
|
@ -523,6 +523,8 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
|
|||
// assert that the internal docid for id=100 document remains same, in each replica, as before
|
||||
LEADER.commit(); // can't get (real) [docid] from the tlogs, need to force a commit
|
||||
assertTrue("Earlier: "+docids+", now: "+getInternalDocIds("100"), docids.equals(getInternalDocIds("100")));
|
||||
|
||||
log.info("ensureRtgWorksWithPartialUpdatesTest: This test passed fine...");
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -580,7 +582,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
|
|||
|
||||
// re-order the updates for NONLEADER 0
|
||||
List<UpdateRequest> reorderedUpdates = new ArrayList<>(updates);
|
||||
Collections.shuffle(reorderedUpdates, random());
|
||||
Collections.shuffle(reorderedUpdates, r);
|
||||
List<Future<UpdateResponse>> updateResponses = new ArrayList<>();
|
||||
for (UpdateRequest update : reorderedUpdates) {
|
||||
AsyncUpdateWithRandomCommit task = new AsyncUpdateWithRandomCommit(update, NONLEADERS.get(0), random().nextLong());
|
||||
|
@ -650,7 +652,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
|
|||
|
||||
// re-order the updates for NONLEADER 0
|
||||
List<UpdateRequest> reorderedUpdates = new ArrayList<>(updates);
|
||||
Collections.shuffle(reorderedUpdates, random());
|
||||
Collections.shuffle(reorderedUpdates, r);
|
||||
List<Future<UpdateResponse>> updateResponses = new ArrayList<>();
|
||||
for (UpdateRequest update : reorderedUpdates) {
|
||||
AsyncUpdateWithRandomCommit task = new AsyncUpdateWithRandomCommit(update, NONLEADERS.get(0), random().nextLong());
|
||||
|
@ -1103,7 +1105,6 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
|
|||
shardToJetty.get(SHARD1).get(1).jetty.getDebugFilter().addDelay(
|
||||
"Waiting for dependant update to timeout", 2, 8000);
|
||||
|
||||
long seed = random().nextLong(); // seed for randomization within the threads
|
||||
ExecutorService threadpool =
|
||||
ExecutorUtil.newMDCAwareFixedThreadPool(updates.size() + 1, new DefaultSolrThreadFactory(getTestName()));
|
||||
for (UpdateRequest update : updates) {
|
||||
|
|
Loading…
Reference in New Issue