You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by "Ishan Chattopadhyaya (JIRA)" <ji...@apache.org> on 2016/10/05 07:50:21 UTC

[jira] [Updated] (SOLR-5944) Support updates of numeric DocValues

     [ https://issues.apache.org/jira/browse/SOLR-5944?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ishan Chattopadhyaya updated SOLR-5944:
---------------------------------------
    Attachment: SOLR-5944.patch

Updated patch, brought up to master. Here are my replies inline (not all of them, but I'll keep editing this comment to provide all replies).


Ok -- it took a while, but here's my notes after reviewing the latest patch....


{panel:title=DistributedUpdateProcessor}

* waitForDependentUpdates
** I know you & shalin went back and forth a bit on the wait call (ie: wait(100) with max retries vs wait(5000)) but i think the way things settled out {{bucket.wait(waitTimeout.timeLeft(TimeUnit.MILLISECONDS));}} would be better then a generic {{wait(5000)}} [Ishan]FIXED[/Ishan]
*** consider the scenerio where: the dependent update is never going to come; a spurious notify/wake happens during the first "wait" call @ 4950ms; the lookupVersion call takes 45ms.  Now we've only got 5ms left on our original TimeOut, but we _could_ wind up "wait"ing another full 5s (total of 10s) unless we get another spurrious notify/wake inthe mean time.
** {{log.info("Fetched the update: " + missingUpdate);}} that's a really good candidate for templating since the AddUpdateCommand.toString() could be expensive if log.info winds up being a no-op (ie: {{log.info("Fetched the update: \{\}", missingUpdate);}}) [Ishan]FIXED[/Ishan]

* fetchMissingUpdateFromLeader
** In response to a previous question you said...{quote}
[FIXED. Initially, I wanted to fetch all missing updates, i.e. from what we have till what we want. Noble suggested that fetching only one at a time makes more sense.]
{quote} ... but from what i can tell skimming RTGC.processGetUpdates() it's still possible that multiple updates will be returned, notably in the case where: {{// Must return all delete-by-query commands that occur after the first add requested}}.  How is that possibility handled in the code paths that use fetchMissingUpdateFromLeader?
*** that seems like a scenerio that would be really easy to test for -- similar to how outOfOrderDeleteUpdatesIndividualReplicaTest works
** {{assert ((List<List>) missingUpdates).size() == 1: "More than 1 update ...}}
*** based on my skimming of the code, an empty list is just as possible, so the assertion is missleading (ideally it should say how many updates it got, or maybe toString() the whole List ?)

{panel}


{panel:title=AtomicUpdateDocumentMerger}

* isSupportedFieldForInPlaceUpdate
** javadocs

* getFieldNamesFromIndex
** javadocs
** method name seems VERY missleading considering what it does [Ishan]Changed it to getSearcherNonStoredDVs[/Ishan]

* isInPlaceUpdate
** javadocs should be clear what hapens to inPlaceUpdatedFields if result is false (even if answer is "undefined"
** based on usage, wouldn't it be simplier if instead of returning a boolean, this method just returned a (new) Set of inplace update fields found, and if the set is empty that means it's not an in place update? [Ishan]FIXED[/Ishan]
** isn't getFieldNamesFromIndex kind of an expensive method to call on every AddUpdateCommand ?
*** couldn't this list of fields be created by the caller and re-used at least for the entire request (ie: when adding multiple docs) ? [Ishan]The set returned is precomputed upon the opening of a searcher. The only cost I see is to create a new unmodifiableSet every time. I'd prefer to take up this optimization later, if needed.[/Ishan]
** {{if (indexFields.contains(fieldName) == false && schema.isDynamicField(fieldName))}}
*** why does it matter one way or the other if it's a dynamicField? [Ishan]Changed the logic to check in the IW for presence of field. Added a comment: "// if dynamic field and this field doesn't exist, DV update can't work"[/Ishan]
** the special {{DELETED}} sentinal value still isn't being checked against the return value of {{getInputDocumentFromTlog}} [Ishan]Not using getInputDocumentFromTlog call anymore[/Ishan]
** this method still seems like it could/should do "cheaper" validation (ie: not requiring SchemaField object creation, or tlog lookups) first.  (Ex: the set of supported atomic ops are checked after isSupportedFieldForInPlaceUpdate & a possible read from the tlog). [Ishan]FIXED[/Ishan]
*** My suggested rewrite would be something like...{code}
Set<String> candidateResults = new HashSet<>();
// first pass, check the things that are virtually free,
// and bail out early if anything is obviously not a valid in-place update
for (String fieldName : sdoc.getFieldNames()) {
  if (schema.getUniqueKeyField().getName().equals(fieldName)
      || fieldName.equals(DistributedUpdateProcessor.VERSION_FIELD)) {
    continue;
  }
  Object fieldValue = sdoc.getField(fieldName).getValue();
  if (! (fieldValue instanceof Map) ) {
    // not even an atomic update, definitely not an in-place update
    return Collections.emptySet();
  }
  // else it's a atomic update map...
  for (String op : ((Map<String, Object>)fieldValue).keySet()) {
    if (!op.equals("set") && !op.equals("inc")) {
      // not a supported in-place update op
      return Collections.emptySet();
    }
  }
  candidateResults.add(fieldName);
}
if (candidateResults.isEmpty()) {
  return Collections.emptySet();
}
// now do the more expensive checks...
Set<String> indexFields = getFieldNamesFromIndex(cmd.getReq().getCore());
SolrInputDocument rtDoc = null; // will lazy read from tlog if needed
for (String fieldName : candidateResults) {
  SchemaField schemaField = schema.getField(fieldName);
  // TODO: check isSupportedFieldForInPlaceUpdate
  // TODO: check copyfields
  // TODO: check indexFields, if not there...
     // TODO: init rtDoc if null
     // TODO: check rtDoc
  // ...if any of these checks fail, immediately return Collections.emptySet()
}
return candidateResults;
{code}

* doInPlaceUpdateMerge
** jdocs should make it clear that updatedFields will be modified, and that the caller should have already ensured they are valid acording to isSupportedFieldForInPlaceUpdate(...) [Ishan]Added a note: "updatedFields passed into the method can be changed, i.e. the version field can be added to the set."[/Ishan]
*** either that, or make isInPlaceUpdate include VERSION_FIELD, and have this method assert that it's there.
** {{if (docid >= 0)}} [Ishan]FIXED: Bailing out to do a full atomic update here[/Ishan]
*** what if it's not? should that trigger an error? ... even if it's "ok" a comment as to *why* it's ok and what is expected to happen instead down the flow of the code would be helpful here.
** {{// Copy over all supported DVs from oldDocument to partialDoc}}
*** why are we copying all supported DVs over? [Ishan]If we update dv1 in one update, and then update dv2, and again update dv1 (without commits in between), the last update would fetch from the tlog the partial doc for dv2 update. If that doc doesn't copy over the previous updates to dv1, then a full resolution (by following previous pointers) would need to be done to calculate the dv1 value. Hence, I decided to copy over older DV updates.[/Ishan]
*** can't we loop over updatedFields and only copy the dv fields we _need_ over?
*** and if we do that, can't we skip the SchemaField creation & isSupportedFieldForInPlaceUpdate(...) check (since isInPlaceUpdate(...) should have already checked that)
** {{if (isSupportedFieldForInPlaceUpdate(schemaField) || fieldName.equals(schema.getUniqueKeyField().getName()))}}
*** fetch the uniqueKey field name outside of the loop, so we're not making 2 method calls every iteration to get it [Ishan]FIXED[/Ishan]
*** if the isSupportedFieldForInPlaceUpdate(...) really is still neccessary: swap the order of the OR so we do the cheap equality test first.
** in response to a previous suggestion i made about adding explicit unit tests to this method, you said: {quote}
[I've refactored this method to make it much simpler and to now call the original merge method. I don't think we need specific tests for this method any longer.]
{quote}... but i very much disagree.  There's still enough complexity here (paticularly with copying old docvals, the possibility that a previous update from the tlog may not have included all the fields we need to udpate now) that i think some unit tests like i described (ahnd creating a sequence of udpates, checking the expected result of merging, and the 'committing' and checking we still get the same results) would still be very useful. [Ishan]TODO[/Ishan]

{panel}

{panel:title=RealTimeGetComponent}

* SearcherInfo
** still has no javadocs

* resolveFullDocument
** I made this suggestion regarding simplifying the callers of this method: {quote}
hy not consolidate those cases into one block of code using (a modified) resolveFullDocument which can start with a call to toSolrDoc(...) and then return immediately if the entry is UpdateLog.ADD ?{quote}. your response was...{quote}[FIXED: but kept the call to toSolrDoc() call outside the resolveFullDocument]{quote}  I explaind why i thought this refactoring simplified the calling code, can you please elaborate / explain why you think the code is better w/o this refactoring? [Ishan]One of the callers of this resolveFullDocument() (RTGC.process()) is calling with a SolrDocument by calling toSolrDoc() on the tlog SID, another (getInputDocumentFromTlog()) is calling this with the tlog SID directly. It wasn't clear to me how to elegantly consolidate these two calls with a common method, and hence kept this SolrDocument vs SolrInputDocument conversion/obtaining outside this method, and used the SolrDocumentBase as a return type.[/Ishan]

* reopenRealtimeSearcherAndGet [Ishan]As I was tackling simultaneous deletions/DBQs, it became clear that this was not a fatal error that deserved an exception. When a doc isn't found during the resolution process, it must be a case of deletes happening. Hence, I decided to bail out from reopenRealtimeSearcherAndGet() with a null (not exception) and let the caller handle the situation.[/Ishan]
** i'm confused about the "throw {{INVALID_STATE}} vs return null" change to this method.
*** in the previous version of this method, it would never return null - instead it threw {{INVALID_STATE}} exception.
*** I asked you about this, and your response was that it was definitely an (unusual) fatal error.
*** But in the latest version of the patch, you've cahnged the method so that it _does_ return null in these cases, and every place it's called is now responsible to checking the reuslt for null, and throwing {{INVALID_STATE}}
**** There's some subtle variation in the {{INVALID_STATE}} msgs, but it's not clear if that's actually intention, or a copy/paste oversight.
** why make this change? why isn't reopenRealtimeSearcherAndGet still throwing the exception itself?
** regardless of who throws these exceptions, we have an {{idTerm}} to work with, so the {{new String(...)}} should probably just be replaced with {{idTerm.text()}}

* getInputDocument
** I'm confused by the permutations of this method..
*** the meat of the implementation is now in a 5 arg method: {{getInputDocument(SolrCore core, BytesRef idBytes, boolean avoidRetrievingStoredFields,Set<String> onlyTheseNonStoredDVs, boolean resolveFullDocument}}
*** you added a 3 arg helper that hardcodes {{avoidRetrievingStoredFields=false}} and {{onlyTheseNonStoredDVs=null}}
*** you removed the existing 2 arg method which was {{getInputDocument(SolrCore core, BytesRef idBytes)}} and updated those callers to use the 3 arg helper
**** in the process i think you broken DocBasedVersionConstraintsProcessorFactory, but i'll come back to that later.
** why not leave the 2 arg helper for backcompat, and any new/modified code that needs more control can use the 5 arg method? [Ishan]FIXED[/Ishan]
*** the 2 arg method should almost certainly delgate to {{getInputDocument(core,idByes,false,null,true}} ... see comments below regarding DocBasedVersionConstraintsProcessorFactory [Ishan]FIXED[/Ishan]

{panel}


{panel:title=DocBasedVersionConstraintsProcessorFactory}

* replacing {{getInputDocument(core,id}} with {{getInputDocument(core,id,false}} smells wrong here [Ishan]FIXED: calling 2 arg ctor, that calls true[/Ishan]
** and ditto for {{getInputDocumentFromTlog(core,id}} -> {{getInputDocumentFromTlog(core,id,false}}

* prior to this patch, there was no such thing as an inplace update -- any call to {{getInputDocument*}} would by definition return a "full document"

* now that the concept of an inplace update exists, and these {{getInputDocument*}} methods take a new third arg, shouldn't all existing calls to these methods pass {{true}} for the third arg to ensure conistent behavior?
** unless of course, there is specific reason why we _know_ some caller code doesn't care about resolving the full document -- in which case there should be a comment on the call as to why {{false}} is hardcoded. (but that doesn't seem to be the case here from my skimming of the usage)

{panel}

{panel:title=DocumentBuilder}

* method javadocs should explain *how* the forInPlaceUpdate param(s) will be used (ie: why does it matter?) [Ishan]FIXED[/Ishan]

* addField
** what if someone passes an IndexableField "val" that is *not* a NumericDocValuesField, and "forInPlaceUpdate=true" ... shouldn't that be invalid? (ie: trip an assert?) [Ishan]FIXED[/Ishan]
** why do we need to check {{uniqueKeyField != null && f.name().equals(uniqueKeyField.getName())}} on every IndexableField ?
*** why not just short circut out at the begining of the method if {{forInPlaceUpdate && field.equals(uniqueKeyField)}} ?
*** or better still: make the caller do it and eliminate the uniqueKeyField param (only 3 places this method is called, and in one of them we're garunteed {{false==forInPlaceUpdate}} so no check would be needed there anyway) [Ishan]FIXED[/Ishan]
** if {{true==forInPlaceUpdate}} and {{createFields(...)}} returns anything that is *not* a NumericDocValuesField (or returns more then one) shouldn't that trip an assert or something? (ie: doesn't that mean this SchemaField isn't valid for using with an in place update, and the code shouldn't have gotten this far?) [Ishan]This is fine, since createFields() generates both NDV and non NDV fields for an indexable field, and the intent is to drop the non NDV one. Added a comment to this effect[/Ishan]
** in general it seems like this method should just be something much simpler that like...{code}
private static void addField(Document doc, SchemaField field, Object val, float boost, boolean forInPlaceUpdate) {
  if (val instanceof IndexableField) {
    assert ( (val instanceof NumericDocValuesField) || ! forInPlaceUpdate)
    // set boost to the calculated compound boost
    ((Field)val).setBoost(boost);
    doc.add((Field)val);
    return;
  }
  int numvals = 0;
  for (IndexableField f : field.getType().createFields(field, val, boost)) {
    if (null == f) continue; // null fields are not added
    assert ( (f instanceof NumericDocValuesField) || ! forInPlaceUpdate)
    doc.add((Field) f);
    numvals++;
  }
  assert ((numvals <= 1) || ! forInPlaceUpdate)
}
{code}
[Ishan]Did a refactoring that seems simple, similar to your suggestion[/Ishan]

* toDocument
** {{// Do not copy this field if this document is to be used for an in-place update, ... and this is the uniqueKey field}}
*** why not?
*** i'm guessing this is just a small optimization? because the uniqueKey can't change so no need to "update" the copyField?
*** if so, the comment should make that reasoning clear [Ishan]FIXED: updated the comment[/Ishan]
** {{We don't need to do this if this document is to be used for in-place updates}}
*** again: comment should explain _why_ we don't need to do this.

{panel}


{panel:title=AddUpdateCommand}

* prevVersion
** why can't this comment be a javadoc comment?
** comment/javadoc should explain what special values (like -1) mean
[Ishan]FIXED[/Ishan]

* getLuceneDocument(boolean)
** javadocs [Ishan]FIXED[/Ishan]
* getLuceneDocument()
** javadocs should be updated to indicate (default) behavior compared to one arg version [Ishan]FIXED[/Ishan]

{panel}


{panel:title=SolrDocumentBase}

* why does containsKey need declared here? it's already part of Map interface [Ishan]FIXED[/Ishan]
* should be able to drop this file from patch

{panel}


{panel:title=XMLLoader}

* whitespace only changes ... drop from patch.

{panel}


{panel:title=DirectUpdateHandler2}

* WTF: printStackTrace [Ishan]FIXED[/Ishan]

* {{solrCoreState.getUpdateLock()}} usage
** I don't know why I didn't ask about this last time, but is this really the appropriate/only lock to use here? [Ishan]Removed the updateLock use here by calling the internally atomic writer.updateDocValues() method. I don't remember why I didn't use it in the first place, I think I couldn't find such a method back then, or could be oversight[/Ishan]
** I'm looking at the other usage of this lock, notably the comments in {{DUH2.deleteByQuery(...)}} and it seems like using this lock may be _neccessary_ to prevent a commit from happening in the middle of multiple in-place docvalue updates ... but is _sufficient_ ?
** notably: i'm wondering what happens if doc#X gets an in-place update of N fields and concurrently some oher client sends a full replacement of doc#X (because DUH2 doesn't wrap existing {{writer.updateDocument}} calls in any synchronization on the {{solrCoreState.getUpdateLock()}} ?
** So imagine the following sequence of ops:
*** Thread #1 enters {{DUH2.doNormalUpdate}} to do an in-place update of doc#X with 2 docvalues fields
*** Thread #2 enters {{DUH2.doNormalUpdate}} to do a full replacement of doc#X
*** Thread #1 acquires updateLock
*** Thread #1 calls {{writer.updateNumericDocValue}} on 1st field
*** Thread #2 calls {{writer.updateDocument}} on doc#x
*** Thread #1 calls {{writer.updateNumericDocValue}} on 2nd field
*** Thread #1 releases updateLock
** Do we need to either wrap all {{writer.updateDocument}} calls in the updateLock, or use some sort of secondary fineer grained "per-uniqueKey" locking as well to prevent bad stuff from happening in sequences like this? [Ishan]There was writer.updateDocValues() that I overlooked in the beginning, or was not present back then.[/Ishan]

{panel}


{panel:title=UpdateLog}

* Please add some javadocs to the new {{*_IDX}} constants you added explaining what they are for -- notably which ones are only relevant if FLAGS_IDX includes UPDATE_INPLACE [Ishan]Fixed[/Ishan]

* applyPartialUpdates
** I had some concerns in my last patch review about the sync block in this method, and how lookupLogs is used -- but i don't see any response to those concerns in previous comments, or any relevant edits to this method? [Ishan]TODO[/Ishan]

* getEntryFromTLog
** in repsonse to a question i had about the {{catch (Exception | Error ex)}} block, you said...{quote}
..If the seek results in an exception due to unable to deserialize the tlog entry, we can ignore it, since it just means we were looking up the wrong tlog. I have added a comment to this effect in the catch block.{quote}...but that debug msg doesn't make it clear that this is "ok" .... if an end user sees {{DEBUG: Exception reading the log...}} in their log file, that seems like something very bad ... what are they suppose to do with that information? [Ishan]Added a "(this is expected, don't worry)" to the exception message.[/Ishan]

* updateCommandFromTlog
** needs javadocs
** why can't this method conditionally set the {{prevVersion}} so that {{case UpdateLog.ADD}} code paths (in both this class, and in other switch/case statements) be refactored to also use this method? [Ishan]FIXED: Named it addUpdateCommandFromTlog()[/Ishan]
** If it's only going to be valid for UPDATE_INPLACE entries, that should be clear from the method name and asserted in case someone tries to miss-use it.
*** even if it's valid for both ADD and UPDATE_INPLACE, that should be clear from the method name (ie: convertAddUpdateCommandFromTlog perhaps?) and asserted in case someone tries to miss-use it.

* doReplay
** the {{case UpdateLog.UPDATE_INPLACE}} block includes {{log.debug("add {}", cmd);}} .. "add" doesn't seem appropriate
*** if "add" is appropriate, then that seems like even more justification for {{updateCommandFromTlog}} being refactored to work on both ADD and UPDATE_INPLACE log entries, so those 2 case statements can be combined completley (via fall through) [Ishan]FIXED: log.debug(oper == ADD? "add ": "update " + cmd);[/Ishan]

{panel}

{panel:title=TransactionLog}

* jdocs for the two write methods seem backwards [Ishan]FIXED[/Ishan]
** if i'm calling the 2 arg version the jdocs should make it clear this can *not* be used for in-place updates
** if i'm calling the 3 arg version, the jdocs should make it clear what prevPointer value i should use if it is *not* an in-place update
* seems like the 3 arg write method should have an assertion that looks something like... 
** {{assert (-1 <= prevPointer && (cmd.isInPlaceUpdate() ^ (-1 == prevPointer)))}}  [Ishan]TODO[/Ishan]

{panel}

{panel:title=test-files: schema.xml & schema15.xml}

* presumably the main purpose of the {{\_version\_}} field def changes in these files is because a docvalues only version field is required by the new/modified tests that use them to support in place updates
** but i didn't see any of these tests assert that the {{\_version\_}} field had the neccessary properties -- so there is no protection against anybody down the road modifying these schema files again and breaking that assumption (so that only "regular" atomic updates happen ... maybe causing confusing test failures, or maybe just weakening the tests.

{panel}

{panel:title=schema-minimal-atomic-stress.xml}

* I would prefer we not make most of the changes in this file.
** Instead I would strongly prefer you add a new, small, purpose built & well commented schema-inplace-atomic-updates.xml file just for the new InPlace tests -- containing only the fields/dynamicFields needed
** having smaller single purpose test configs makes it a lot easier for people reviewing tests to understand what the minimum expectations are for the test to function
** Even if there was some incredibly strong reason to keep all these tests usign the same file...
*** we should keep all the field/fieldType/dynamicField naming conventions cosistent
*** if the existing naming convention of "long_dv" won't work for the new fields needed (ie: "float_dv", "int_dv", etc...) because the new tests need more then one field of each type, then the existing field declarations should be removed and TestStressCloudBlindAtomicUpdates should be updated to use field names based on the new dynamicField conventions (ie: "long_dv" -> "*_l_dvo")

* in general, regarldess of what schema file is used by all the new tests...
** we should stop using the ambiguious "ratings" and "price" field names
** instead the tests should use field names that clearly convey what the expecation of the field is -- either vie xplicit schema declaration, or via one of the dynamicFields you already defined (ie: "inplace_updatable_x_long" or "fieldX_l_dvo")
*** probably a good idea for one to be an explicitly declared {{<field/>}} and the other to come from a {{<dynamicField/>}} just to ensure good test coverate of both scenerios -- but either way the name as seen in the test files should make it clear what matters about the field)

{panel}



{panel:title=UpdateLogTest}

* the changes forced on the CommitTracker should be done in the breforeClass method so they don't suprise the hell out of anyone who tries adding new test methods to this class later.
** use a static UpdateLog variable so test methods like testApplyPartialUpdates can re-use it (and be confident it's the same one with the CommitTracker changes

* in general there are a lot of LocalSolrQueryRequest objects created in this class that are never closed
** that may not be causing problems right now, but it's arecipe for confusing leaks later
** the convinience methods that build up UpdateCommants (ulogCommit & getDeleteUpdate & getAddUpdate) sould be refactored to take in a SolrQueryRequest which should be closed when the logical bit of code related to testing that UpdateCommand is finished. something like...{code}
try (SolrQueryRequest req = req()) {
  ulog.add(getAddUpdate(req, ...));
  // TODO: anything else that needs to be part of this same request for test purposes?
}
// TODO ... assert stuff about ulog
...
{code}

* in general, it's confusing to me how exactly the commands being tested here represent "in-place" updates considering the sdocs genreated don't included any inc/set atomic ops at all
** i think what's happening here is that we're testing at a low enough level that the logic for actualy *applying* the in-place update has already happened, and here we're just testing the merging of that doc (after the inc/set has been applied) with the older versions?
** if my understanding is correct, some comments/javadocs in the helper methods making this more clear would be helpful.
** if i'm wrong, then please clarify...

* testApplyPartialUpdates
** javadocs: what is being tested here? use an \@see tag if nothing else
** in general it feels like this should be split up into multipe discreet test methods (ie: testApplyPartialUpdatesOnMultipleInPlaceUpdatesInSequence, testApplyPartialUpdatesAfterMultipleCommits, etc...)
** {{// If an in-place update depends on a non-add, assert that an exception is thrown.}}
*** use expectThrows...{code}
SolrException ex = expectThrows(SolrException.class, () -> {
  returnVal = ulog.applyPartialUpdates(cmd.getIndexedId(), prevPointer, prevVersion, partialDoc);
});
// TODO: assert stuff about ex.code, ex.getMessage()
{code}
** what about testing DBQs in the ulog and how they affect partial updates?

* ulogCommit
** javadocs
** why can't this be static?
** see previous comment about method args vs un-closed SolrQueryRequest

* getDeleteUpdate
** javadocs
** why can't this be static?
** see previous comment about method args vs un-closed SolrQueryRequest

* getAddUpdate
** javadocs
** see previous comment about method args vs un-closed SolrQueryRequest
** i would change this method to take in a SolrInputDocument
*** {{getAddUpdate(42, sdoc(....));}} is just as easy to read as {{getAddUpdate(42, ....);}} but more versitile
** aren't very bad things garunteed to happen unless {{cmd.setVersion}} is set?
*** i would remove the conditional wrapping {{cmd.setVersion}} and instead just assert that there is a version in the solrDoc to copy
*** either that, or have some usage/test cases where the doc being added doesn't include a version field

* toSolrDoc (& toSolrDoc)
** why are these methods being copied/pasted from RTGC?
** if they are useful for testing, don't clone them -- change them to public, anotate them with \@lucene.experimental and use them directly from the test.
*** only the 3 arg seems needed directly by the test, the 2 arg method can probably remain private.
{panel}




{panel:title=PeerSyncTest}

* WTF: System.out.println() ???

* {{// lets add some in-place updates}}
** inPlaceParams is only used once, so there aren't "some in-place updates" there is exactly 1 "in-place update"
** i'm guessing you inteded to re-use inPlaceParams in all the add(...) calls but had a copy/paste glitch?
** if i'm wrong, please include a comment explaining what exactly you're testing .. what's the point of the subsequent adds that do _not_ set an DISTRIB_INPLACE_PREVVERSION

* v++
** please avoid these kind of "postfix increment happens after variable is evaluated" short cuts -- especailly since every other place in this testclase uses _prefix_ increment to precivesly to avoid this kind of confusion.
** the test would be much more readable if you just use concrete hardcoded literal version values in place of "v++" in every line
*** especially since DISTRIB_INPLACE_PREVVERSION has a hard coded assumption about what the value of "v" was
*** the fact that the "id" in use here is also the same as the starting value of "v" makes the DISTRIB_INPLACE_PREVVERSION actaully seem wrong at first glance

* your new code is adding the docs to client0, but then also calling assertSync on client0, and passing {{shardsArr\[0\]}} for the syncWith param.
** I'm 99% certain you're just asking client0 to sync with itself.
** if you add some queryAndCompare() calls to ensure these syncs are doing what you think, i'm pretty sure you'll see client0 and client1 disagree
*** even if i'm wrong, adding some queryAndCompare calls would be useful to ensure the peersync is actually doing something useful and not just silently returning "true"
*** please don't follow the bad example usgae of queryAndCompare earlier in this test class -- actually look at the QueryResponse that method returns and check the numfound, and returned field values ,etc...)

* what about mixing in-place updates with deletes (both by id and query?)

{panel}




{panel:title=TestRecovery}

* why is {{// assert that in-place update is retained}} at teh end of the test, instead of when the (first) log replay happens and \*:\* -> numFound=3 is asserted? (ie: before "A2" is added)

* what about mixing in-place updates with deletes (both by id and query?)

{panel}



{panel:title=TestInPlaceUpdatesStandalone}

* test setup still doesn't seem to be doing anything to assert that autocommit is disabled
** this is important to prevent future changes to configs/defaults from weaking the test or causing spurrious / confusing failures.

* clearIndex & softCommit
** these appear at the begining of every test method
** instead add an \@Before method to ensure this happens automatically

* testUpdatingDocValues
** checking return values from addAndGetVersion / addAndAssertVersion
*** in a previous review of these tests, i pointed out...{quote}
* for addAndGetVersion calls where we don't care about the returned version, don't bother assigning it to a variable (distracting)
* for addAndGetVersion calls where we do care about the returned version, we need check it for every update to that doc...
{quote}... but in the new patch things are still very confusing.
*** this new patch now includes an {{addAndAssertVersion}} which looks like it was designed to ensure that all updates (even atomic, in place or otherwiser) produce a new version greater then the old version -- but many usages of this method that i see just pass "0" for the "expected" version --- even though the *actual* expected version is already known...{code}
long version1 = addAndGetVersion(sdoc("id", "1", "title_s", "first"), null);
...
// Check docValues were "set"
version1 = addAndAssertVersion(0, "id", "1", "ratings", map("set", 200));
{code}
** the docid# vars should have a comment making it clear that the reason we're fetching these is to validate that the subsequent updates are done in place and don't cause the docids to change
** {{// Check back to back "inc"s are working (off the transaction log)}}
*** why isn't the query assertion here also checking the docid?
** {{// Check optimistic concurrency works}}
*** please use expectThrows here, and actually validate that the Exception we get is a SolrException with code=409
**** we don't want the test to silently overlook if some bug is introduced when using optimistic concurrency with in-place updates
*** Instead of "123456," (which might randomly be the *actual* version of the doc) use "-1" (which indicates yo uexpect the document to not exist)
*** before committing & doing the assertQ to check the searchable results, we should also be checking the results of an RTG against the tlog to ensure they're returning the in-place updated values as well.


* testUpdateTwoDifferedFields
** ditto previous concerns about passing "0" to addAndAssertVersion in many cases
** {{// RTG}}}
*** this is ... bizare ... i don't understand how this is remotely suppose to demonstrate an RTG.
*** please replace with something like...{code}
assertQ("RTG check",
        req("qt","/get","id",id, "wt","xml", "fl","score,val_i"),
        "//doc/float[@name='ratings'][.='202.0']",
        "//doc/int[@name='price'][.='12']",
        "//doc/long[@name='_version_'][.='"+version1+"']",
        "//doc/int[@name='[docid]'][.='"+docid1+"']"
        );
{code}
*** i also don't understand why this is only done conditionally based on a random boolean -- it has no side effects, and it's not an alternative to checking a regular query -- so why not do it in every test run before the softCommit + query?
** i was really confused that this test never actaully verified that you could do in-place updates on 2 diff fields in a single request ... let's add that please...{code}
version1 = addAndAssertVersion(0, "id", "1", "ratings", map("inc", 66), "price", map("inc", 44));
{code}

* testDVUpdatesWithDelete
** {{// Add the document back to index}}
*** doesn't this weaken the test?
*** shouldn't an add with {{map("set", 200)}} succeed even after a delete (w/o needing add the doc first)
*** shouldn't the point here be that we don't pick up some already deleted doc (from index or tlog) and apply an in-place update to it?
*** something like...{code}
for (boolean postAddCommit : Arrays.asList(true, false)) {
  for (boolean delById : Arrays.asList(true, false)) {
    for (boolean postDelCommit : Arrays.asList(true, false)) {
      addAndGetVersion(sdoc("id", "1", "title_s", "first"), null);
      if (postAddCommit) assertU(commit());
      assertU(delById ? delI("1") : delQ("id:1"));
      if (postDelCommit) assertU(commit());
      version1 = addAndGetVersion(sdoc("id", "1", "ratings", map("set", 200)));
      // TODO: assert current doc#1 doesn't have old value of "title_s"
{code}

* addAndAssertVersion
** we should renamed "version" arg to "expectedCurrentVersion" to be more clear what it's purpose is
** need {{assert 0 < expectedCurrentVersion}} to prevent missuse (see above)

* testUpdateOfNonExistentDVsShouldNotFail
** i think the idea here is that it's hte only method using "val_i_dvo" -- so adding id=0 helps test when the field doesn't exist at all, and id=1 helps test when the field does't exist for a specified doc?
*** need a whitebox check of the actual index reader to assert that field doesn't exist (in case someone adds a new test w/o realing it makes this one useless
*** ideally use a more distinctive field name as well
** this test should also assert the docid doesn't change (when we dont' expect it to)

* testOnlyPartialUpdatesBetweenCommits
** why not test "inc" here?
** should be asserting expected versions in final query.

* getFieldValue
** this should be using RTG so it does't depend on a commit

* DocInfo
** is val1 used _anywhere_ ?  can it be removed?
*** if it can be removed: lets rename val2 just "value"
*** if we need them both, let's rename val1 and val2 to intFieldValue and longFieldValue (or somethign like that) so it's not as easy to get them confused

* testReplay3, testReplay4, testReplay6, testReplay7
** these should have better names
** the String constants should be replaced with sdoc calls (see comments on checkReplay below)

* checkReplay
** since we're no longer using this method to read from text files this method should be refactored...
*** let's replace the String[] param with an Object[] param
**** if it's a SolrInputDoc add it
**** else look for some "private static final Object" sentinal objects for hard commit & soft commit

* getSdoc
** once checkReplay is refactored, this method can be killed

* testMixedInPlaceAndNonInPlaceAtomicUpdates
** again: this is not a good way to test RTG
** again: there's no reason to randomize if we do an RTG check
** ideally we should do an RTG check after every one of the addAndAssertVersion calls (and still do a commit & query check a the end of the method


* I'd still like to see some 100% randomized testing using checkReplay.  It's a really powerful helper method -- we should be taking full advantage of it.

{panel}


{panel:title=TestStressInPlaceUpdates}

* A really large number of comments i made the last time i reviewed this class still seem problematic, and i can't find any (jira) comments addressing them either...{quote}

* ...(class) javadocs, and extending SolrCloudTestCase once LUCENE-7301 is fixed and we're sure this test passes reliably. ....
** also: we should really make this test use multiple shards

* it would be a lot cleaner/clearer if we refactored these anonymous Thread classes into private static final (inner) classes and instantiated them like normal objects
** makes it a lot easier to see what threads access/share what state
** better still would be implementing these "workers" as Callable instances and using an ExecutorService

* "operations" comment is bogus (it's not just for queries)

* I'm not convinced the "{{synchronize \{...\}; commit stuff; syncrhonize \{ ... \};}}" sequence is actually thread safe...
** T-W1: commit sync block 1: newCommittedModel = copy(model), version = snapshotCount++;
** T-W2: updates a doc and adds it to model
** T-W1: commit
** T-W1: commit sync block 2: committedModel = newCommittedModel
** T-R3: read sync block: get info from committedModel
** T-R3: query for doc
** ...

* ... in the above sequence, query results seen by thread T-R3 won't match the model because the update from T-W2 made it into the index before the commit, but after the model was copied
** i guess it's not a huge problem because the query thread doesn't bother to assert anything unless the versions match -- but that seems kind of risky ... we could theoretically never assert anything

* having at least one pass over the model checking every doc at the end of the test seems like a good idea no matter what

* I'm certain the existing "synchronized (model)" block is not thread safe relative to the synchronized blocks that copy the model into commitedModel, because the "model.put(...)" calls can change the iterator and trigger a ConcurrentModificationException

* the writer threads should construct the SolrInputDocument themselves, and log the whole document (not just the id) when they log things, so it's easier to tell from the logs what updates succeed and which were rejected because of version conflicts

* there's a lot of "instanceof ArrayList" checks that make no sense to me since the object came from getFirstValue

* verbose
** why does this method exist? why aren't callers just using log.info(...) directly?
** or if callers really need to pass big sequences of stuff, they can use {{log.info("\{\}", Arrays.asList(...))}}
** or worst case: this method can simplified greatly to do that internally

* addDocAndGetVersion
** using SolrTestCaseJ4.sdoc and SolrTestCaseJ4.params will make this method a lot -sorder- shorter

{quote}

* this block reads very awkwardly...{code}
if (oper < commitPercent + deletePercent) { // deleteById
  returnedVersion = deleteDocAndGetVersion(Integer.toString(id), params("_version_",Long.toString(info.version)), false);
} else { // deleteByQuery
  returnedVersion = deleteDocAndGetVersion(Integer.toString(id), params("_version_",Long.toString(info.version)), true);
}
log.info((oper < commitPercent + deletePercent? "DBI": "DBQ")+": Deleting id=" + id + "], version=" + info.version
    + ".  Returned version=" + returnedVersion);
{code}
** wouldn't something like this be functionally equivilent and easier to make sense of? {code}
final boolean dbq = (commitPercent + deletePercent <= oper);
returnedVersion = deleteDocAndGetVersion(Integer.toString(id), params("_version_",Long.toString(info.version)), dbq);
log.info((dbq ? "DBI": "DBQ") +": Deleting id=" + id + "], version=" + info.version  + ".  Returned version=" + returnedVersion);
{code}

* this looks like dead code ... or is something actaully still missing and needs done? ... {code}
} else if (oper < commitPercent + deletePercent + deleteByQueryPercent) {
  // TODO
} else {
{code}


* DocInfo
** still needs javadocs
** let's rename val1 and val2 to intFieldValue and longFieldValue (or somethign like that) so it's not as easy to get them confused

* {{synchronized (leaderClient)}}
** I understand why we are only sending to leader (SOLR-8733) but i still don't udnerstand why the updates are synchronized

* it doesn't take long to get this test to fail ... here's some seeds that failed for me when hammering thetests, (but succeeded when i tried the reproduce line) ...
** {noformat}
   [junit4]   2> NOTE: reproduce with: ant test  -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=CE02ABC183DF7342 -Dtests.slow=true -Dtests.locale=ar-LB -Dtests.timezone=America/Manaus -Dtests.asserts=true -Dtests.file.encoding=UTF-8
   [junit4] ERROR   31.1s | TestStressInPlaceUpdates.stressTest <<<
   [junit4]    > Throwable #1: com.carrotsearch.randomizedtesting.UncaughtExceptionError: Captured an uncaught exception in thread: Thread[id=192, name=READER2, state=RUNNABLE, group=TGRP-TestStressInPlaceUpdates]
   [junit4]    >        at __randomizedtesting.SeedInfo.seed([CE02ABC183DF7342:A564746CBD0AA7B8]:0)
   [junit4]    > Caused by: java.lang.RuntimeException: org.apache.solr.client.solrj.impl.HttpSolrClient$RemoteSolrException: Error from server at http://127.0.0.1:44464/muf/l/collection1: Unable to resolve the last full doc in tlog fully, and document not found in index even after opening new rt searcher. id=1, partial document=SolrDocument{id=stored,indexed,omitNorms,indexOptions=DOCS<id:1>, val1_i_dvo=docValuesType=NUMERIC<val1_i_dvo:13>, val2_l_dvo=docValuesType=NUMERIC<val2_l_dvo:13000000039>, _version_=docValuesType=NUMERIC<_version_:1541124487571832832>}
   [junit4]    >        at __randomizedtesting.SeedInfo.seed([CE02ABC183DF7342]:0)
   [junit4]    >        at org.apache.solr.cloud.TestStressInPlaceUpdates$2.run(TestStressInPlaceUpdates.java:376)
   [junit4]    > Caused by: org.apache.solr.client.solrj.impl.HttpSolrClient$RemoteSolrException: Error from server at http://127.0.0.1:44464/muf/l/collection1: Unable to resolve the last full doc in tlog fully, and document not found in index even after opening new rt searcher. id=1, partial document=SolrDocument{id=stored,indexed,omitNorms,indexOptions=DOCS<id:1>, val1_i_dvo=docValuesType=NUMERIC<val1_i_dvo:13>, val2_l_dvo=docValuesType=NUMERIC<val2_l_dvo:13000000039>, _version_=docValuesType=NUMERIC<_version_:1541124487571832832>}
   [junit4]    >        at org.apache.solr.client.solrj.impl.HttpSolrClient.executeMethod(HttpSolrClient.java:606)
   [junit4]    >        at org.apache.solr.client.solrj.impl.HttpSolrClient.request(HttpSolrClient.java:259)
   [junit4]    >        at org.apache.solr.client.solrj.impl.HttpSolrClient.request(HttpSolrClient.java:248)
   [junit4]    >        at org.apache.solr.client.solrj.SolrRequest.process(SolrRequest.java:149)
   [junit4]    >        at org.apache.solr.client.solrj.SolrClient.query(SolrClient.java:942)
   [junit4]    >        at org.apache.solr.client.solrj.SolrClient.query(SolrClient.java:957)
   [junit4]    >        at org.apache.solr.cloud.TestStressInPlaceUpdates$2.run(TestStressInPlaceUpdates.java:343)
{noformat}
** {noformat}
   [junit4]   2> NOTE: reproduce with: ant test  -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=D3059BEABF12E831 -Dtests.slow=true -Dtests.locale=nl-BE -Dtests.timezone=Asia/Thimbu -Dtests.asserts=true -Dtests.file.encoding=UTF-8
   [junit4] ERROR   33.1s | TestStressInPlaceUpdates.stressTest <<<
   [junit4]    > Throwable #1: com.carrotsearch.randomizedtesting.UncaughtExceptionError: Captured an uncaught exception in thread: Thread[id=186, name=READER6, state=RUNNABLE, group=TGRP-TestStressInPlaceUpdates]
   [junit4]    >        at __randomizedtesting.SeedInfo.seed([D3059BEABF12E831:B863444781C73CCB]:0)
   [junit4]    > Caused by: java.lang.RuntimeException: org.apache.solr.client.solrj.impl.HttpSolrClient$RemoteSolrException: Error from server at http://127.0.0.1:37544/ct_/px/collection1: Unable to resolve the last full doc in tlog fully, and document not found in index even after opening new rt searcher. id=0, partial document=SolrDocument{val2_l_dvo=docValuesType=NUMERIC<val2_l_dvo:5000000010>, id=stored,indexed,omitNorms,indexOptions=DOCS<id:0>, _version_=docValuesType=NUMERIC<_version_:1541142011874115584>}
   [junit4]    >        at __randomizedtesting.SeedInfo.seed([D3059BEABF12E831]:0)
   [junit4]    >        at org.apache.solr.cloud.TestStressInPlaceUpdates$2.run(TestStressInPlaceUpdates.java:376)
   [junit4]    > Caused by: org.apache.solr.client.solrj.impl.HttpSolrClient$RemoteSolrException: Error from server at http://127.0.0.1:37544/ct_/px/collection1: Unable to resolve the last full doc in tlog fully, and document not found in index even after opening new rt searcher. id=0, partial document=SolrDocument{val2_l_dvo=docValuesType=NUMERIC<val2_l_dvo:5000000010>, id=stored,indexed,omitNorms,indexOptions=DOCS<id:0>, _version_=docValuesType=NUMERIC<_version_:1541142011874115584>}
   [junit4]    >        at org.apache.solr.client.solrj.impl.HttpSolrClient.executeMethod(HttpSolrClient.java:606)
   [junit4]    >        at org.apache.solr.client.solrj.impl.HttpSolrClient.request(HttpSolrClient.java:259)
   [junit4]    >        at org.apache.solr.client.solrj.impl.HttpSolrClient.request(HttpSolrClient.java:248)
   [junit4]    >        at org.apache.solr.client.solrj.SolrRequest.process(SolrRequest.java:149)
   [junit4]    >        at org.apache.solr.client.solrj.SolrClient.query(SolrClient.java:942)
   [junit4]    >        at org.apache.solr.client.solrj.SolrClient.query(SolrClient.java:957)
   [junit4]    >        at org.apache.solr.cloud.TestStressInPlaceUpdates$2.run(TestStressInPlaceUpdates.java:343)
{noformat}


{panel}



{panel:title=TestInPlaceUpdatesCopyFields}

* WTF: {{TestInPlaceUpdatesCopyFields extends AbstractBadConfigTestBase}}

* this test seems to jumpt through hoops to use a mutable managed schema -- but as far as i can tell it doesn't actaully test anything that requires the schema to change during the course of the test...
** it would be a lot simpler & easier to read if it just started up with a simple schema containing all of the copyFields needed
** unless you want to change the test so it does things like "assert in-place update of foo_i_dvo works; add a copyField from foo_i_dvo to foo_stored; assert update of foo_i_dvo is no longer in place"

* the name of this class seems a bit too narrow
** it's not just a test of in-place updates using copy fields, it's a lot of unit tests of AtomicUpdateDocumentMerger.isInPlaceUpdate
** suggest: TestAtomicUpdateDocMergerIsInPlace
** or just make it a test method in TestInPlaceUpdatesStandalone
*** especially if you simplify it to use pre-declared copyFields and don't need the mutable schema.

* I found multiple seeds that fails 100% of the time on the same assert ... i haven't looked into why...
** {noformat}
   [junit4]   2> NOTE: reproduce with: ant test  -Dtestcase=TestInPlaceUpdatesCopyFields -Dtests.method=testIsInPlaceUpdate -Dtests.seed=54280A18530C3306 -Dtests.slow=true -Dtests.locale=en-ZA -Dtests.timezone=Europe/Tirane -Dtests.asserts=true -Dtests.file.encoding=US-ASCII
   [junit4] FAILURE 0.10s J2 | TestInPlaceUpdatesCopyFields.testIsInPlaceUpdate <<<
   [junit4]    > Throwable #1: java.lang.AssertionError
   [junit4]    >        at __randomizedtesting.SeedInfo.seed([54280A18530C3306:99BC22C9123C0682]:0)
   [junit4]    >        at org.apache.solr.update.TestInPlaceUpdatesCopyFields.testIsInPlaceUpdate(TestInPlaceUpdatesCopyFields.java:118)
   [junit4]    >        at java.lang.Thread.run(Thread.java:745)
{noformat}
** {noformat}
   [junit4]   2> NOTE: reproduce with: ant test  -Dtestcase=TestInPlaceUpdatesCopyFields -Dtests.method=testIsInPlaceUpdate -Dtests.seed=A4B7A0F71938C5FE -Dtests.slow=true -Dtests.locale=en -Dtests.timezone=America/Tijuana -Dtests.asserts=true -Dtests.file.encoding=UTF-8
   [junit4] FAILURE 2.47s J2 | TestInPlaceUpdatesCopyFields.testIsInPlaceUpdate <<<
   [junit4]    > Throwable #1: java.lang.AssertionError
   [junit4]    >        at __randomizedtesting.SeedInfo.seed([A4B7A0F71938C5FE:692388265808F07A]:0)
   [junit4]    >        at org.apache.solr.update.TestInPlaceUpdatesCopyFields.testIsInPlaceUpdate(TestInPlaceUpdatesCopyFields.java:118)
   [junit4]    >        at java.lang.Thread.run(Thread.java:745)
{noformat}

{panel}


{panel:title=TestInPlaceUpdatesDistrib}


* again A large number of comments i made the last time i reviewed this class still seem problematic, and i can't find any (jira) comments addressing them either...{quote}

* Once LUCENE-7301 is fixed and we can demonstate that this passes reliably all of the time, we should ideally refactor this to subclass SolrCloudTestCase

* In general, the "pick a random client" logic should be refactored so that sometimes it randomly picks a CloudSolrClient

* ensureRtgWorksWithPartialUpdatesTest
** even if we're only going to test -one- _a few_ doc, we should ensure there are a random num docs in the index (some before the doc we're editing, and some after)
** _2 docs before/after is not a random number ... random means random: we need to test edge cases of first docid in index, last docid in index, first/last docid in segment, etc..._

* outOfOrderUpdatesIndividualReplicaTest
** ditto comments about only one doc
** if we are going to use an ExecutorService, then the result of awaitTermination has to be checked
** ... and shutdown & awaitTermination have to be called in that order
** since this tests puts replicas out of sync, a ... "wait for recovers" should happen at the end of this test (or just in between every test) .. especially if we refactor it (or to protect someone in the future who might refactor it)

* outOfOrderUpdatesIndividualReplicaTest  (followup comments)
** lots more comments in the test code to make it clear that we use multiple threads because each update may block if it depends on another update
** create {{atLeast(3)}} updates instead of just a fixed set of "3" so we increase our odds of finding potential bugs when more then one update is out of order.
** loop over multiple (random) permutations of orderings of the updates
*** don't worry about wether a given ordering is actually correct, that's a valid random ordering for the purposes of the test
*** a simple comment saying we know it's possible but it doesn't affect any assumptions/assertions in the test is fine
** for each random permutation, execute it (and check the results) multiple times
*** this will help increase the odds that the thread scheduling actaully winds up running our updates in the order we were hoping for.
** essentially this should be a a micro "stress test" of updates in arbitrary order. Something like... {code}
final String ID = "0";
final int numUpdates = atLeast(3);
final int numPermutationTotest = atLeast(5);
for (int p = 0; p < numPermutationTotest; p++) {
  del("*:*);
  commit();
  index("id",ID, ...); // goes to all replicas
  commit();
  long version = assertExpectedValuesViaRTG(LEADER, ID, ...);
  List<UpdateRequest> updates = makeListOfSequentialSimulatedUpdates(ID, version, numUpdates);
  for (UpdateRequest req : updates) {
    assertEquals(0, REPLICA_1.requets(req).getStatus());
  }
  Collections.shuffle(updates, random());
  // this method is where you'd comment the hell out of why we use threads for this,
  // and can be re-used in the other place where a threadpool is used...
  assertSendUpdatesInThreadsWithDelay(REPLICA_0, updates, 100ms);
  for (SolrClient client : NONLEADERS) [
    // assert value on replica matches original value + numUpdates
  }
}
{code}
** As a related matter -- if we are expecting a replica to "block & eventually time out" when it sees an out of order update, then there should be a white box test asserting the expected failure situation as well -- something like... {code}
final String ID = "0";
del("*:*);
commit();
index("id",ID, ...);
UpdateRequest req = simulatedUpdateRequest(version + 1, ID, ...);
Timer timer = new Timer();
timer.start();
SolrServerException e = expectThrows(() -> { REPLICA_0.request(req); });
timer.stop();
assert( /* elapsed time of timer is at least the X that we expect it to block for */ )
assert(e.getgetHttpStatusMesg().contains("something we expect it to say if the update was out of order"))
assertEquls(/* whatever we expect in this case */, e.getHttpStatusCode());
{code}


* delayedReorderingFetchesMissingUpdateFromLeaderTest
** Is there no way we can programatically tell if LIR has kicked in? ... pehaps by setting a ZK watch? ... this "Thread.sleep(500);" is no garuntee and seens arbitrary.
*** at a minimum polling in a loop for the expected results seems better then just a hardcoded sleep

{quote}



* test()
** {{// assert that schema has autocommit disabled}}
*** this doesn't assert autocommit is disabled, all it does is assert that a sys property was set in beforeSuperClass
*** nothing about this actually asserts that the configs/defaults in use don't have autocommit -- use the Config API to be certain

* docValuesUpdateTest
** the lower limit of numDocs=1 seems absurd ... why not {{atLeast(100)}} or something?
** please don't use (for) loops w/o braces.
** I don't understand anything about the use of {{luceneDocids}} in this method...
*** correctness...
**** for starters, matchResults seems completely broken -- but i'll get to that later.  let's assume for now that it works...
**** the initial list comes from a randomly choosen client, and then later it's compared to the list from another randomly choosen client -- how is this comparison safe?
**** If there is any sort of hicup during indexing that requires the leader to retry sending a doc, then the docids won't match up.
**** it seems like these checkers are really just relying on the fact that if there is any discrepency between the replicas, we'll retry enough times that eventually we'll get lucky and query the first replica again.
*** even if everything in the code as written is 100% fine:
**** the iniial list of luceneDocids is populated by a query that doesn't do any retry logic and will fail fast if the numDocs doesn't match the result.size()
**** this does nothing to address the problem noted in the older version of the test: waiting for the commit to propogate and open new searchers on all shards: {{Thread.sleep(500); // wait for the commit to be distributed}}
*** In general, this seems like a convoluted way to try and kill two birds with one stone: 1) make sure all replicas have opened searchers with the new docs; 2) give us something to compare to later to ensure the update was truely in place
**** i really think the first problem should be addressed the way i suggested previously:{quote}
* if initially no docs have a rating value, then make the (first) test query be for {{rating:\[\* TO \*\]}} and execute it in a rety loop until the numFound matches numDocs.
* likewise if we ensure all ratings have a value such that abs(ratings) < X, then the second update can use an increment such that abs(inc) > X\*3 and we can use {{-ratings:\[-X TO X\]}} as the query in a retry loop
{quote}
**** the second problem should be solved by either using the segments API, or by checking the docids on _every_ replica (w/o any retries) ... _after_ independently verifying the searcher has been re-opened.

* matchResults
** no javadocs
*** I'm assuming the point is to return true if the (same ordered) luceneDocids & ratings match the results
* returns true if the number of results doesn't match the number of luceneDocids
** seems broken: if a shard hasn't re-opened a searcher yet (ie: {{0==results.size()}}) ... implies results do match when they are grossly difference.
* {{int l = ...}} ... why is this variable named "l" ???

* ensureRtgWorksWithPartialUpdatesTest
** instead of {{log.info("FIRST: " + sdoc);}} and {{log.info("SECOND: " + sdoc);}} just put the sdoc.toString() in the assert messages...{code}
assertEquals("RTG tlog price: " + sdoc, (int) 100, sdoc.get("price"));
{code}
** nothing in this test asserts that the update is actually in place
*** checking {{fl=\[docid\]}} in cloud RTG is currently broken (SOLR-9289) but we can/should be checking via the segments API anyway (if we have a general helper method for comparing the segments API responses of multiple replicas betwen multiple calls, it could be re-used in every test in this class)

* outOfOrderUpdatesIndividualReplicaTest
** {{long seed = random().nextLong(); // seed for randomization within the threads}}
*** each AsyncUpdateWithRandomCommit task needs it's own seed value, otherwise they'll all make the exact same choices.

* outOfOrderDeleteUpdatesIndividualReplicaTest
** new test, most of the comments i had about outOfOrderUpdatesIndividualReplicaTest (both the new comments, and the older comments that i don't see any updates/replies regarding) also apply here.

* delayedReorderingFetchesMissingUpdateFromLeaderTest
** this is sketchy brittle -- just create a {{List<SolrClient> ALL_CLIENTS}} when creating {{LEADER}} and {{NONLEADERS}}...{code}
for (SolrClient client: new SolrClient\[\] \{LEADER, NONLEADERS.get(0),
    NONLEADERS.get(1)}) { // nonleader 0 re-ordered replica, nonleader 1 well-ordered replica
{code}

* simulatedUpdateRequest
** i have not looked into where the compiler is finding the {{id}} variable used in this method, but it's definitely not coming fro mthe method args, or the {{doc}} -- so it's probably broken.
** whatever the fix is for the {{baseUrl}} init code, please refactor it into a static helper method so we don't have these 4 lines duplicated here and in simulatedDeleteRequest

* addDocAndGetVersion
** {{synchronized (cloudClient)}}
*** why are we synchronized on cloud client but updating via LEADER?
*** why are we synchronized at all?

{panel}

{panel:title=JettySolrRunner}

* use MethodHandles for static logger init

* if there are multiple delays whose counters have "triggered", why only execute a single delay of the "max" time? ... shouldn't it be the sum?

* now that an individual "Delay" can be "delayed" (ie: there's a count involved and the delay may not happen for a while) let's add a {{String reason}} param to addDelay and the Delay class and log that message as we loop over the delay objects


{panel}


{panel:title=General Questions / Concerns}

* precommit was failing for me due to javadoc warnings

* LUCENE-7344
** previous comment from Ishan...{quote}
As I was incorporating Hoss' suggestions, I wrote a test for DV updates with DBQ on updated values. This was failing if there was no commit between the update and the DBQ. I think this is due to LUCENE-7344.
{quote}
** I was expecting to find this test code somewhere, but i did not
** We still need some sort of solution to this problem -- the suggested requirement/documentation workarround suggested by McCandless in that issue doesn't really fit with how all work on this jira to date has kept the  the decision of when/if to do an "in-place" update a low level implementation detail .... that would have to radically change if we wanted to "pass the buck" up to the user to say "you an't use DBQ on a docvalue field that you also use for in place updates"
** so what's our Solr solution / workaround?  do we have any?

* Block join docs?
**  I never really considered this before, but it jumped out at me when reviewing some of the code
** What happens today (w/o patch) if someone has block join docs and does an an atomic update that updates both parent and child?
*** I'm assuming that currently "works" (ie: store fields of both docs are read, update applied to both, and then written back as a new block)
** What happens w/this patch in the same situation?
*** what if in both docs, the field being updated supports in-place updates? ... does it work and update the both docs in place?
*** what if only othe parent doc's update involves an in-place updatable field, but the child doc update is on a field that is stored/indexed?  .... does the "isInPlaceUpdate" style logic kick in correctly for all the docs in the hierarchy? (so that the entire block is updated as a "regular" atomic update?)

{panel}

> Support updates of numeric DocValues
> ------------------------------------
>
>                 Key: SOLR-5944
>                 URL: https://issues.apache.org/jira/browse/SOLR-5944
>             Project: Solr
>          Issue Type: New Feature
>            Reporter: Ishan Chattopadhyaya
>            Assignee: Shalin Shekhar Mangar
>         Attachments: DUP.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, SOLR-5944.patch, TestStressInPlaceUpdates.eb044ac71.beast-167-failure.stdout.txt, TestStressInPlaceUpdates.eb044ac71.beast-587-failure.stdout.txt, TestStressInPlaceUpdates.eb044ac71.failures.tar.gz, defensive-checks.log.gz, hoss.62D328FA1DEA57FD.fail.txt, hoss.62D328FA1DEA57FD.fail2.txt, hoss.62D328FA1DEA57FD.fail3.txt, hoss.D768DD9443A98DC.fail.txt, hoss.D768DD9443A98DC.pass.txt
>
>
> LUCENE-5189 introduced support for updates to numeric docvalues. It would be really nice to have Solr support this.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org