Uploaded image for project: 'Solr'
  1. Solr
  2. SOLR-5944

Support updates of numeric DocValues

    Details

    • Type: New Feature
    • Status: Resolved
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 6.5, master (7.0)
    • Component/s: None
    • Labels:
      None

      Description

      LUCENE-5189 introduced support for updates to numeric docvalues. It would be really nice to have Solr support this.

      1. defensive-checks.log.gz
        52 kB
        Ishan Chattopadhyaya
      2. demo-why-dynamic-fields-cannot-be-inplace-updated-first-time.patch
        4 kB
        Ishan Chattopadhyaya
      3. DUP.patch
        12 kB
        Noble Paul
      4. hoss.62D328FA1DEA57FD.fail.txt
        1.39 MB
        Hoss Man
      5. hoss.62D328FA1DEA57FD.fail2.txt
        8.00 MB
        Hoss Man
      6. hoss.62D328FA1DEA57FD.fail3.txt
        9.49 MB
        Hoss Man
      7. hoss.D768DD9443A98DC.fail.txt
        8.65 MB
        Hoss Man
      8. hoss.D768DD9443A98DC.pass.txt
        14.20 MB
        Hoss Man
      9. master-vs-5944-regular-updates.png
        18 kB
        Ishan Chattopadhyaya
      10. regular-vs-dv-updates.png
        20 kB
        Ishan Chattopadhyaya
      11. SOLR-5944.patch
        301 kB
        Ishan Chattopadhyaya
      12. SOLR-5944.patch
        298 kB
        Ishan Chattopadhyaya
      13. SOLR-5944.patch
        258 kB
        Hoss Man
      14. SOLR-5944.patch
        249 kB
        Hoss Man
      15. SOLR-5944.patch
        228 kB
        Ishan Chattopadhyaya
      16. SOLR-5944.patch
        228 kB
        Ishan Chattopadhyaya
      17. SOLR-5944.patch
        228 kB
        Ishan Chattopadhyaya
      18. SOLR-5944.patch
        228 kB
        Ishan Chattopadhyaya
      19. SOLR-5944.patch
        228 kB
        Ishan Chattopadhyaya
      20. SOLR-5944.patch
        228 kB
        Ishan Chattopadhyaya
      21. SOLR-5944.patch
        227 kB
        Ishan Chattopadhyaya
      22. SOLR-5944.patch
        204 kB
        Ishan Chattopadhyaya
      23. SOLR-5944.patch
        204 kB
        Ishan Chattopadhyaya
      24. SOLR-5944.patch
        203 kB
        Ishan Chattopadhyaya
      25. SOLR-5944.patch
        202 kB
        Ishan Chattopadhyaya
      26. SOLR-5944.patch
        202 kB
        Ishan Chattopadhyaya
      27. SOLR-5944.patch
        199 kB
        Ishan Chattopadhyaya
      28. SOLR-5944.patch
        187 kB
        Ishan Chattopadhyaya
      29. SOLR-5944.patch
        187 kB
        Ishan Chattopadhyaya
      30. SOLR-5944.patch
        180 kB
        Ishan Chattopadhyaya
      31. SOLR-5944.patch
        180 kB
        Ishan Chattopadhyaya
      32. SOLR-5944.patch
        180 kB
        Ishan Chattopadhyaya
      33. SOLR-5944.patch
        178 kB
        Ishan Chattopadhyaya
      34. SOLR-5944.patch
        157 kB
        Ishan Chattopadhyaya
      35. SOLR-5944.patch
        151 kB
        Shalin Shekhar Mangar
      36. SOLR-5944.patch
        151 kB
        Ishan Chattopadhyaya
      37. SOLR-5944.patch
        149 kB
        Ishan Chattopadhyaya
      38. SOLR-5944.patch
        145 kB
        Ishan Chattopadhyaya
      39. SOLR-5944.patch
        143 kB
        Ishan Chattopadhyaya
      40. SOLR-5944.patch
        127 kB
        Ishan Chattopadhyaya
      41. SOLR-5944.patch
        118 kB
        Ishan Chattopadhyaya
      42. SOLR-5944.patch
        119 kB
        Ishan Chattopadhyaya
      43. SOLR-5944.patch
        117 kB
        Ishan Chattopadhyaya
      44. SOLR-5944.patch
        126 kB
        Ishan Chattopadhyaya
      45. SOLR-5944.patch
        153 kB
        Hoss Man
      46. SOLR-5944.patch
        150 kB
        Hoss Man
      47. SOLR-5944.patch
        151 kB
        Ishan Chattopadhyaya
      48. SOLR-5944.patch
        140 kB
        Ishan Chattopadhyaya
      49. SOLR-5944.patch
        136 kB
        Hoss Man
      50. SOLR-5944.patch
        139 kB
        Hoss Man
      51. SOLR-5944.patch
        138 kB
        Noble Paul
      52. SOLR-5944.patch
        128 kB
        Ishan Chattopadhyaya
      53. SOLR-5944.patch
        131 kB
        Ishan Chattopadhyaya
      54. SOLR-5944.patch
        131 kB
        Ishan Chattopadhyaya
      55. SOLR-5944.patch
        131 kB
        Ishan Chattopadhyaya
      56. SOLR-5944.patch
        131 kB
        Ishan Chattopadhyaya
      57. SOLR-5944.patch
        133 kB
        Ishan Chattopadhyaya
      58. SOLR-5944.patch
        157 kB
        Ishan Chattopadhyaya
      59. SOLR-5944.patch
        107 kB
        Ishan Chattopadhyaya
      60. SOLR-5944.patch
        111 kB
        Ishan Chattopadhyaya
      61. SOLR-5944.patch
        96 kB
        Ishan Chattopadhyaya
      62. SOLR-5944.patch
        91 kB
        Ishan Chattopadhyaya
      63. SOLR-5944.patch
        74 kB
        Ishan Chattopadhyaya
      64. SOLR-5944.patch
        95 kB
        Ishan Chattopadhyaya
      65. SOLR-5944.patch
        74 kB
        Ishan Chattopadhyaya
      66. SOLR-5944.patch
        26 kB
        Ishan Chattopadhyaya
      67. SOLR-5944.patch
        22 kB
        Ishan Chattopadhyaya
      68. SOLR-5944.patch
        20 kB
        Ishan Chattopadhyaya
      69. SOLR-5944.patch
        17 kB
        Ishan Chattopadhyaya
      70. SOLR-5944.patch
        12 kB
        Ishan Chattopadhyaya
      71. SOLR-5944.patch
        9 kB
        Ishan Chattopadhyaya
      72. SOLR-5944.patch
        9 kB
        Ishan Chattopadhyaya
      73. SOLR-5944.patch
        9 kB
        Ishan Chattopadhyaya
      74. TestStressInPlaceUpdates.eb044ac71.beast-167-failure.stdout.txt
        1.37 MB
        Steve Rowe
      75. TestStressInPlaceUpdates.eb044ac71.beast-587-failure.stdout.txt
        1.80 MB
        Steve Rowe
      76. TestStressInPlaceUpdates.eb044ac71.failures.tar.gz
        2.57 MB
        Steve Rowe

        Issue Links

          Activity

          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Here's an early patch where I'm just attempting to modify DirectUpdateHandler2 to do an in-place update to a document for which the only fields to update are stored=false, indexed=false, docValues=true.

          However, I'm stuck with the unit test for this. Whenever I run the unit test (added in the patch), about 40% of the times it passes, 40% of the time it fails (the default docvalues are used, not the updated ones) and 20% of the time I'm hitting the lucene3x codecs issue (as noted by Mikhail [0] in LUCENE-5189), even though I've explicitly suppressed "lucene3x". When Solr runs as a server, I manually test that I'm able to update the NDVs via the update request handler's REST endpoint and updated values are immediately reflected for sorting queries.

          I am wondering if there is some fundamental issue with the approach taken in the patch, or if there's some issue with the test setup (or both). Help and feedback are most welcome! If I can get the tests to pass, I can begin to address various other corner cases and issues with replication etc.

          [0] - https://issues.apache.org/jira/browse/LUCENE-5189?focusedCommentId=13958205&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13958205

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Here's an early patch where I'm just attempting to modify DirectUpdateHandler2 to do an in-place update to a document for which the only fields to update are stored=false, indexed=false, docValues=true. However, I'm stuck with the unit test for this. Whenever I run the unit test (added in the patch), about 40% of the times it passes, 40% of the time it fails (the default docvalues are used, not the updated ones) and 20% of the time I'm hitting the lucene3x codecs issue (as noted by Mikhail [0] in LUCENE-5189 ), even though I've explicitly suppressed "lucene3x". When Solr runs as a server, I manually test that I'm able to update the NDVs via the update request handler's REST endpoint and updated values are immediately reflected for sorting queries. I am wondering if there is some fundamental issue with the approach taken in the patch, or if there's some issue with the test setup (or both). Help and feedback are most welcome! If I can get the tests to pass, I can begin to address various other corner cases and issues with replication etc. [0] - https://issues.apache.org/jira/browse/LUCENE-5189?focusedCommentId=13958205&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13958205
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Updating patch to trunk.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Updating patch to trunk.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Updating the patch, the test passes now! Thanks to Shalin Shekhar Mangar for pointing out that I wasn't suppressing the Lucene40, 41, .. 45 codecs in the test.
          I shall start working on addressing other issues with the patch.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Updating the patch, the test passes now! Thanks to Shalin Shekhar Mangar for pointing out that I wasn't suppressing the Lucene40, 41, .. 45 codecs in the test. I shall start working on addressing other issues with the patch.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Updated the patch to cover cases for replication of the in-place update. However, the patch still doesn't have tests for replication; working on it. Manually verified: after in-place updating a document in leader, the same document in a replica reflects the in-place update.

          TODO:
          1) Write more tests
          2) Support update of non-numeric fields using binaryDocValues

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Updated the patch to cover cases for replication of the in-place update. However, the patch still doesn't have tests for replication; working on it. Manually verified: after in-place updating a document in leader, the same document in a replica reflects the in-place update. TODO: 1) Write more tests 2) Support update of non-numeric fields using binaryDocValues
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Updated the patch to add a test for update of numeric docValues in a distributed setup (1 shard, many replicas).

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Updated the patch to add a test for update of numeric docValues in a distributed setup (1 shard, many replicas).
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Added support for the "inc" update operator (increment the existing docValue by given amount).

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Added support for the "inc" update operator (increment the existing docValue by given amount).
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Updated the patch, added support for updating a docValue using a javascript expression which binds the old docValue, thus allowing users to use simple functions to compute the updated docValue.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Updated the patch, added support for updating a docValue using a javascript expression which binds the old docValue, thus allowing users to use simple functions to compute the updated docValue.
          Hide
          shalinmangar Shalin Shekhar Mangar added a comment -

          added support for updating a docValue using a javascript expression which binds the old docValue, thus allowing users to use simple functions to compute the updated docValue

          Oh, that's awesome, Ishan! I think that should be a general improvement i.e. non-updateable numeric fields can also use this feature. Why don't we separate this feature out into it's own issue?

          Show
          shalinmangar Shalin Shekhar Mangar added a comment - added support for updating a docValue using a javascript expression which binds the old docValue, thus allowing users to use simple functions to compute the updated docValue Oh, that's awesome, Ishan! I think that should be a general improvement i.e. non-updateable numeric fields can also use this feature. Why don't we separate this feature out into it's own issue?
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Sure, that makes sense. Added SOLR-5979 to track this.
          Will remove the JavaScript part from the patch here and have a consolidated patch for both in SOLR-5979.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Sure, that makes sense. Added SOLR-5979 to track this. Will remove the JavaScript part from the patch here and have a consolidated patch for both in SOLR-5979 .
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Updated the patch with the following changes:
          1. Removed the script based updates
          2. Added support for incrementing docvalues based on uncommitted documents (from transaction log)
          3. Earlier, atomic updates were not carrying forward the non-stored, non-indexed docvalues to the new document. Added a fix for this.
          4. Added tests for 3 and 4.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Updated the patch with the following changes: 1. Removed the script based updates 2. Added support for incrementing docvalues based on uncommitted documents (from transaction log) 3. Earlier, atomic updates were not carrying forward the non-stored, non-indexed docvalues to the new document. Added a fix for this. 4. Added tests for 3 and 4.
          Hide
          mkhludnev Mikhail Khludnev added a comment -

          Ishan Chattopadhyaya I checked the last patch. It seems well. Perhaps, it's possible to better integrate it with the current atomic updates.
          I wonder if introducing DISTRIB_INPLACE_UPDATE is really necessary? I just can't see its' usage in tests.

          Committers would you mind to pay attention? It's a great alternative to clunky ExternalFileField.

          Show
          mkhludnev Mikhail Khludnev added a comment - Ishan Chattopadhyaya I checked the last patch. It seems well. Perhaps, it's possible to better integrate it with the current atomic updates. I wonder if introducing DISTRIB_INPLACE_UPDATE is really necessary? I just can't see its' usage in tests. Committers would you mind to pay attention? It's a great alternative to clunky ExternalFileField.
          Hide
          shalinmangar Shalin Shekhar Mangar added a comment -

          Hi Mikhail Khludnev, the replicas expect that the leader will send a fully expanded document and they can just add the document to the index as-is. That's not true when we're updating DocValues because we wouldn't want to lookup the stored fields at all if there were no non-updateable fields used in an atomic update request. This is why the DISTRIB_INPLACE_UPDATE is required.

          Perhaps, it's possible to better integrate it with the current atomic updates

          That and performance is my only concern at this point. We can commit it if we can deal with the docid lookup in a better/cleaner way.

          Show
          shalinmangar Shalin Shekhar Mangar added a comment - Hi Mikhail Khludnev , the replicas expect that the leader will send a fully expanded document and they can just add the document to the index as-is. That's not true when we're updating DocValues because we wouldn't want to lookup the stored fields at all if there were no non-updateable fields used in an atomic update request. This is why the DISTRIB_INPLACE_UPDATE is required. Perhaps, it's possible to better integrate it with the current atomic updates That and performance is my only concern at this point. We can commit it if we can deal with the docid lookup in a better/cleaner way.
          Hide
          yseeley@gmail.com Yonik Seeley added a comment -

          My main concern is correctness... does this patch handle reordered writes on a replica? If so, how?

          Show
          yseeley@gmail.com Yonik Seeley added a comment - My main concern is correctness... does this patch handle reordered writes on a replica? If so, how?
          Hide
          shalinmangar Shalin Shekhar Mangar added a comment -

          My main concern is correctness... does this patch handle reordered writes on a replica? If so, how?

          It's been some time since I looked at the patch. If version is passed from leader to replica then should re-ordered updates require any special handling?

          Show
          shalinmangar Shalin Shekhar Mangar added a comment - My main concern is correctness... does this patch handle reordered writes on a replica? If so, how? It's been some time since I looked at the patch. If version is passed from leader to replica then should re-ordered updates require any special handling?
          Hide
          shalinmangar Shalin Shekhar Mangar added a comment -

          If version is passed from leader to replica then should re-ordered updates require any special handling?

          Well, of course it does, because we're updating docValues only and not the version field so passing it along to a replica also doesn't make a difference. Let me think more on this. Do you have a suggestion?

          Show
          shalinmangar Shalin Shekhar Mangar added a comment - If version is passed from leader to replica then should re-ordered updates require any special handling? Well, of course it does, because we're updating docValues only and not the version field so passing it along to a replica also doesn't make a difference. Let me think more on this. Do you have a suggestion?
          Hide
          yseeley@gmail.com Yonik Seeley added a comment -

          If version is passed from leader to replica then should re-ordered updates require any special handling?

          Yes, given that real partial updates and reordered updates (as currently implemented) seem fundamentally incompatible.

          What we need is a test with 1 leader, multiple replicas, and many threads sending updates at the same time (a mix of full documents and partial updates). My guess is that would fail unless this patch (that I haven't looked at yet) does something tricky that I hadn't thought of.

          Show
          yseeley@gmail.com Yonik Seeley added a comment - If version is passed from leader to replica then should re-ordered updates require any special handling? Yes, given that real partial updates and reordered updates (as currently implemented) seem fundamentally incompatible. What we need is a test with 1 leader, multiple replicas, and many threads sending updates at the same time (a mix of full documents and partial updates). My guess is that would fail unless this patch (that I haven't looked at yet) does something tricky that I hadn't thought of.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited

          Thanks for having a look at the patch!

          Yonik Seeley I didn't tackle the reordered updates case in the patch, and can forsee a problem of correctness, as you pointed out. Since the version field could be a non-docvalue field, I didn't consider sending it in to the replicas as part of the partial document.

          Do you think enforcing the version field to be docvalue in order to use in place updates to other docvalue fields would work, whereby an updated the version field can be used in a replica to ensure consistency of reordered updates?

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited Thanks for having a look at the patch! Yonik Seeley I didn't tackle the reordered updates case in the patch, and can forsee a problem of correctness, as you pointed out. Since the version field could be a non-docvalue field, I didn't consider sending it in to the replicas as part of the partial document. Do you think enforcing the version field to be docvalue in order to use in place updates to other docvalue fields would work, whereby an updated the version field can be used in a replica to ensure consistency of reordered updates?
          Hide
          yseeley@gmail.com Yonik Seeley added a comment -

          I was just chatting with Shalin while we were both at ApacheCon. In addition to leader->replica reordering issues,
          we also need to handle realtime-get in the single-node case. The way to do this is just add the update to the tlog like normal (with some indication that it's a partial update and doesn't contain all the fields). When /get is invoked and we find an update from the in-memory tlog map for that document, we need to go through the same logic as a soft commit (open a new realtime-searcher and clear the tlog map), and then use the realtime-searcher to get the latest document.

          Oh, and version will need to use DocValues so it can be updated at the same time of course.

          Show
          yseeley@gmail.com Yonik Seeley added a comment - I was just chatting with Shalin while we were both at ApacheCon. In addition to leader->replica reordering issues, we also need to handle realtime-get in the single-node case. The way to do this is just add the update to the tlog like normal (with some indication that it's a partial update and doesn't contain all the fields). When /get is invoked and we find an update from the in-memory tlog map for that document, we need to go through the same logic as a soft commit (open a new realtime-searcher and clear the tlog map), and then use the realtime-searcher to get the latest document. Oh, and version will need to use DocValues so it can be updated at the same time of course.
          Hide
          gpatwa Gopal Patwa added a comment -

          not sure if this patch is complete but it would be nice to have this in 5.0

          Show
          gpatwa Gopal Patwa added a comment - not sure if this patch is complete but it would be nice to have this in 5.0
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Bringing the last patch up to trunk, so that further work can be done on this issue. This still suffers from the potential consistency issue in replicas if updates are reordered, as Yonik mentioned.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Bringing the last patch up to trunk, so that further work can be done on this issue. This still suffers from the potential consistency issue in replicas if updates are reordered, as Yonik mentioned.
          Hide
          erickerickson Erick Erickson added a comment -

          Is it possible to not deal with reordering if we put a big fat warning on requirements that the indexing program has to follow and work on other problems later?

          I know of a bunch of clients who would be ecstatic to have update-in-place capabilities for DocValues fields even if it required that the indexing program follow specific rules like "only one update for a particular doc can be sent per indexing request (e.g. solr.add(doclist). It'd be nice for the value to be returned in get and in results list, but even that is secondary I think.

          WARNING: I haven't looked at the code so I'm just raising the idea in case people can run with it. It'd also keep poor Ishan from continually having to update the patch...

          No chance at all of putting this in 5.1 of course, even if people like the idea. 5.2 at the earliest.

          Show
          erickerickson Erick Erickson added a comment - Is it possible to not deal with reordering if we put a big fat warning on requirements that the indexing program has to follow and work on other problems later? I know of a bunch of clients who would be ecstatic to have update-in-place capabilities for DocValues fields even if it required that the indexing program follow specific rules like "only one update for a particular doc can be sent per indexing request (e.g. solr.add(doclist) . It'd be nice for the value to be returned in get and in results list, but even that is secondary I think. WARNING: I haven't looked at the code so I'm just raising the idea in case people can run with it. It'd also keep poor Ishan from continually having to update the patch... No chance at all of putting this in 5.1 of course, even if people like the idea. 5.2 at the earliest.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Thanks for bringing this up. If we're fine with going ahead without dealing with reordered updates case, the current patch could be fine. However, it has the potential to leave the replicas out of sync, if the user is willing to take this risk / informed decision.

          I am planning to tackle this more correctly by:

          • Making _version_ as a dv field.
          • Updates that are distributed to the replicas having "old version" (if any) as well as the new version of the doc updated.
          • If, upon receiving the update on a replica, the doc version on index/tlog is not the "old version" (that means we've missed in update in between to the doc, because of reordering), then we can write this update to tlog (and mark it somehow as something we're waiting on) but not actually update the doc in the index until we receive the update whose update "old version" is what we are expecting. After doing this (for all pending updates for the doc), we could unmark the documents.
          • If we don't receive an update we're waiting for after a reasonable time period, we could mark the replica as down.

          Also, I had another idea that I like lesser than the above:
          For each dv field, we could have a dv version field, and update the document partially using just the dv field and its own version field.
          There maybe some subtleties I haven't fully thought through with this approach.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Thanks for bringing this up. If we're fine with going ahead without dealing with reordered updates case, the current patch could be fine. However, it has the potential to leave the replicas out of sync, if the user is willing to take this risk / informed decision. I am planning to tackle this more correctly by: Making _ version _ as a dv field. Updates that are distributed to the replicas having "old version" (if any) as well as the new version of the doc updated. If, upon receiving the update on a replica, the doc version on index/tlog is not the "old version" (that means we've missed in update in between to the doc, because of reordering), then we can write this update to tlog (and mark it somehow as something we're waiting on) but not actually update the doc in the index until we receive the update whose update "old version" is what we are expecting. After doing this (for all pending updates for the doc), we could unmark the documents. If we don't receive an update we're waiting for after a reasonable time period, we could mark the replica as down. Also, I had another idea that I like lesser than the above: For each dv field, we could have a dv version field, and update the document partially using just the dv field and its own version field. There maybe some subtleties I haven't fully thought through with this approach.
          Hide
          shaie Shai Erera added a comment -

          Making version as a dv field.

          I think that's a good idea and can be handled separately, as a precursor to this issue?

          For each dv field, we could have a dv version field, and update the document partially using just the dv field and its own version field.

          I think it complicates things. Now there is a _version_ field for the whole document, which isn't updated on NDV update, but a separate dvField_version... seems will get confusing really quickly. And on a whole document update operation, need to remember to reset the dvField_version for every NDV, whether it was updated in the past or not, which also adds unnecessary information to the index.

          Show
          shaie Shai Erera added a comment - Making version as a dv field. I think that's a good idea and can be handled separately, as a precursor to this issue? For each dv field, we could have a dv version field, and update the document partially using just the dv field and its own version field. I think it complicates things. Now there is a _version_ field for the whole document, which isn't updated on NDV update, but a separate dvField_version... seems will get confusing really quickly. And on a whole document update operation, need to remember to reset the dvField_version for every NDV, whether it was updated in the past or not, which also adds unnecessary information to the index.
          Hide
          yseeley@gmail.com Yonik Seeley added a comment -

          Is it possible to not deal with reordering if we put a big fat warning on requirements that the indexing program has to follow and work on other problems later?

          Too hard for us to try and figure out what can result in reordered docs, much less a user. For example, I think maybe even peer-sync could cause trouble here.

          Show
          yseeley@gmail.com Yonik Seeley added a comment - Is it possible to not deal with reordering if we put a big fat warning on requirements that the indexing program has to follow and work on other problems later? Too hard for us to try and figure out what can result in reordered docs, much less a user. For example, I think maybe even peer-sync could cause trouble here.
          Hide
          noble.paul Noble Paul added a comment -

          I would recommend the solution where _version_ is a dv field and no separate version for each updateable dv field.

          Show
          noble.paul Noble Paul added a comment - I would recommend the solution where _version_ is a dv field and no separate version for each updateable dv field.
          Hide
          yseeley@gmail.com Yonik Seeley added a comment -

          I would recommend the solution where version is a dv field and no separate version for each updateable dv field.

          +1

          Show
          yseeley@gmail.com Yonik Seeley added a comment - I would recommend the solution where version is a dv field and no separate version for each updateable dv field. +1
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Here's a WIP strawman patch.

          • Using version as docValue field.
          • Transaction log items have a previous pointer: to point from a partial update document to its previous version of the document in the tlog.
          • Realtime get works for partial updates.
          • TODO: Adding reordered updates in a follower replica doesn't work yet. (many of the supporting pieces to do this are there).
          • TODO: Lots of nocommits, which would require some careful treatment of docvalue types or need some refactoring.
          • TODO: Tests.
          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Here's a WIP strawman patch. Using version as docValue field. Transaction log items have a previous pointer: to point from a partial update document to its previous version of the document in the tlog. Realtime get works for partial updates. TODO: Adding reordered updates in a follower replica doesn't work yet. (many of the supporting pieces to do this are there). TODO: Lots of nocommits, which would require some careful treatment of docvalue types or need some refactoring. TODO: Tests.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          WIP patch, updated to trunk. The logic to deal with reorders in a replica is still TODO. Also, another TODO: this doesn't deal with multivalued docvalues yet.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - WIP patch, updated to trunk. The logic to deal with reorders in a replica is still TODO. Also, another TODO: this doesn't deal with multivalued docvalues yet.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Made some progress on this.

          • Reordered updates working.
          • Added test for reordered updates. (TODO: some more needed)
          • Fixed some issues with RealTimeGet of updates from my previous patch.

          Need to clean up the nocommits, add more tests. Also, need to deal with multivalued docvalues fields: my initial thought is to not support in-place partial updates to multivalued docvalues fields.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Made some progress on this. Reordered updates working. Added test for reordered updates. (TODO: some more needed) Fixed some issues with RealTimeGet of updates from my previous patch. Need to clean up the nocommits, add more tests. Also, need to deal with multivalued docvalues fields: my initial thought is to not support in-place partial updates to multivalued docvalues fields.
          Hide
          erickerickson Erick Erickson added a comment -

          bq: my initial thought is to not support in-place partial updates to multivalued docvalues fields

          +1, "progress not perfection" and all that.

          Seriously, there would be sooooo many people thrilled with update-in-place for a non-multiValued field that I think waiting for that support isn't necessary. Of course people will clamor for multiValued support

          Show
          erickerickson Erick Erickson added a comment - bq: my initial thought is to not support in-place partial updates to multivalued docvalues fields +1, "progress not perfection" and all that. Seriously, there would be sooooo many people thrilled with update-in-place for a non-multiValued field that I think waiting for that support isn't necessary. Of course people will clamor for multiValued support
          Hide
          yseeley@gmail.com Yonik Seeley added a comment -

          Whew... this seems really tricky. I've been diving into the Chaos* fails recently, and at first blush it seems like this would add more complexity to recovery as well (log replays, peer sync, etc?) What are the implications there?

          If, upon receiving the update on a replica, the doc version on index/tlog is not the "old version" (that means we've missed in update in between to the doc, because of reordering), then we can write this update to tlog (and mark it somehow as something we're waiting on) but not actually update the doc in the index until we receive the update whose update "old version" is what we are expecting. After doing this (for all pending updates for the doc), we could unmark the documents.

          It seems like we can't return success on an update until that update has actually been applied?
          Also, what happens to this prevPointer you are writing to the tlog if there is a commit in-between?

          Another approach would be to get rid of update reordering... i.e. ensure that updates are not reordered when sending from leader to replicas.

          Show
          yseeley@gmail.com Yonik Seeley added a comment - Whew... this seems really tricky. I've been diving into the Chaos* fails recently, and at first blush it seems like this would add more complexity to recovery as well (log replays, peer sync, etc?) What are the implications there? If, upon receiving the update on a replica, the doc version on index/tlog is not the "old version" (that means we've missed in update in between to the doc, because of reordering), then we can write this update to tlog (and mark it somehow as something we're waiting on) but not actually update the doc in the index until we receive the update whose update "old version" is what we are expecting. After doing this (for all pending updates for the doc), we could unmark the documents. It seems like we can't return success on an update until that update has actually been applied? Also, what happens to this prevPointer you are writing to the tlog if there is a commit in-between? Another approach would be to get rid of update reordering... i.e. ensure that updates are not reordered when sending from leader to replicas.
          Hide
          yseeley@gmail.com Yonik Seeley added a comment -

          Another "progress but not perfection" approach would be to get single-node working and committed and open a new issue for cloud mode support.

          Show
          yseeley@gmail.com Yonik Seeley added a comment - Another "progress but not perfection" approach would be to get single-node working and committed and open a new issue for cloud mode support.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited

          Thanks for looking at it.

          Whew... this seems really tricky. I've been diving into the Chaos* fails recently, and at first blush it seems like this would add more complexity to recovery as well (log replays, peer sync, etc?) What are the implications there?

          I need to do the due diligence and write some tests to verify that things will work with log replays and peer sync.

          Actually, since the following comment, things changed a bit (maybe be simpler?):

          If, upon receiving the update on a replica, the doc version on index/tlog is not the "old version" (that means we've missed in update in between to the doc, because of reordering), then we can write this update to tlog (and mark it somehow as something we're waiting on) but not actually update the doc in the index until we receive the update whose update "old version" is what we are expecting. After doing this (for all pending updates for the doc), we could unmark the documents.

          Changed the above to the following:

          If, upon receiving the update on a replica, the last doc's __version__ in index/tlog is
           not the "prevVersion" of the update (that means we've missed one/more updates in 
          between to the doc, because of reordering), then we write this in-place update to a temporary
           in-memory buffer and not actually write this to the tlog/index until we receive
           the update whose __version__ is what we are expecting as the prevVersion for the buffered
           update. As buffered updates get written to the tlog/index, they are removed
           from the in-memory buffer.
          

          This ensures that the tlog entries are always exactly in the order in which the documents were written.

          It seems like we can't return success on an update until that update has actually been applied?

          Good point, I haven't thought about this. Is it okay to return success if it was written to (at least) the in-memory buffer (which holds these reordered updates)? Of course, that would entail the risk of queries to this replica to return the updated document till before the point at which reordering started.

          Also, what happens to this prevPointer you are writing to the tlog if there is a commit in-between?

          This prevPointer is just used (in the patch) for RTGs. In the InPlaceUpdateDistribTest, I've introduced commits (with 1/3 probability) in between the re-ordered updates, and the RTG seems to work fine.

          Another approach would be to get rid of update reordering... i.e. ensure that updates are not reordered when sending from leader to replicas.

          Sounds interesting. How do you suggest can this be achieved?

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited Thanks for looking at it. Whew... this seems really tricky. I've been diving into the Chaos* fails recently, and at first blush it seems like this would add more complexity to recovery as well (log replays, peer sync, etc?) What are the implications there? I need to do the due diligence and write some tests to verify that things will work with log replays and peer sync. Actually, since the following comment, things changed a bit (maybe be simpler?): If, upon receiving the update on a replica, the doc version on index/tlog is not the "old version" (that means we've missed in update in between to the doc, because of reordering), then we can write this update to tlog (and mark it somehow as something we're waiting on) but not actually update the doc in the index until we receive the update whose update "old version" is what we are expecting. After doing this (for all pending updates for the doc), we could unmark the documents. Changed the above to the following: If, upon receiving the update on a replica, the last doc's __version__ in index/tlog is not the "prevVersion" of the update (that means we've missed one/more updates in between to the doc, because of reordering), then we write this in-place update to a temporary in-memory buffer and not actually write this to the tlog/index until we receive the update whose __version__ is what we are expecting as the prevVersion for the buffered update. As buffered updates get written to the tlog/index, they are removed from the in-memory buffer. This ensures that the tlog entries are always exactly in the order in which the documents were written. It seems like we can't return success on an update until that update has actually been applied? Good point, I haven't thought about this. Is it okay to return success if it was written to (at least) the in-memory buffer (which holds these reordered updates)? Of course, that would entail the risk of queries to this replica to return the updated document till before the point at which reordering started. Also, what happens to this prevPointer you are writing to the tlog if there is a commit in-between? This prevPointer is just used (in the patch) for RTGs. In the InPlaceUpdateDistribTest , I've introduced commits (with 1/3 probability) in between the re-ordered updates, and the RTG seems to work fine. Another approach would be to get rid of update reordering... i.e. ensure that updates are not reordered when sending from leader to replicas. Sounds interesting. How do you suggest can this be achieved?
          Hide
          yseeley@gmail.com Yonik Seeley added a comment -

          I need to do the due diligence and write some tests to verify that things will work with log replays and peer sync.

          Yeah, things are tricky enough in this area (distributed updates / recovery in general) that one can't really validate through tests. Need to brainstorm and think through all of the different failure scenarios and then try to use tests to uncover scenarios you hadn't considered.

          This prevPointer is just used (in the patch) for RTGs. In the InPlaceUpdateDistribTest, I've introduced commits (with 1/3 probability) in between the re-ordered updates, and the RTG seems to work fine.

          Ah, that makes sense now (for standalone / leader at least).

          Off the top of my head, here's a possible issue:

          • replica buffers a partial update in memory (because it was reordered)
          • a commit comes in, and we roll over to a new tlog
          • node goes down and then comes back up, and the in-memory update is no longer in memory, and the old tlog won't be replayed. It will look like we applied that update.

          Is it okay to return success if it was written to (at least) the in-memory buffer (which holds these reordered updates)?

          I don't think so... another scenario:

          • a client does an in-place update
          • the replicas receive the update reordered, and buffer in memory.
          • the client gets the response
          • the leader goes down (and stays down... hard drive crash)
          • one of the other replicas takes over as leader, but we've now lost data we confirmed as written (the in-place update was only ever applied on the leader), even though we only lost 1 server.

          And then there is the even simpler scenario I think you were alluding to: if an update is ack'd, then a RTG on any active replica should see that update (or a later one).

          Show
          yseeley@gmail.com Yonik Seeley added a comment - I need to do the due diligence and write some tests to verify that things will work with log replays and peer sync. Yeah, things are tricky enough in this area (distributed updates / recovery in general) that one can't really validate through tests. Need to brainstorm and think through all of the different failure scenarios and then try to use tests to uncover scenarios you hadn't considered. This prevPointer is just used (in the patch) for RTGs. In the InPlaceUpdateDistribTest, I've introduced commits (with 1/3 probability) in between the re-ordered updates, and the RTG seems to work fine. Ah, that makes sense now (for standalone / leader at least). Off the top of my head, here's a possible issue: replica buffers a partial update in memory (because it was reordered) a commit comes in, and we roll over to a new tlog node goes down and then comes back up, and the in-memory update is no longer in memory, and the old tlog won't be replayed. It will look like we applied that update. Is it okay to return success if it was written to (at least) the in-memory buffer (which holds these reordered updates)? I don't think so... another scenario: a client does an in-place update the replicas receive the update reordered, and buffer in memory. the client gets the response the leader goes down (and stays down... hard drive crash) one of the other replicas takes over as leader, but we've now lost data we confirmed as written (the in-place update was only ever applied on the leader), even though we only lost 1 server. And then there is the even simpler scenario I think you were alluding to: if an update is ack'd, then a RTG on any active replica should see that update (or a later one).
          Hide
          yseeley@gmail.com Yonik Seeley added a comment -

          > Another approach would be to get rid of update reordering... i.e. ensure that updates are not reordered when sending from leader to replicas.

          Sounds interesting. How do you suggest can this be achieved?

          Don't reorder updates between leader and replicas:

          • create a new ConcurrentUpdateSolrClient that uses a single channel and can return individual responses... perhaps this fits into HTTP/2 ?
          • have only a single SolrClient on the leader talk to each replica
          • order the udpates in _version_ order when sending
            • prob multiple ways to achieve this... reserve a slot when getting the version, or change versions so that they are contiguous so we know if we are missing one.

          The only additional reason to use multiple threads when sending is to increase indexing performance. We can also implement multi-threading for increased parallelism on the server side. This should also simplify clients (no more batching, multiple threads, etc), as well as make our general recovery system more robust.

          Show
          yseeley@gmail.com Yonik Seeley added a comment - > Another approach would be to get rid of update reordering... i.e. ensure that updates are not reordered when sending from leader to replicas. Sounds interesting. How do you suggest can this be achieved? Don't reorder updates between leader and replicas: create a new ConcurrentUpdateSolrClient that uses a single channel and can return individual responses... perhaps this fits into HTTP/2 ? have only a single SolrClient on the leader talk to each replica order the udpates in _version_ order when sending prob multiple ways to achieve this... reserve a slot when getting the version, or change versions so that they are contiguous so we know if we are missing one. The only additional reason to use multiple threads when sending is to increase indexing performance. We can also implement multi-threading for increased parallelism on the server side. This should also simplify clients (no more batching, multiple threads, etc), as well as make our general recovery system more robust.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited

          I was thinking on changing the approach to make sure such inconsistencies (during recovery, failures etc.) can be avoided. Essentially, I'm thinking of changing the approach from buffering the inplace updates (as in the last patch) to, instead, letting leader threads, carrying out of order updates, to wait till the dependent update has been applied on the replica before the replica writes the current inplace update to its tlog/index and only then getting an ACK.
          I think, though I may be missing something, the reordered updates would be a rare occurrence and the delay for the required update to arrive will be in order of milliseconds, and hence there wouldn't be too much of an overhead to waiting on those rare occasions.
          My initial thought is that doing this will not add any more complexity to log replays as of today, and the replicas will stay in sync.
          Yonik Seeley What do you think?

          Don't reorder updates between leader and replicas:

          Provided the above approach (or some simple variant thereof) works, maybe we don't absolutely need to do this for supporting inplace updates? I think we explore this anyway, irrespective of this current issue. I can open another issue for doing this.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited I was thinking on changing the approach to make sure such inconsistencies (during recovery, failures etc.) can be avoided. Essentially, I'm thinking of changing the approach from buffering the inplace updates (as in the last patch) to, instead, letting leader threads, carrying out of order updates, to wait till the dependent update has been applied on the replica before the replica writes the current inplace update to its tlog/index and only then getting an ACK. I think, though I may be missing something, the reordered updates would be a rare occurrence and the delay for the required update to arrive will be in order of milliseconds, and hence there wouldn't be too much of an overhead to waiting on those rare occasions. My initial thought is that doing this will not add any more complexity to log replays as of today, and the replicas will stay in sync. Yonik Seeley What do you think? Don't reorder updates between leader and replicas: Provided the above approach (or some simple variant thereof) works, maybe we don't absolutely need to do this for supporting inplace updates? I think we explore this anyway, irrespective of this current issue. I can open another issue for doing this.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited

          Here's the approach I'm talking about, in details (also for the benefit of someone who hasn't followed the previous approaches/discussions closely):

          1. Make _version_ as a docValues field, indexed=false, stored=false SOLR-6337.
          2. When an inplace update command (which syntactically looks like an atomic update) is received on the leader, the leader
            1. versions the update and applies it to its index,
            2. obtains the prevVersion as the _version_ of the document on which the update was applied to,
            3. writes the inplace update (only the partial document) in its tlog along with a prevPointer, corresponding to the tlog entry of the document with prevVersion or -1 if such an entry doesn't exist, of the update,
            4. forward the update to the other replicas, along with the prevVersion.
          3. At a replica, upon receiving an inplace update, the last version (lets call it lastVersion) for the document id is obtained from the tlog/index.
            Now,
            1. If this matches the prevVersion, then apply the update, write to tlog and return success to the leader.
            2. If this doesn't match the prevVersion (case of an out of order update), wait (till a timeout) for the update with prevVersion to be written to tlog/index, possibly in other threads, and then apply the current update and return success to the leader. If a timeout has reached, but the dependent update hasn't arrived, a failure is sent back to the leader (which can then mark the replica in recovery).
          4. RealTimeGet for documents that have inplace updates can be resolved by successively following a prevPointer in the tlog backwards, either to a full update in tlog or to the index.
          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited Here's the approach I'm talking about, in details (also for the benefit of someone who hasn't followed the previous approaches/discussions closely): Make _version_ as a docValues field, indexed=false, stored=false SOLR-6337 . When an inplace update command (which syntactically looks like an atomic update) is received on the leader, the leader versions the update and applies it to its index, obtains the prevVersion as the _version_ of the document on which the update was applied to, writes the inplace update (only the partial document) in its tlog along with a prevPointer , corresponding to the tlog entry of the document with prevVersion or -1 if such an entry doesn't exist, of the update, forward the update to the other replicas, along with the prevVersion. At a replica, upon receiving an inplace update, the last version (lets call it lastVersion ) for the document id is obtained from the tlog/index. Now, If this matches the prevVersion , then apply the update, write to tlog and return success to the leader. If this doesn't match the prevVersion (case of an out of order update), wait (till a timeout) for the update with prevVersion to be written to tlog/index, possibly in other threads, and then apply the current update and return success to the leader. If a timeout has reached, but the dependent update hasn't arrived, a failure is sent back to the leader (which can then mark the replica in recovery). RealTimeGet for documents that have inplace updates can be resolved by successively following a prevPointer in the tlog backwards, either to a full update in tlog or to the index.
          Hide
          noble.paul Noble Paul added a comment -

          If this doesn't match the prevVersion (case of an out of order update), wait (till a timeout) for the update with prevVersion to be written to tlog/index, possibly in other threads,

          Looks like this needs to be nailed in more detail. What exactly happens to other writes when an out of order update reaches a replica?. Will the proposed change have any impact on the performance for users who do not use this feature?

          Don't reorder updates between leader and replicas: create a new ConcurrentUpdateSolrClient that uses a single channel and can return individual responses...

          This is probably beyond the scope of this ticket. If the solution proposed by Ishan Chattopadhyaya is fine (looks fine to me) we should go ahead with his solution and tackle the other stuff in a separate ticket.

          Show
          noble.paul Noble Paul added a comment - If this doesn't match the prevVersion (case of an out of order update), wait (till a timeout) for the update with prevVersion to be written to tlog/index, possibly in other threads, Looks like this needs to be nailed in more detail. What exactly happens to other writes when an out of order update reaches a replica?. Will the proposed change have any impact on the performance for users who do not use this feature? Don't reorder updates between leader and replicas: create a new ConcurrentUpdateSolrClient that uses a single channel and can return individual responses... This is probably beyond the scope of this ticket. If the solution proposed by Ishan Chattopadhyaya is fine (looks fine to me) we should go ahead with his solution and tackle the other stuff in a separate ticket.
          Hide
          yseeley@gmail.com Yonik Seeley added a comment -

          I think, though I may be missing something, the reordered updates would be a rare occurrence and the delay for the required update to arrive will be in order of milliseconds, and hence there wouldn't be too much of an overhead to waiting on those rare occasions.

          It's not so rare if multiple clients are updating the same doc. Milliseconds can be pushed to many seconds due to GC, etc. Also, it gets very tricky if a replica goes down... I haven't thought through that part. What happens if the dependent update fails on one shard?

          instead, letting leader threads, carrying out of order updates, to wait till the dependent update has been applied on the replica

          "leader threads" meaning they are on the leader? If so, this is sort of where I was going before (handling more on the leader)... except instead of detecting and waiting for the dependent update, just send the dependent update first (i.e. don't reorder). Seems easier?
          Hmmm, but looking at your follow-up, it sounds more like you're blocking on the replica?

          > Don't reorder updates between leader and replicas
          This is probably beyond the scope of this ticket.

          It certainly is beyond the scope of this ticket, but it also would solve a lot of issues. We need to continue to think about any and all approaches to make this feature bullet-proof.

          Will all "atomic updates" now be handled this way? Seems desirable in general.

          Scanning back over this issue, I happened to see:

          3. Earlier, atomic updates were not carrying forward the non-stored, non-indexed docvalues to the new document. Added a fix for this.

          If it hasn't been done already, this could be broken out into it's own issue and fixed now?

          Show
          yseeley@gmail.com Yonik Seeley added a comment - I think, though I may be missing something, the reordered updates would be a rare occurrence and the delay for the required update to arrive will be in order of milliseconds, and hence there wouldn't be too much of an overhead to waiting on those rare occasions. It's not so rare if multiple clients are updating the same doc. Milliseconds can be pushed to many seconds due to GC, etc. Also, it gets very tricky if a replica goes down... I haven't thought through that part. What happens if the dependent update fails on one shard? instead, letting leader threads, carrying out of order updates, to wait till the dependent update has been applied on the replica "leader threads" meaning they are on the leader? If so, this is sort of where I was going before (handling more on the leader)... except instead of detecting and waiting for the dependent update, just send the dependent update first (i.e. don't reorder). Seems easier? Hmmm, but looking at your follow-up, it sounds more like you're blocking on the replica? > Don't reorder updates between leader and replicas This is probably beyond the scope of this ticket. It certainly is beyond the scope of this ticket, but it also would solve a lot of issues. We need to continue to think about any and all approaches to make this feature bullet-proof. Will all "atomic updates" now be handled this way? Seems desirable in general. Scanning back over this issue, I happened to see: 3. Earlier, atomic updates were not carrying forward the non-stored, non-indexed docvalues to the new document. Added a fix for this. If it hasn't been done already, this could be broken out into it's own issue and fixed now?
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          What happens if the dependent update fails on one shard?

          If the dependent update (or even the out of order update) failed on a replica (isn't this what you meant by "shard"?), I think the leader will put that replica into LIR.

          "leader threads" meaning they are on the leader? If so, this is sort of where I was going before (handling more on the leader)... except instead of detecting and waiting for the dependent update, just send the dependent update first (i.e. don't reorder). Seems easier? Hmmm, but looking at your follow-up, it sounds more like you're blocking on the replica?

          What I meant by "leader threads" was that since at the leader the DistributedUpdateProcessor distributes these commands to the replicas synchronously [0], the leader thread will remain blocked till the replica will not write and return success/failure. However, my intention was, as you point out, to actually do the wait/"blocking on the replica".

          [0] - DistributedUpdateProcessor

                  for (Node subShardLeader : subShardLeaders) {
                    cmdDistrib.distribAdd(cmd, Collections.singletonList(subShardLeader), params, true);
                  }
          

          (the last boolean being for synchronous)

          It certainly is beyond the scope of this ticket, but it also would solve a lot of issues. We need to continue to think about any and all approaches to make this feature bullet-proof.

          Totally agree. I really want to evaluate doing this, even if not immediately (unless we absolutely need it for this feature). If there is not a significant performance loss to using a single channel client to each replica, then this approach would be really very neat.

          Will all "atomic updates" now be handled this way? Seems desirable in general.

          For atomic updates, as they stand today, I think sending only partial document to the replicas would help if there are lots of fields as we save the cost of sending and maybe also storage/memory space at the tlog. However, if the other stored fields require very expensive analysis, it might be better to let the leader send the full document to the replicas? Is there something else too which I missed? If we can get in-place updates to work for both numeric and binary docvalues, I think we'll get one step closer to recommending users to do all updates this way.

          If it hasn't been done already, this could be broken out into it's own issue and fixed now?

          Sure, good catch. I'll create another issue for this.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - What happens if the dependent update fails on one shard? If the dependent update (or even the out of order update) failed on a replica (isn't this what you meant by "shard"?), I think the leader will put that replica into LIR. "leader threads" meaning they are on the leader? If so, this is sort of where I was going before (handling more on the leader)... except instead of detecting and waiting for the dependent update, just send the dependent update first (i.e. don't reorder). Seems easier? Hmmm, but looking at your follow-up, it sounds more like you're blocking on the replica? What I meant by "leader threads" was that since at the leader the DistributedUpdateProcessor distributes these commands to the replicas synchronously [0] , the leader thread will remain blocked till the replica will not write and return success/failure. However, my intention was, as you point out, to actually do the wait/"blocking on the replica". [0] - DistributedUpdateProcessor for (Node subShardLeader : subShardLeaders) { cmdDistrib.distribAdd(cmd, Collections.singletonList(subShardLeader), params, true); } (the last boolean being for synchronous) It certainly is beyond the scope of this ticket, but it also would solve a lot of issues. We need to continue to think about any and all approaches to make this feature bullet-proof. Totally agree. I really want to evaluate doing this, even if not immediately (unless we absolutely need it for this feature). If there is not a significant performance loss to using a single channel client to each replica, then this approach would be really very neat. Will all "atomic updates" now be handled this way? Seems desirable in general. For atomic updates, as they stand today, I think sending only partial document to the replicas would help if there are lots of fields as we save the cost of sending and maybe also storage/memory space at the tlog. However, if the other stored fields require very expensive analysis, it might be better to let the leader send the full document to the replicas? Is there something else too which I missed? If we can get in-place updates to work for both numeric and binary docvalues, I think we'll get one step closer to recommending users to do all updates this way. If it hasn't been done already, this could be broken out into it's own issue and fixed now? Sure, good catch. I'll create another issue for this.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited

          Looks like this needs to be nailed in more detail. What exactly happens to other writes when an out of order update reaches a replica?.

          I think those other writes (i.e. writes for other documents) would continue to happen, since they are being performed using separate threads. As for other writes to the same document, I think the consequence would be that they'll keep getting blocked until the dependent update is written to the replica (after which they'll get unblocked one by one, unless they timeout).

          Milliseconds can be pushed to many seconds due to GC, etc. Also, it gets very tricky if a replica goes down...

          Do you guys think it would be reasonable to let user configure this, and document somewhere on the lines of: if you have lots of back of back updates happening on the same documents, then there will be a possibility of threads waiting for other threads, and hence this timeout can be increased, say, to seconds.

          Will the proposed change have any impact on the performance for users who do not use this feature?

          I think there will be no impact, since this wait would only affect in-place updates. Unless if we want to make sure there will be no reordering in the first place.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited Looks like this needs to be nailed in more detail. What exactly happens to other writes when an out of order update reaches a replica?. I think those other writes (i.e. writes for other documents) would continue to happen, since they are being performed using separate threads. As for other writes to the same document, I think the consequence would be that they'll keep getting blocked until the dependent update is written to the replica (after which they'll get unblocked one by one, unless they timeout). Milliseconds can be pushed to many seconds due to GC, etc. Also, it gets very tricky if a replica goes down... Do you guys think it would be reasonable to let user configure this, and document somewhere on the lines of: if you have lots of back of back updates happening on the same documents, then there will be a possibility of threads waiting for other threads, and hence this timeout can be increased, say, to seconds. Will the proposed change have any impact on the performance for users who do not use this feature? I think there will be no impact, since this wait would only affect in-place updates. Unless if we want to make sure there will be no reordering in the first place.
          Hide
          shaie Shai Erera added a comment -

          I'll admit that I haven't dug through all the comments on the issue, and I'm not very familiar w/ the internals of Solr document updates, but at least from a Lucene perspective, numeric DV updates occur in order with any other document updates. So, simplifying on purpose, if Solr's tlog already handles document updates in order, and it will now list the updates like so:

          DOC_UPDATE: { "id: "doc1", "fields": {field1, field2 ....}}
          DOC_UPDATE: { "id: "doc2", "fields": {field1, field2 ....}}
          NUMERIC_UPDATE: { "term" : { "field" : "id", "value" : "doc1"}, "dvField": 123 }
          DOC_UPDATE: { "id: "doc1", "fields": {field1, field2 ....}}
          

          What would cause the second update to 'doc1' apply before the NUMERIC update of 'dvField' of 'doc1'? Also, if the numeric update affects a group of documents, this should be OK still (from a Lucene standpoint).

          Or ... is the problem you're discussing a numeric update can affect a group of documents that are located on different shards? If so, doesn't Solr already protect against that, versioning each change?

          If I'm missing something fundamental in how Solr works, then I'd be happy go learn/read about it. I wanted to give the perspective of numeric DV updates from a Lucene standpoint.

          Show
          shaie Shai Erera added a comment - I'll admit that I haven't dug through all the comments on the issue, and I'm not very familiar w/ the internals of Solr document updates, but at least from a Lucene perspective, numeric DV updates occur in order with any other document updates. So, simplifying on purpose, if Solr's tlog already handles document updates in order, and it will now list the updates like so: DOC_UPDATE: { "id: "doc1", "fields": {field1, field2 ....}} DOC_UPDATE: { "id: "doc2", "fields": {field1, field2 ....}} NUMERIC_UPDATE: { "term" : { "field" : "id", "value" : "doc1"}, "dvField": 123 } DOC_UPDATE: { "id: "doc1", "fields": {field1, field2 ....}} What would cause the second update to 'doc1' apply before the NUMERIC update of 'dvField' of 'doc1'? Also, if the numeric update affects a group of documents, this should be OK still (from a Lucene standpoint). Or ... is the problem you're discussing a numeric update can affect a group of documents that are located on different shards? If so, doesn't Solr already protect against that, versioning each change? If I'm missing something fundamental in how Solr works, then I'd be happy go learn/read about it. I wanted to give the perspective of numeric DV updates from a Lucene standpoint.
          Hide
          yseeley@gmail.com Yonik Seeley added a comment -

          What would cause the second update to 'doc1' apply before the NUMERIC update of 'dvField' of 'doc1'?

          The fundamental issue is this: the leader can forward updates to a replica over different threads / connections, and can be handled my multiple threads on the replica side. This leads to many opportunities for update reorders.

          Solr does currently detect (via versioning) and handle all of the reorder cases today. When a document is sent to a replica, the whole thing is sent. If a replica gets an older version of a document, it can simply drop it. This type of handling doesn't work when you're talking about re-ordered partial / in-place updates... just dropping old versions doesn't work.

          Show
          yseeley@gmail.com Yonik Seeley added a comment - What would cause the second update to 'doc1' apply before the NUMERIC update of 'dvField' of 'doc1'? The fundamental issue is this: the leader can forward updates to a replica over different threads / connections, and can be handled my multiple threads on the replica side. This leads to many opportunities for update reorders. Solr does currently detect (via versioning) and handle all of the reorder cases today. When a document is sent to a replica, the whole thing is sent. If a replica gets an older version of a document, it can simply drop it. This type of handling doesn't work when you're talking about re-ordered partial / in-place updates... just dropping old versions doesn't work.
          Hide
          yseeley@gmail.com Yonik Seeley added a comment - - edited

          Stepping back and looking the high level view of Ishan's last proposal/approach, I think it looks fine.
          Once we get into true partial udpates, we can't really allow reorders, so they either have to be ordered on the sender side, or the receiver side. From a complexity POV, once we have to order certain types of updates, it's no more complex to order them all.

          If I could wave a magic wand, we'd have the order-on-the-sender-side approach, because it should also solve known weaknesses like I appear to be hitting in SOLR-8129, but I recognize it would be a lot of work. Of course, reordering on the receiver side is not going to be a picnic either - the synchronization involved may be quite difficult to get right.

          edit: some of the other parts of the design, like the prevPointer & prevVersion seem fine regardless of how other parts of this puzzle are solved.

          Show
          yseeley@gmail.com Yonik Seeley added a comment - - edited Stepping back and looking the high level view of Ishan's last proposal/approach, I think it looks fine. Once we get into true partial udpates, we can't really allow reorders, so they either have to be ordered on the sender side, or the receiver side. From a complexity POV, once we have to order certain types of updates, it's no more complex to order them all. If I could wave a magic wand, we'd have the order-on-the-sender-side approach, because it should also solve known weaknesses like I appear to be hitting in SOLR-8129 , but I recognize it would be a lot of work. Of course, reordering on the receiver side is not going to be a picnic either - the synchronization involved may be quite difficult to get right. edit: some of the other parts of the design, like the prevPointer & prevVersion seem fine regardless of how other parts of this puzzle are solved.
          Hide
          shaie Shai Erera added a comment -

          Thanks Yonik for the clarification. The issue then is that the version field is not 'updateable' right? If it was e.g. a numeric DV field itself, then each update to a document could involve updating both the version field, as well the numeric DV field in question. And regular doc updates would work fine as well.

          I remember reading about making the version field a numeric DV for that (or similar) purpose, so apologies if I re-iterate someone else's idea...

          Show
          shaie Shai Erera added a comment - Thanks Yonik for the clarification. The issue then is that the version field is not 'updateable' right? If it was e.g. a numeric DV field itself, then each update to a document could involve updating both the version field, as well the numeric DV field in question. And regular doc updates would work fine as well. I remember reading about making the version field a numeric DV for that (or similar) purpose, so apologies if I re-iterate someone else's idea...
          Hide
          yseeley@gmail.com Yonik Seeley added a comment -

          The issue then is that the version field is not 'updateable' right?

          Well, that was one issue we identified earlier and decided we would move to docvals for that field. It's a good idea in any case.

          Show
          yseeley@gmail.com Yonik Seeley added a comment - The issue then is that the version field is not 'updateable' right? Well, that was one issue we identified earlier and decided we would move to docvals for that field. It's a good idea in any case.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          I am close to a patch for the above proposal, and shall post it soon.

          One place where I am somewhere stuck is in the log buffering/replaying part. Here's the problem:
          When a replica is put into recovery by the leader, it comes back up and tries to perform a peersync. This seems to be happening in a two phase process: buffering (where the updates, after being obtained from the leader's tlog, are played back and written to the replica's tlog but not its index/ulog) and replaying (where the tlog is replayed and the updates are written to ulog/index, but not into tlog again). The problem I'm facing is that during this buffering phase, the inplace updates can't find dependent updates if they are not in the index, since the updates are not written to ulog in the buffering phase.

          I have two choices at the moment to get around this:

          1. During a buffering phase, I can keep a separate map of all updates (id to tlog pointer) to be used during and discarded after the buffering phase. That map can help resolve inplace updates that follow. (Pro: fast, Con: memory)
          2. For every inplace update, I traverse back into the tlog and linearly scan for the required dependent update. (Pro: no memory, Con: Slow / O)

          At this point, I'm inclined to go for option 1, but I'm wondering if there are any serious downsides to doing this. Any suggestions, please?
          Also, am I correct in my assumption that the no. of updates processed during this buffering phase will not be more than numUpdatesToKeep?
          In case I sound confused/unclear, please let me know and I'll post the relevant failing test for this.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - I am close to a patch for the above proposal, and shall post it soon. One place where I am somewhere stuck is in the log buffering/replaying part. Here's the problem: When a replica is put into recovery by the leader, it comes back up and tries to perform a peersync. This seems to be happening in a two phase process: buffering (where the updates, after being obtained from the leader's tlog, are played back and written to the replica's tlog but not its index/ulog) and replaying (where the tlog is replayed and the updates are written to ulog/index, but not into tlog again). The problem I'm facing is that during this buffering phase, the inplace updates can't find dependent updates if they are not in the index, since the updates are not written to ulog in the buffering phase. I have two choices at the moment to get around this: During a buffering phase, I can keep a separate map of all updates (id to tlog pointer) to be used during and discarded after the buffering phase. That map can help resolve inplace updates that follow. (Pro: fast, Con: memory) For every inplace update, I traverse back into the tlog and linearly scan for the required dependent update. (Pro: no memory, Con: Slow / O ) At this point, I'm inclined to go for option 1, but I'm wondering if there are any serious downsides to doing this. Any suggestions, please? Also, am I correct in my assumption that the no. of updates processed during this buffering phase will not be more than numUpdatesToKeep ? In case I sound confused/unclear, please let me know and I'll post the relevant failing test for this.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Adding patch for the above approach.

          1. Added a test that times out a reordered update, when it exceeds 1s (currently hardcoded), and puts the replica in LIR and subsequently performs a peer sync.
          2. Used a bufferingMap in UpdateLog, and populated it at DistributedUpdateProcessor at a time when ulog is in buffering state.

          TODOs:

          1. Remove the nocommits, and some of the TODOs.
          2. Add more tests.
          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Adding patch for the above approach. Added a test that times out a reordered update, when it exceeds 1s (currently hardcoded), and puts the replica in LIR and subsequently performs a peer sync. Used a bufferingMap in UpdateLog, and populated it at DistributedUpdateProcessor at a time when ulog is in buffering state. TODOs: Remove the nocommits, and some of the TODOs. Add more tests.
          Hide
          yseeley@gmail.com Yonik Seeley added a comment -

          Also, am I correct in my assumption that the no. of updates processed during this buffering phase will not be more than numUpdatesToKeep?

          That's a minimum, not a maximum. There really is no maximum.

          Show
          yseeley@gmail.com Yonik Seeley added a comment - Also, am I correct in my assumption that the no. of updates processed during this buffering phase will not be more than numUpdatesToKeep? That's a minimum, not a maximum. There really is no maximum.
          Hide
          yseeley@gmail.com Yonik Seeley added a comment -

          Do we have any sort of conditional update yet (or something generic like update-by-script)? That would be great to help test this stuff out (i.e. to make sure that partial updates aren't reordered with respect to each-other).

          If you're going to be around the conference this week, It would be great to chat about some of this stuff in person...

          Show
          yseeley@gmail.com Yonik Seeley added a comment - Do we have any sort of conditional update yet (or something generic like update-by-script)? That would be great to help test this stuff out (i.e. to make sure that partial updates aren't reordered with respect to each-other). If you're going to be around the conference this week, It would be great to chat about some of this stuff in person...
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Do we have any sort of conditional update yet (or something generic like update-by-script)? That would be great to help test this stuff out (i.e. to make sure that partial updates aren't reordered with respect to each-other).

          "inc" is supported, I can try writing a suite that indexes bunch of full documents, "set" updates and "inc" updates (perhaps along with _version_s for optimistic concurrency) and then test for consistency across replicas. By update-by-script, do you mean something like SOLR-5979 (Alas it is not there yet)?

          If you're going to be around the conference this week, It would be great to chat about some of this stuff in person...

          +1. I'm planning to be around on 15th and 16th.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Do we have any sort of conditional update yet (or something generic like update-by-script)? That would be great to help test this stuff out (i.e. to make sure that partial updates aren't reordered with respect to each-other). "inc" is supported, I can try writing a suite that indexes bunch of full documents, "set" updates and "inc" updates (perhaps along with _version_s for optimistic concurrency) and then test for consistency across replicas. By update-by-script, do you mean something like SOLR-5979 (Alas it is not there yet)? If you're going to be around the conference this week, It would be great to chat about some of this stuff in person... +1. I'm planning to be around on 15th and 16th.
          Hide
          noble.paul Noble Paul added a comment -

          I guess yonik meant something like CAS. Anyway let's meet up during LSR

          Show
          noble.paul Noble Paul added a comment - I guess yonik meant something like CAS. Anyway let's meet up during LSR
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Had productive discussions around this at Revolution with Yonik Seeley and Noble Paul. Here's a list of TODO items I am going to work on:
          Here's a summary:

          1. Expand the current tests to do more updates, using more threads, something like stress reorder test does. [Yonik]
          2. Before timing out (in a replica) after waiting for dependent update long enough, instead of throwing an error (which triggers LIR), request from replica to leader to pull a range of updates (the missing ones), i.e. from what last version we have to the version we expect. This can be easily done by using (a slightly modified version of) the RTG's getUpdates handler. [Noble]
          3. Since there are changes to the javabin format (for inplace updates), add tests for backcompat. [Noble]
          4. Code cleanup, fix the todos/nocommits etc.
          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Had productive discussions around this at Revolution with Yonik Seeley and Noble Paul . Here's a list of TODO items I am going to work on: Here's a summary: Expand the current tests to do more updates, using more threads, something like stress reorder test does. [Yonik] Before timing out (in a replica) after waiting for dependent update long enough, instead of throwing an error (which triggers LIR), request from replica to leader to pull a range of updates (the missing ones), i.e. from what last version we have to the version we expect. This can be easily done by using (a slightly modified version of) the RTG's getUpdates handler. [Noble] Since there are changes to the javabin format (for inplace updates), add tests for backcompat. [Noble] Code cleanup, fix the todos/nocommits etc.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Updating patch, depends on SOLR-8220.
          If waiting for pending/reordered dependent update fails, it tries to get the updates from the leader now (as per point 2 in above comment). Fixed a few nocommits.
          Still need to add the extensive stress tests, and maybe some more refactoring needed.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Updating patch, depends on SOLR-8220 . If waiting for pending/reordered dependent update fails, it tries to get the updates from the leader now (as per point 2 in above comment). Fixed a few nocommits. Still need to add the extensive stress tests, and maybe some more refactoring needed.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Updating to trunk.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Updating to trunk.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Updating the patch to latest master.

          1. Added stress test, TestStressInPlaceUpdates, similar to TestStressReorder
          2. Has lots and lots of unnecessary logging info, which I left there as I tried to reproduce a bug (LUCENE-7025) due ot which the stress test is failing.
          3. Lots of nocommits due to formatting/refactoring/unnecessary logging etc.

          Currently, in the stress test, RTG works, but committed model / main searcher based search seems to be inconsistent, possibly due to the LUCENE-7025 issue or something to do with that. Seems to me that whatever is being written through an IndexWriter is not exactly what I search via searcher at the time when parallel commits are happening. I am still investigating why that could be.

          It seems the problem of reordered updates is solved (in terms of correctness). Also seems the RTG is working properly.

          I am currently looking into the searcher inconsistency while commit is happening. Once I get past that, I'll do a serious cleanup of the code and the logging messages I am printing out.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Updating the patch to latest master. Added stress test, TestStressInPlaceUpdates, similar to TestStressReorder Has lots and lots of unnecessary logging info, which I left there as I tried to reproduce a bug ( LUCENE-7025 ) due ot which the stress test is failing. Lots of nocommits due to formatting/refactoring/unnecessary logging etc. Currently, in the stress test, RTG works, but committed model / main searcher based search seems to be inconsistent, possibly due to the LUCENE-7025 issue or something to do with that. Seems to me that whatever is being written through an IndexWriter is not exactly what I search via searcher at the time when parallel commits are happening. I am still investigating why that could be. It seems the problem of reordered updates is solved (in terms of correctness). Also seems the RTG is working properly. I am currently looking into the searcher inconsistency while commit is happening. Once I get past that, I'll do a serious cleanup of the code and the logging messages I am printing out.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Updated the patch to trunk with the following:
          1. Some bug fixes.
          2. Stress test.
          3. Some cleanups.
          4. Fixed nocommits.

          Noble Paul, Yonik Seeley, could you please review? Thanks.

          There is one very very rare situation whereby after a commit the recently reopened searcher doesn't see a dv update which was supposed to be in this newly opened searcher. There maybe some issue in having the updates flushed properly at the Lucene level; I have so far been unable to reproduce this during one off plain Lucene tests or Solr tests, but it is exposed only during the Stress tests (once in 2000 runs or so). I'll continue to investigate this, perhaps by writing a similar stress test for plain Lucene level DV updates. Apart from that, I think this patch is feature complete.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Updated the patch to trunk with the following: 1. Some bug fixes. 2. Stress test. 3. Some cleanups. 4. Fixed nocommits. Noble Paul , Yonik Seeley , could you please review? Thanks. There is one very very rare situation whereby after a commit the recently reopened searcher doesn't see a dv update which was supposed to be in this newly opened searcher. There maybe some issue in having the updates flushed properly at the Lucene level; I have so far been unable to reproduce this during one off plain Lucene tests or Solr tests, but it is exposed only during the Stress tests (once in 2000 runs or so). I'll continue to investigate this, perhaps by writing a similar stress test for plain Lucene level DV updates. Apart from that, I think this patch is feature complete.
          Hide
          gpatwa Gopal Patwa added a comment -

          Great! to see progress and near to completion this feature. can this be part of 6.0 release?

          Show
          gpatwa Gopal Patwa added a comment - Great! to see progress and near to completion this feature. can this be part of 6.0 release?
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Hi Gopal, sorry to have missed your comment. I think this can land in Solr 6.1 or later, since 6.0 release branch has already been cut.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Hi Gopal, sorry to have missed your comment. I think this can land in Solr 6.1 or later, since 6.0 release branch has already been cut.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Updating the patch to master, since parts of this patch are now committed as part of SOLR-8831 and SOLR-8865. Would be nice if someone could please review.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Updating the patch to master, since parts of this patch are now committed as part of SOLR-8831 and SOLR-8865 . Would be nice if someone could please review.
          Hide
          noble.paul Noble Paul added a comment -

          In DUP.waitForDependentUpdates() it is not wise to blindly wait for 10 milliseconds. What if the dependent update came right after it entered the sleep() . In that case we are unnecessarily waiting while the thread could proceed immediately . We can use a wait()/notify() mechanism to avoid that

          Show
          noble.paul Noble Paul added a comment - In DUP.waitForDependentUpdates() it is not wise to blindly wait for 10 milliseconds. What if the dependent update came right after it entered the sleep() . In that case we are unnecessarily waiting while the thread could proceed immediately . We can use a wait()/notify() mechanism to avoid that
          Hide
          noble.paul Noble Paul added a comment -

          use a wait()/notify() instead of sleep()

          Show
          noble.paul Noble Paul added a comment - use a wait()/notify() instead of sleep()
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Thanks Noble, I added your changes to the full patch.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Thanks Noble, I added your changes to the full patch.
          Hide
          noble.paul Noble Paul added a comment -

          There are a lot of places where a toString() is done and then parsing is done. It much cheaper to do an instanceof check
          look at the following block in AtomicUpdateDocumentMerger.doInPlaceUpdateMerge()

           if (oldValue instanceof Long) {
                              result = ((Long) oldValue).longValue() + Long.parseLong(value.toString());
                            } else if (oldValue instanceof Float) {
                              result = ((Float) oldValue).floatValue() + Float.parseFloat(value.toString());
                            } else if (oldValue instanceof Double) {
                              result = ((Double) oldValue).doubleValue() + Double.parseDouble(value.toString());
                            } else {
                              // int, short, byte
                              result = ((Integer) oldValue).intValue() + Integer.parseInt(value.toString());
                            }
          

          can be optimized as follows

          
              if (oldValue instanceof Long) {
                              result = (Long) oldValue + (value instanceof Number ? ((Number) value).longValue() : Long.parseLong(value.toString()));
                            } else if (oldValue instanceof Float) {
                              result = (Float) oldValue + (value instanceof Number ? ((Number) value).floatValue() : Float.parseFloat(value.toString()));
                            } else if (oldValue instanceof Double) {
                              result = (Double) oldValue + (value instanceof Number ? ((Number) value).doubleValue() : Double.parseDouble(value.toString()));
                            } else {
                              // int, short, byte
                              result = (Integer) oldValue + (value instanceof Number ? ((Number) value).intValue() : Integer.parseInt(value.toString()));
                            }
          
          Show
          noble.paul Noble Paul added a comment - There are a lot of places where a toString() is done and then parsing is done. It much cheaper to do an instanceof check look at the following block in AtomicUpdateDocumentMerger.doInPlaceUpdateMerge() if (oldValue instanceof Long ) { result = (( Long ) oldValue).longValue() + Long .parseLong(value.toString()); } else if (oldValue instanceof Float ) { result = (( Float ) oldValue).floatValue() + Float .parseFloat(value.toString()); } else if (oldValue instanceof Double ) { result = (( Double ) oldValue).doubleValue() + Double .parseDouble(value.toString()); } else { // int , short , byte result = (( Integer ) oldValue).intValue() + Integer .parseInt(value.toString()); } can be optimized as follows if (oldValue instanceof Long ) { result = ( Long ) oldValue + (value instanceof Number ? (( Number ) value).longValue() : Long .parseLong(value.toString())); } else if (oldValue instanceof Float ) { result = ( Float ) oldValue + (value instanceof Number ? (( Number ) value).floatValue() : Float .parseFloat(value.toString())); } else if (oldValue instanceof Double ) { result = ( Double ) oldValue + (value instanceof Number ? (( Number ) value).doubleValue() : Double .parseDouble(value.toString())); } else { // int , short , byte result = ( Integer ) oldValue + (value instanceof Number ? (( Number ) value).intValue() : Integer .parseInt(value.toString())); }
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited

          Thanks Noble, I've added your suggestion and your DUP patch to the full patch. Also did some cleanup, javadocs, removed some unnecessary logging.

          The corresponding change to the patch are:
          https://github.com/chatman/lucene-solr/commit/8f38aa663e57335afe5f136139acc2e7293d2a9d
          https://github.com/chatman/lucene-solr/commit/f203d770dbdcb36edf472544902bc62bbbc28794

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited Thanks Noble, I've added your suggestion and your DUP patch to the full patch. Also did some cleanup, javadocs, removed some unnecessary logging. The corresponding change to the patch are: https://github.com/chatman/lucene-solr/commit/8f38aa663e57335afe5f136139acc2e7293d2a9d https://github.com/chatman/lucene-solr/commit/f203d770dbdcb36edf472544902bc62bbbc28794
          Hide
          noble.paul Noble Paul added a comment - - edited

          fixed logging.

          • It's not OK to cache the values of log.isDebugEnabled() etc. These can change in between
          • if a string is constructed, always do conditional logging
          • if a template is used upto two variables eg: log.info("v1:{}, v2:{}", v1,v2 ) , it is OK not to do conditional logging, because no new objects are created
          Show
          noble.paul Noble Paul added a comment - - edited fixed logging. It's not OK to cache the values of log.isDebugEnabled() etc. These can change in between if a string is constructed, always do conditional logging if a template is used upto two variables eg: log.info("v1:{}, v2:{}", v1,v2 ) , it is OK not to do conditional logging, because no new objects are created
          Hide
          noble.paul Noble Paul added a comment -

          is there a testcase for partial update as the first operation? instead of a full insert (like set and inc)

          Show
          noble.paul Noble Paul added a comment - is there a testcase for partial update as the first operation? instead of a full insert (like set and inc)
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Thats a good point, Noble; there isn't a test, I'll add one.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Thats a good point, Noble; there isn't a test, I'll add one.
          Hide
          noble.paul Noble Paul added a comment -

          Ishan Chattopadhyaya I'm done with my review. I have done some minor changes to your private branch https://github.com/chatman/lucene-solr and branch solr_5944 . We have an issue that causes some data loss very rarely. if that is fixed, I'm +1 to commit this

          Show
          noble.paul Noble Paul added a comment - Ishan Chattopadhyaya I'm done with my review. I have done some minor changes to your private branch https://github.com/chatman/lucene-solr and branch solr_5944 . We have an issue that causes some data loss very rarely. if that is fixed, I'm +1 to commit this
          Hide
          jdeolive Justin Deoliveira added a comment -

          I've been following this patch for a while, and am super excited about recent progress. I just applied the latest patch locally and built with maven and it resulted in some forbidden api failures. I don't know of this helps but here is a minor patch that addresses them.

          Show
          jdeolive Justin Deoliveira added a comment - I've been following this patch for a while, and am super excited about recent progress. I just applied the latest patch locally and built with maven and it resulted in some forbidden api failures. I don't know of this helps but here is a minor patch that addresses them.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited

          The only remaining issue is a test failure, which used to happen very intermittently for me with TestStressInPlaceUpdates. Here are the partial logs (from April):

            [beaster]   2> NOTE: reproduce with: ant test  -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=3E8431D13FA03373 -Dtests.slow=true -Dtests.locale=sr-Latn -Dtests.timezone=Antarctica/Syowa -Dtests.asserts=true -Dtests.file.encoding=US-ASCII
            [beaster] [18:34:34.713] ERROR   51.8s | TestStressInPlaceUpdates.stressTest <<<
            [beaster]    > Throwable #1: com.carrotsearch.randomizedtesting.UncaughtExceptionError: Captured an uncaught exception in thread: Thread[id=181, name=READER10, state=RUNNABLE, group=TGRP-TestStressInPlaceUpdates]
            [beaster]    >        at __randomizedtesting.SeedInfo.seed([3E8431D13FA03373:55E2EE7C0175E789]:0)
            [beaster]    > Caused by: java.lang.RuntimeException: java.lang.AssertionError: Vals are: 3, 2000000008, id=4
            [beaster]    >        at __randomizedtesting.SeedInfo.seed([3E8431D13FA03373]:0)
            [beaster]    >        at org.apache.solr.cloud.TestStressInPlaceUpdates$3.run(TestStressInPlaceUpdates.java:416)
            [beaster]    > Caused by: java.lang.AssertionError: Vals are: 3, 2000000008, id=4
            [beaster]    >        at org.junit.Assert.fail(Assert.java:93)
            [beaster]    >        at org.junit.Assert.assertTrue(Assert.java:43)
            [beaster]    >        at org.apache.solr.cloud.TestStressInPlaceUpdates$3.run(TestStressInPlaceUpdates.java:390)
          

          However, I ran a beast of the same test on a 24 core (48 thread) machine, 1000 rounds, 16 at a time (http://imgur.com/Y2929rI), and the beasting passed. Maybe I'll need to try harder to reproduce it over the week or the coming weekend.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited The only remaining issue is a test failure, which used to happen very intermittently for me with TestStressInPlaceUpdates. Here are the partial logs (from April): [beaster] 2> NOTE: reproduce with: ant test -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=3E8431D13FA03373 -Dtests.slow= true -Dtests.locale=sr-Latn -Dtests.timezone=Antarctica/Syowa -Dtests.asserts= true -Dtests.file.encoding=US-ASCII [beaster] [18:34:34.713] ERROR 51.8s | TestStressInPlaceUpdates.stressTest <<< [beaster] > Throwable #1: com.carrotsearch.randomizedtesting.UncaughtExceptionError: Captured an uncaught exception in thread: Thread [id=181, name=READER10, state=RUNNABLE, group=TGRP-TestStressInPlaceUpdates] [beaster] > at __randomizedtesting.SeedInfo.seed([3E8431D13FA03373:55E2EE7C0175E789]:0) [beaster] > Caused by: java.lang.RuntimeException: java.lang.AssertionError: Vals are: 3, 2000000008, id=4 [beaster] > at __randomizedtesting.SeedInfo.seed([3E8431D13FA03373]:0) [beaster] > at org.apache.solr.cloud.TestStressInPlaceUpdates$3.run(TestStressInPlaceUpdates.java:416) [beaster] > Caused by: java.lang.AssertionError: Vals are: 3, 2000000008, id=4 [beaster] > at org.junit.Assert.fail(Assert.java:93) [beaster] > at org.junit.Assert.assertTrue(Assert.java:43) [beaster] > at org.apache.solr.cloud.TestStressInPlaceUpdates$3.run(TestStressInPlaceUpdates.java:390) However, I ran a beast of the same test on a 24 core (48 thread) machine, 1000 rounds, 16 at a time ( http://imgur.com/Y2929rI ), and the beasting passed. Maybe I'll need to try harder to reproduce it over the week or the coming weekend.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Thanks for the forbidden API fixes. I shall incorporate them into the next patch.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Thanks for the forbidden API fixes. I shall incorporate them into the next patch.
          Hide
          hossman Hoss Man added a comment -

          Noble Paul added a comment - 04/Apr/16 02:20

          Ishan Chattopadhyaya I'm done with my review. I have done some minor changes to your private branch https://github.com/chatman/lucene-solr and branch solr_5944 . We have an issue that causes some data loss very rarely. if that is fixed, I'm +1 to commit this

          There are so many questions raised by this comment...

          • what are the minor changes? This comment was posted long after the most recent patch attached this this issue (on Mar31), implying that whatever the the changes are they didn't make it into any attached patch. The comments says the changes were made to a github repo on a "solr_5944" branch, but the linked repo doesn't have a branch with that name – it DOES have a branch named "SOLR-5944" but the most recent commit on that branch is from Feb – 2 months before this comment was posted (and way older then the most recent patch attached to this issue)
          • what is the issue that causes data loss? can you describe the "rarely" situation? is there a test that demonstrates the problem? ... is this the same (non-reproducible) test failure that Ishan mentioned in in his comment this morning (May16) or is there some other bug we need to be worried about?
          Show
          hossman Hoss Man added a comment - Noble Paul added a comment - 04/Apr/16 02:20 Ishan Chattopadhyaya I'm done with my review. I have done some minor changes to your private branch https://github.com/chatman/lucene-solr and branch solr_5944 . We have an issue that causes some data loss very rarely. if that is fixed, I'm +1 to commit this There are so many questions raised by this comment... what are the minor changes? This comment was posted long after the most recent patch attached this this issue (on Mar31), implying that whatever the the changes are they didn't make it into any attached patch. The comments says the changes were made to a github repo on a "solr_5944" branch, but the linked repo doesn't have a branch with that name – it DOES have a branch named " SOLR-5944 " but the most recent commit on that branch is from Feb – 2 months before this comment was posted (and way older then the most recent patch attached to this issue) what is the issue that causes data loss? can you describe the "rarely" situation? is there a test that demonstrates the problem? ... is this the same (non-reproducible) test failure that Ishan mentioned in in his comment this morning (May16) or is there some other bug we need to be worried about?
          Hide
          hossman Hoss Man added a comment -

          this is the most recent patch (attached by Noble on Mar31) updated to master (some hunk offset shifts and one trivial conflict) with Justin's precommit fixes added in.

          (precommit & the new test classes pass, still running fulltests to ensure no obvious problems, have not actually reviewed anything personally)

          Show
          hossman Hoss Man added a comment - this is the most recent patch (attached by Noble on Mar31) updated to master (some hunk offset shifts and one trivial conflict) with Justin's precommit fixes added in. (precommit & the new test classes pass, still running fulltests to ensure no obvious problems, have not actually reviewed anything personally)
          Hide
          hossman Hoss Man added a comment -

          small update – the previous patch was breaking QueryEqualityTest because of how it abused ValueSourceParser.addParser (SOLR-9119) to add a ValueSource parser to the static list. If this special ValueSource parser was critical to the test, it could have been registered in the solrconfig.xml used by the test, but it didn't appear to serve any purpose beyond what the DocIdAugmenterFactory already provided, so i replaced the existing usages with that transformer (if anything the old code could have suffered from false positives since the valuesource was returning the segment docId, but the transformer returns the id used by the top level SolrIndexSearcher(

          Show
          hossman Hoss Man added a comment - small update – the previous patch was breaking QueryEqualityTest because of how it abused ValueSourceParser.addParser ( SOLR-9119 ) to add a ValueSource parser to the static list. If this special ValueSource parser was critical to the test, it could have been registered in the solrconfig.xml used by the test, but it didn't appear to serve any purpose beyond what the DocIdAugmenterFactory already provided, so i replaced the existing usages with that transformer (if anything the old code could have suffered from false positives since the valuesource was returning the segment docId, but the transformer returns the id used by the top level SolrIndexSearcher(
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Thanks for looking into the issue, Hoss.

          The comments says the changes were made to a github repo on a "solr_5944" branch, but the linked repo doesn't have a branch with that name

          https://github.com/chatman/lucene-solr/tree/solr_5944
          Noble's last commit is: 4572983839e3943b7dea52a8a2d55aa2b3b5ca3a

          what is the issue that causes data loss

          When it happens, my understanding is that some uncommitted in-place updates don't reflect after a commit in a re-opened searcher.

          can you describe the "rarely" situation? is there a test that demonstrates the problem?

          TestStressInPlaceUpdates is the affected test. Used to happen once in around 2000 runs, on a 3Ghz 5960X CPU (I'm unable to reproduce, as I mentioned in my previous comment, on dual 2.0Ghz Xeon E5 2658 V3).

          is this the same (non-reproducible) test failure that Ishan mentioned in in his comment this morning (May16)

          https://issues.apache.org/jira/browse/SOLR-5944?focusedCommentId=15193478&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15193478
          I think Noble is referring to the same. It could be due to some underlying issue with either how Solr uses Lucene or how Lucene is flushing the dv updates during a commit.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Thanks for looking into the issue, Hoss. The comments says the changes were made to a github repo on a "solr_5944" branch, but the linked repo doesn't have a branch with that name https://github.com/chatman/lucene-solr/tree/solr_5944 Noble's last commit is: 4572983839e3943b7dea52a8a2d55aa2b3b5ca3a what is the issue that causes data loss When it happens, my understanding is that some uncommitted in-place updates don't reflect after a commit in a re-opened searcher. can you describe the "rarely" situation? is there a test that demonstrates the problem? TestStressInPlaceUpdates is the affected test. Used to happen once in around 2000 runs, on a 3Ghz 5960X CPU (I'm unable to reproduce, as I mentioned in my previous comment, on dual 2.0Ghz Xeon E5 2658 V3). is this the same (non-reproducible) test failure that Ishan mentioned in in his comment this morning (May16) https://issues.apache.org/jira/browse/SOLR-5944?focusedCommentId=15193478&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15193478 I think Noble is referring to the same. It could be due to some underlying issue with either how Solr uses Lucene or how Lucene is flushing the dv updates during a commit.
          Hide
          hossman Hoss Man added a comment -

          https://github.com/chatman/lucene-solr/tree/solr_5944 Noble's last commit is: 4572983839e3943b7dea52a8a2d55aa2b3b5ca3a

          Ugh ... somehow i completley missed seeing this comment yesterday, I don't know why i couldn't find that branch on github.

          Show
          hossman Hoss Man added a comment - https://github.com/chatman/lucene-solr/tree/solr_5944 Noble's last commit is: 4572983839e3943b7dea52a8a2d55aa2b3b5ca3a Ugh ... somehow i completley missed seeing this comment yesterday, I don't know why i couldn't find that branch on github.
          Hide
          hossman Hoss Man added a comment -

          oh ... wait ... i see now, it was posted after my other comments/attachments ... but using the "Reply" feature so it appears inline.

          Show
          hossman Hoss Man added a comment - oh ... wait ... i see now, it was posted after my other comments/attachments ... but using the "Reply" feature so it appears inline.
          Hide
          steve_rowe Steve Rowe added a comment -

          but using the "Reply" feature so it appears inline.

          Yeah, I've stopped using the Reply thing because of this - you can't find all the new posts at the bottom if people use this misfeature (as Muir called it).

          Show
          steve_rowe Steve Rowe added a comment - but using the "Reply" feature so it appears inline. Yeah, I've stopped using the Reply thing because of this - you can't find all the new posts at the bottom if people use this misfeature (as Muir called it).
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Combined Justin's fixes, Hoss' fixes and Noble's fixes (which were already there), updated to master and committed to the solr_5944 branch (https://github.com/chatman/lucene-solr/tree/solr_5944). Attached the patch for the same here.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Combined Justin's fixes, Hoss' fixes and Noble's fixes (which were already there), updated to master and committed to the solr_5944 branch ( https://github.com/chatman/lucene-solr/tree/solr_5944 ). Attached the patch for the same here.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Yeah, I've stopped using the Reply thing because of this

          I see.. I'll consider stopping the use of the reply feature now Sorry for the confusion.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Yeah, I've stopped using the Reply thing because of this I see.. I'll consider stopping the use of the reply feature now Sorry for the confusion.
          Hide
          steve_rowe Steve Rowe added a comment -

          I'm beasting 2000 iterations of TestStressInPlaceUpdates with Miller's beast script against https://github.com/chatman/lucene-solr/tree/solr_5944 at revision eb044ac71 and have so far seen two failures, at iteration 167 and at iteration 587, the stdout from which I'm attaching here.

          Show
          steve_rowe Steve Rowe added a comment - I'm beasting 2000 iterations of TestStressInPlaceUpdates with Miller's beast script against https://github.com/chatman/lucene-solr/tree/solr_5944 at revision eb044ac71 and have so far seen two failures, at iteration 167 and at iteration 587, the stdout from which I'm attaching here.
          Hide
          hossman Hoss Man added a comment -

          FWIW, I'm testing ishan's latest patch against lucene-solr master and the two "reproduce" lines from steve's logs (minus the linedocs path) fail 100% of the time for me on my box - although the specific doc listed in the failure message varies from run to run, presumably because of the parallel threads? ...

          ant test  -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=FFC46C473EC471E6 -Dtests.slow=true -Dtests.locale=sr-ME -Dtests.timezone=Europe/Riga -Dtests.asserts=true -Dtests.file.encoding=UTF-8
          
          ant test  -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=15E180DC7142CBF3 -Dtests.slow=true -Dtests.locale=pt-BR -Dtests.timezone=Africa/Juba -Dtests.asserts=true -Dtests.file.encoding=UTF-8
          
          Show
          hossman Hoss Man added a comment - FWIW, I'm testing ishan's latest patch against lucene-solr master and the two "reproduce" lines from steve's logs (minus the linedocs path) fail 100% of the time for me on my box - although the specific doc listed in the failure message varies from run to run, presumably because of the parallel threads? ... ant test -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=FFC46C473EC471E6 -Dtests.slow=true -Dtests.locale=sr-ME -Dtests.timezone=Europe/Riga -Dtests.asserts=true -Dtests.file.encoding=UTF-8 ant test -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=15E180DC7142CBF3 -Dtests.slow=true -Dtests.locale=pt-BR -Dtests.timezone=Africa/Juba -Dtests.asserts=true -Dtests.file.encoding=UTF-8
          Hide
          steve_rowe Steve Rowe added a comment -

          Both of those seeds (FFC46C473EC471E6 and 15E180DC7142CBF3) reproduce for me too (only tried each one once).

          A third beasting failure, run 783, does NOT reproduce for me (0 failures out of 4 runs):

          ant test  -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=CCB5FA74FA9BB974 -Dtests.slow=true -Dtests.locale=sr -Dtests.timezone=Africa/Gaborone -Dtests.asserts=true -Dtests.file.encoding=US-ASCII
          
          Show
          steve_rowe Steve Rowe added a comment - Both of those seeds (FFC46C473EC471E6 and 15E180DC7142CBF3) reproduce for me too (only tried each one once). A third beasting failure, run 783, does NOT reproduce for me (0 failures out of 4 runs): ant test -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=CCB5FA74FA9BB974 -Dtests.slow=true -Dtests.locale=sr -Dtests.timezone=Africa/Gaborone -Dtests.asserts=true -Dtests.file.encoding=US-ASCII
          Hide
          hossman Hoss Man added a comment -

          I've been reviewing the logs from sarowe's failures – I won't pretend to understand half of what i'm looking at here (I'm still not up on most of the new code) but here's some interesting patterns i've noticed...

          • in both failure logs posted, doc "13" was the doc having problems
          • the specific docId is probably just a coincidence, but it does mean that the same egrep command works on both log files to give you the particularly interesting bits realtive to the failure...
            egrep add=\\[13\|id=13\|ids=13 TestStressInPlaceUpdates.eb044ac71.beast-167-failure.stdout.txt > beast-167.important.txt
            egrep add=\\[13\|id=13\|ids=13 TestStressInPlaceUpdates.eb044ac71.beast-587-failure.stdout.txt > beast-587.important.txt
            
          • looking first at beast-587.important.txt:
            • the ERROR that failed the test was first logged by READER2 @ (timestamp) 34456:
              34456 ERROR (READER2) [    ] o.a.s.c.TestStressInPlaceUpdates Realtime=true, ERROR, id=13 found={response={numFound=1,start=0,docs=[SolrDocument{id=13, title_s=[title13], val1_i_dvo=3, val2_l_dvo=3000000006, _version_=1534607778351415296, ratings=0.0, price=0}]}} model=[1534607780231512064, 3, 3000000012]
              
            • Working backwards, that expected version 1534607780231512064 was logged by WRITER10 as being returned to a PARTIAL update @ 31219:
              31219 INFO  (WRITER10) [    ] o.a.s.c.TestStressInPlaceUpdates PARTIAL: Writing id=13, val=[3,3000000012], version=1534607779993485312, Prev was=[3,3000000009].  Returned version=1534607780231512064
              
              • WRITER10's logging of this "Returned version=1534607780231512064" came after core_node1, core_node2, and core_node3 all logged it being written to their TLOG & reported it via LogUpdateProc:
                30557 INFO  (qtp2010985731-180) [n:127.0.0.1:37972__m c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607780231512064, prevVersion=1534607779993485312, prevPtr=2343) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-1-001/cores/collection1/data/tlog/tlog.0000000000000000004 refcount=1} LogPtr(2396) map=1977112331, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000012, _version_=1534607780231512064, val1_i_dvo=3])
                30589 INFO  (qtp1755078679-232) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607780231512064, prevVersion=1534607779993485312, prevPtr=2343) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-2-001/cores/collection1/data/tlog/tlog.0000000000000000002 refcount=1} LogPtr(2396) map=1630836284, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000012, _version_=1534607780231512064, val1_i_dvo=3])
                30589 INFO  (qtp1755078679-232) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.p.LogUpdateProcessorFactory [collection1]  webapp=/_m path=/update params={update.distrib=FROMLEADER&distrib.from=http://127.0.0.1:37972/_m/collection1/&distrib.inplace.prevversion=1534607779993485312&wt=javabin&version=2&distrib.inplace.update=true}{add=[13 (1534607780231512064)]} 0 0
                31216 INFO  (qtp2143623462-144) [n:127.0.0.1:58295__m c:collection1 s:shard1 r:core_node3 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607780231512064, prevVersion=1534607779993485312, prevPtr=2343) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-3-001/cores/collection1/data/tlog/tlog.0000000000000000002 refcount=1} LogPtr(2396) map=1500522809, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000012, _version_=1534607780231512064, val1_i_dvo=3])
                31216 INFO  (qtp2143623462-144) [n:127.0.0.1:58295__m c:collection1 s:shard1 r:core_node3 x:collection1] o.a.s.u.p.LogUpdateProcessorFactory [collection1]  webapp=/_m path=/update params={update.distrib=FROMLEADER&distrib.from=http://127.0.0.1:37972/_m/collection1/&distrib.inplace.prevversion=1534607779993485312&wt=javabin&version=2&distrib.inplace.update=true}{add=[13 (1534607780231512064)]} 0 0
                31219 INFO  (qtp2010985731-180) [n:127.0.0.1:37972__m c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.p.LogUpdateProcessorFactory [collection1]  webapp=/_m path=/update params={versions=true&wt=javabin&version=2}{add=[13 (1534607780231512064)]} 0 662
                
            • but looking after the ERROR was first logged @ 34456, we see that before the test had a chance to shut down all the nodes, there was some suspicious looking logging from core_node2 regarding updates out of order, that refer to the expected 1534607780231512064 version:
              34976 INFO  (qtp1755078679-111) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.p.DistributedUpdateProcessor Fetched the update: add{_version_=1534607780231512064,id=13}
              35810 INFO  (qtp1755078679-111) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.p.DistributedUpdateProcessor Fetched the update: add{_version_=1534607779993485312,id=13}
              35811 INFO  (qtp1755078679-111) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607779993485312, prevVersion=1534607778351415296, prevPtr=-1) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-2-001/cores/collection1/data/tlog/tlog.0000000000000000002 refcount=1} LogPtr(3955) map=748019145, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000009, _version_=1534607779993485312, val1_i_dvo=3])
              35811 INFO  (qtp1755078679-111) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.p.DistributedUpdateProcessor Fetched missing dependent updates from leader, which likely succeeded. Current update: add{_version_=1534607780231512064,id=13}, Missing update: add{_version_=1534607779993485312,id=13}
              35811 INFO  (qtp1755078679-111) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607780231512064, prevVersion=1534607779993485312, prevPtr=3955) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-2-001/cores/collection1/data/tlog/tlog.0000000000000000002 refcount=1} LogPtr(4015) map=748019145, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000012, _version_=1534607780231512064, val1_i_dvo=3])
              35811 INFO  (qtp1755078679-111) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.p.DistributedUpdateProcessor Fetched missing dependent updates from leader, which likely succeeded. Current update: add{,id=13}, Missing update: add{_version_=1534607780231512064,id=13}
              35811 INFO  (qtp1755078679-111) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607783219953664, prevVersion=1534607780231512064, prevPtr=4015) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-2-001/cores/collection1/data/tlog/tlog.0000000000000000002 refcount=1} LogPtr(4069) map=748019145, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000015, _version_=1534607783219953664, val1_i_dvo=3])
              35812 INFO  (qtp1755078679-111) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.p.LogUpdateProcessorFactory [collection1]  webapp=/_m path=/update params={update.distrib=FROMLEADER&distrib.from=http://127.0.0.1:37972/_m/collection1/&distrib.inplace.prevversion=1534607780231512064&wt=javabin&version=2&distrib.inplace.update=true}{add=[13 (1534607783219953664)]} 0 1714
              
              • these might be unrelated to the failure, since (IIRC) realtime get is always routed to the leader (can somone confirm this?) so this replica needing to fetch these out of order updates from the leader may not actaully be an indication of a problem – but it did catch my eye.
            • if we change our focus and direct it at the "actual" version 1534607778351415296 that was returned by the realtime get causing the test failure, we notice it being recorded by WRITER1 as a partial update @ 28769:
              28769 INFO  (WRITER11) [    ] o.a.s.c.TestStressInPlaceUpdates PARTIAL: Writing id=13, val=[3,3000000006], version=1534607778339880960, Prev was=[3,3000000003].  Returned version=1534607778351415296
              
              • This is subsequent to the expected TLOG and LogUpdateProc messages we would expect for this update
            • Starting at timestamp 30330, we see the expected log messages for the update that should have replaced 1534607778351415296, with params indicating that 1534607778351415296 is the previous version:
              30330 INFO  (qtp2010985731-180) [n:127.0.0.1:37972__m c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607779993485312, prevVersion=1534607778351415296, prevPtr=636) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-1-001/cores/collection1/data/tlog/tlog.0000000000000000004 refcount=1} LogPtr(2343) map=1977112331, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000009, _version_=1534607779993485312, val1_i_dvo=3])
              30331 INFO  (qtp2143623462-144) [n:127.0.0.1:58295__m c:collection1 s:shard1 r:core_node3 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607779993485312, prevVersion=1534607778351415296, prevPtr=636) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-3-001/cores/collection1/data/tlog/tlog.0000000000000000002 refcount=1} LogPtr(2343) map=1500522809, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000009, _version_=1534607779993485312, val1_i_dvo=3])
              30332 INFO  (qtp2143623462-144) [n:127.0.0.1:58295__m c:collection1 s:shard1 r:core_node3 x:collection1] o.a.s.u.p.LogUpdateProcessorFactory [collection1]  webapp=/_m path=/update params={update.distrib=FROMLEADER&distrib.from=http://127.0.0.1:37972/_m/collection1/&distrib.inplace.prevversion=1534607778351415296&wt=javabin&version=2&distrib.inplace.update=true}{add=[13 (1534607779993485312)]} 0 0
              30336 INFO  (qtp1755078679-241) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607779993485312, prevVersion=1534607778351415296, prevPtr=636) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-2-001/cores/collection1/data/tlog/tlog.0000000000000000002 refcount=1} LogPtr(2343) map=1630836284, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000009, _version_=1534607779993485312, val1_i_dvo=3])
              30336 INFO  (qtp1755078679-241) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.p.LogUpdateProcessorFactory [collection1]  webapp=/_m path=/update params={update.distrib=FROMLEADER&distrib.from=http://127.0.0.1:37972/_m/collection1/&distrib.inplace.prevversion=1534607778351415296&wt=javabin&version=2&distrib.inplace.update=true}{add=[13 (1534607779993485312)]} 0 0
              30396 INFO  (qtp2143623462-227) [n:127.0.0.1:58295__m c:collection1 s:shard1 r:core_node3 x:collection1] o.a.s.c.S.Request [collection1]  webapp=/_m path=/get params={qt=/get&ids=13&wt=javabin&version=2} status=0 QTime=0
              30441 INFO  (qtp1755078679-106) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.c.S.Request [collection1]  webapp=/_m path=/get params={qt=/get&ids=13&wt=javabin&version=2} status=0 QTime=0
              30469 INFO  (qtp2010985731-180) [n:127.0.0.1:37972__m c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.p.LogUpdateProcessorFactory [collection1]  webapp=/_m path=/update params={versions=true&wt=javabin&version=2}{add=[13 (1534607779993485312)]} 0 140
              30556 INFO  (WRITER10) [    ] o.a.s.c.TestStressInPlaceUpdates PARTIAL: Writing id=13, val=[3,3000000009], version=1534607778351415296, Prev was=[3,3000000006].  Returned version=1534607779993485312
              
            • BUT! Just before this logging about replacing 1534607778351415296 w/ 1534607779993485312 there is a log message mentioning version 1534607778351415296 that jumped out at me @ 30329:
              30329 INFO  (qtp2010985731-180) [n:127.0.0.1:37972__m c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.p.AtomicUpdateDocumentMerger Uncommitted doc is: SolrInputDocument(fields: [id=13, val2_l_dvo=3000000006, _version_=1534607778351415296, val1_i_dvo=3])
              
              • I don't know anything about this AtomicUpdateDocumentMerger class, but this logging smells particularly suspicious to me because it is the last time AtomicUpdateDocumentMerger logs anything about doc #13
              • is this an indication that the "state" AtomicUpdateDocumentMerger is tracking for doc #13 isn't getting updated when the (subsequently expected) version 1534607780231512064 is added?
          • switching to beast-167.important.txt:
            • the ERROR that failed the test was first logged by READER0 @ 32666:
              32666 ERROR (READER0) [    ] o.a.s.c.TestStressInPlaceUpdates Realtime=true, ERROR, id=13 found={response={numFound=1,start=0,docs=[SolrDocument{id=13, title_s=[title13], val1_i_dvo=2, val2_l_dvo=2000000002, _version_=1534599975700267008, ratings=0.0, price=0}]}} model=[1534599977073901568, 2, 2000000004]
              
            • this test didn't linger long enough to see if any interesting "out of order update" messages were logged by replicas after the failure.
            • the expected version 1534599977073901568 is reported as the returned version by WRITER5 @ 32108:
              32108 INFO  (WRITER5) [    ] o.a.s.c.TestStressInPlaceUpdates PARTIAL: Writing id=13, val=[2,2000000004], version=1534599975700267008, Prev was=[2,2000000002].  Returned version=1534599977073901568
              
              • This is subsequent to the expected TLOG and LogUpdateProc messages we would expect for this update
            • We again see no logging from AtomicUpdateDocumentMerger regarding the expected version 1534599977073901568
            • We DO see logging from AtomicUpdateDocumentMerger regarding the actual version return in the request that failed the assertion:
              32082 INFO  (qtp498157202-244) [n:127.0.0.1:55136__nge%2Fr c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.p.AtomicUpdateDocumentMerger Uncommitted doc is: SolrInputDocument(fields: [id=13, val2_l_dvo=2000000002, _version_=1534599975700267008])
              
              • This is again the last (and only) log message from AtomicUpdateDocumentMerger regarding doc #13

          In my own hammering of TestStressInPlaceUpdates, i've encountered a handful of failures that haven't reproduced 100% reliably, but most of the failures i've seen have followed this pattern

          {red}The most recent version logged by AtomicUpdateDocumentMerger matches the 'actual' version logged in an ERROR assertion failure message{red}
          • hoss.62D328FA1DEA57FD.fail.txt - follows pattern:
            23263 INFO  (qtp905061545-62) [n:127.0.0.1:43930_i%2Fop c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.p.AtomicUpdateDocumentMerger Uncommitted doc is: SolrInputDocument(fields: [id=4, val2_l_dvo=3000000009, _version_=1534612078380187648, val1_i_dvo=3])
            ...
            23543 ERROR (READER3) [    ] o.a.s.c.TestStressInPlaceUpdates Realtime=true, ERROR, id=4 found={response={numFound=1,start=0,docs=[SolrDocument{id=4, title_s=[title4], val1_i_dvo=3, val2_l_dvo=3000000009, _version_=1534612078380187648, ratings=0.0, price=0}]}} model=[1534612078708391936, 3, 3000000012]
            
          • hoss.62D328FA1DEA57FD.fail2.txt - does NOT follow the pattern:
            32481 INFO  (qtp1749706169-67) [n:127.0.0.1:48694_i%2Fop c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.p.AtomicUpdateDocumentMerger Uncommitted doc is: SolrInputDocument(fields: [id=13, val2_l_dvo=23000000023, _version_=1534622450566823936, val1_i_dvo=23])
            ...
            32581 INFO  (qtp1749706169-238) [n:127.0.0.1:48694_i%2Fop c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.p.AtomicUpdateDocumentMerger Uncommitted doc is: SolrInputDocument(fields: [id=13, val2_l_dvo=23000000046, _version_=1534622450634981376, val1_i_dvo=23])
            ...
            32661 ERROR (READER7) [    ] o.a.s.c.TestStressInPlaceUpdates Realtime=false, ERROR, id=13 found={response={numFound=1,start=0,docs=[SolrDocument{id=13, title_s=[title13], val1_i_dvo=23, val2_l_dvo=23000000023, _version_=1534622450566823936, ratings=0.0, price=0}]}} model=[1534622450634981376, 23, 23000000046]
            
          • hoss.62D328FA1DEA57FD.fail3.txt - follows pattern:
            33869 INFO  (qtp1530691049-201) [n:127.0.0.1:33944_i%2Fop c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.p.AtomicUpdateDocumentMerger Uncommitted doc is: SolrInputDocument(fields: [id=1, val2_l_dvo=7000000035, _version_=1534622560585515008])
            ...
            34070 ERROR (READER1) [    ] o.a.s.c.TestStressInPlaceUpdates Realtime=true, ERROR, id=1 found={response={numFound=1,start=0,docs=[SolrDocument{id=1, title_s=[title1], val1_i_dvo=7, val2_l_dvo=7000000035, _version_=1534622560585515008, ratings=0.0, price=0}]}} model=[1534622560698761216, 7, 7000000042]
            
          • hoss.D768DD9443A98DC.fail.txt - follows pattern:
            29666 INFO  (qtp2123501190-71) [n:127.0.0.1:45821_norsg%2Fy c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.p.AtomicUpdateDocumentMerger Uncommitted doc is: SolrInputDocument(fields: [id=16, val2_l_dvo=7000000014, _version_=1534604230269075456, val1_i_dvo=7])
            ...
            31018 ERROR (READER9) [    ] o.a.s.c.TestStressInPlaceUpdates Realtime=true, ERROR, id=16 found={response={numFound=1,start=0,docs=[SolrDocument{id=16, title_s=[title16], val1_i_dvo=7, val2_l_dvo=7000000014, _version_=1534604230269075456, ratings=0.0, price=0}]}} model=[1534604230432653312, 7, 7000000021]
            
            • NOTE: this failure didn't reproduce for me reliably, see hoss.D768DD9443A98DC.pass.txt

          ... hoss.62D328FA1DEA57FD.fail2.txt is currently the odd duck out.

          my best guess (having not yet looked into AtomicUpdateDocumentMerger and what exactly it does and it's "Uncommitted doc is: ..." logging and what that refers to) is that some state tracked/updated by AtomicUpdateDocumentMerger is happening asyncronously and in the failure cases isn't "real" at the moments the test failures are happening – and that hoss.62D328FA1DEA57FD.fail2.txt may represent a situation where AtomicUpdateDocumentMerger is logging the version info in the "new" state before it's made it "real" and available for RTG.

          ...that's my theory anyway.

          Show
          hossman Hoss Man added a comment - I've been reviewing the logs from sarowe's failures – I won't pretend to understand half of what i'm looking at here (I'm still not up on most of the new code) but here's some interesting patterns i've noticed... in both failure logs posted, doc "13" was the doc having problems the specific docId is probably just a coincidence, but it does mean that the same egrep command works on both log files to give you the particularly interesting bits realtive to the failure... egrep add=\\[13\|id=13\|ids=13 TestStressInPlaceUpdates.eb044ac71.beast-167-failure.stdout.txt > beast-167.important.txt egrep add=\\[13\|id=13\|ids=13 TestStressInPlaceUpdates.eb044ac71.beast-587-failure.stdout.txt > beast-587.important.txt looking first at beast-587.important.txt: the ERROR that failed the test was first logged by READER2 @ (timestamp) 34456: 34456 ERROR (READER2) [ ] o.a.s.c.TestStressInPlaceUpdates Realtime=true, ERROR, id=13 found={response={numFound=1,start=0,docs=[SolrDocument{id=13, title_s=[title13], val1_i_dvo=3, val2_l_dvo=3000000006, _version_=1534607778351415296, ratings=0.0, price=0}]}} model=[1534607780231512064, 3, 3000000012] Working backwards, that expected version 1534607780231512064 was logged by WRITER10 as being returned to a PARTIAL update @ 31219: 31219 INFO (WRITER10) [ ] o.a.s.c.TestStressInPlaceUpdates PARTIAL: Writing id=13, val=[3,3000000012], version=1534607779993485312, Prev was=[3,3000000009]. Returned version=1534607780231512064 WRITER10's logging of this "Returned version=1534607780231512064" came after core_node1, core_node2, and core_node3 all logged it being written to their TLOG & reported it via LogUpdateProc: 30557 INFO (qtp2010985731-180) [n:127.0.0.1:37972__m c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607780231512064, prevVersion=1534607779993485312, prevPtr=2343) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-1-001/cores/collection1/data/tlog/tlog.0000000000000000004 refcount=1} LogPtr(2396) map=1977112331, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000012, _version_=1534607780231512064, val1_i_dvo=3]) 30589 INFO (qtp1755078679-232) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607780231512064, prevVersion=1534607779993485312, prevPtr=2343) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-2-001/cores/collection1/data/tlog/tlog.0000000000000000002 refcount=1} LogPtr(2396) map=1630836284, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000012, _version_=1534607780231512064, val1_i_dvo=3]) 30589 INFO (qtp1755078679-232) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.p.LogUpdateProcessorFactory [collection1] webapp=/_m path=/update params={update.distrib=FROMLEADER&distrib.from=http://127.0.0.1:37972/_m/collection1/&distrib.inplace.prevversion=1534607779993485312&wt=javabin&version=2&distrib.inplace.update=true}{add=[13 (1534607780231512064)]} 0 0 31216 INFO (qtp2143623462-144) [n:127.0.0.1:58295__m c:collection1 s:shard1 r:core_node3 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607780231512064, prevVersion=1534607779993485312, prevPtr=2343) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-3-001/cores/collection1/data/tlog/tlog.0000000000000000002 refcount=1} LogPtr(2396) map=1500522809, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000012, _version_=1534607780231512064, val1_i_dvo=3]) 31216 INFO (qtp2143623462-144) [n:127.0.0.1:58295__m c:collection1 s:shard1 r:core_node3 x:collection1] o.a.s.u.p.LogUpdateProcessorFactory [collection1] webapp=/_m path=/update params={update.distrib=FROMLEADER&distrib.from=http://127.0.0.1:37972/_m/collection1/&distrib.inplace.prevversion=1534607779993485312&wt=javabin&version=2&distrib.inplace.update=true}{add=[13 (1534607780231512064)]} 0 0 31219 INFO (qtp2010985731-180) [n:127.0.0.1:37972__m c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.p.LogUpdateProcessorFactory [collection1] webapp=/_m path=/update params={versions=true&wt=javabin&version=2}{add=[13 (1534607780231512064)]} 0 662 but looking after the ERROR was first logged @ 34456, we see that before the test had a chance to shut down all the nodes, there was some suspicious looking logging from core_node2 regarding updates out of order, that refer to the expected 1534607780231512064 version: 34976 INFO (qtp1755078679-111) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.p.DistributedUpdateProcessor Fetched the update: add{_version_=1534607780231512064,id=13} 35810 INFO (qtp1755078679-111) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.p.DistributedUpdateProcessor Fetched the update: add{_version_=1534607779993485312,id=13} 35811 INFO (qtp1755078679-111) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607779993485312, prevVersion=1534607778351415296, prevPtr=-1) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-2-001/cores/collection1/data/tlog/tlog.0000000000000000002 refcount=1} LogPtr(3955) map=748019145, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000009, _version_=1534607779993485312, val1_i_dvo=3]) 35811 INFO (qtp1755078679-111) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.p.DistributedUpdateProcessor Fetched missing dependent updates from leader, which likely succeeded. Current update: add{_version_=1534607780231512064,id=13}, Missing update: add{_version_=1534607779993485312,id=13} 35811 INFO (qtp1755078679-111) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607780231512064, prevVersion=1534607779993485312, prevPtr=3955) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-2-001/cores/collection1/data/tlog/tlog.0000000000000000002 refcount=1} LogPtr(4015) map=748019145, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000012, _version_=1534607780231512064, val1_i_dvo=3]) 35811 INFO (qtp1755078679-111) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.p.DistributedUpdateProcessor Fetched missing dependent updates from leader, which likely succeeded. Current update: add{,id=13}, Missing update: add{_version_=1534607780231512064,id=13} 35811 INFO (qtp1755078679-111) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607783219953664, prevVersion=1534607780231512064, prevPtr=4015) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-2-001/cores/collection1/data/tlog/tlog.0000000000000000002 refcount=1} LogPtr(4069) map=748019145, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000015, _version_=1534607783219953664, val1_i_dvo=3]) 35812 INFO (qtp1755078679-111) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.p.LogUpdateProcessorFactory [collection1] webapp=/_m path=/update params={update.distrib=FROMLEADER&distrib.from=http://127.0.0.1:37972/_m/collection1/&distrib.inplace.prevversion=1534607780231512064&wt=javabin&version=2&distrib.inplace.update=true}{add=[13 (1534607783219953664)]} 0 1714 these might be unrelated to the failure, since (IIRC) realtime get is always routed to the leader (can somone confirm this?) so this replica needing to fetch these out of order updates from the leader may not actaully be an indication of a problem – but it did catch my eye. if we change our focus and direct it at the "actual" version 1534607778351415296 that was returned by the realtime get causing the test failure, we notice it being recorded by WRITER1 as a partial update @ 28769: 28769 INFO (WRITER11) [ ] o.a.s.c.TestStressInPlaceUpdates PARTIAL: Writing id=13, val=[3,3000000006], version=1534607778339880960, Prev was=[3,3000000003]. Returned version=1534607778351415296 This is subsequent to the expected TLOG and LogUpdateProc messages we would expect for this update Starting at timestamp 30330, we see the expected log messages for the update that should have replaced 1534607778351415296, with params indicating that 1534607778351415296 is the previous version: 30330 INFO (qtp2010985731-180) [n:127.0.0.1:37972__m c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607779993485312, prevVersion=1534607778351415296, prevPtr=636) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-1-001/cores/collection1/data/tlog/tlog.0000000000000000004 refcount=1} LogPtr(2343) map=1977112331, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000009, _version_=1534607779993485312, val1_i_dvo=3]) 30331 INFO (qtp2143623462-144) [n:127.0.0.1:58295__m c:collection1 s:shard1 r:core_node3 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607779993485312, prevVersion=1534607778351415296, prevPtr=636) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-3-001/cores/collection1/data/tlog/tlog.0000000000000000002 refcount=1} LogPtr(2343) map=1500522809, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000009, _version_=1534607779993485312, val1_i_dvo=3]) 30332 INFO (qtp2143623462-144) [n:127.0.0.1:58295__m c:collection1 s:shard1 r:core_node3 x:collection1] o.a.s.u.p.LogUpdateProcessorFactory [collection1] webapp=/_m path=/update params={update.distrib=FROMLEADER&distrib.from=http://127.0.0.1:37972/_m/collection1/&distrib.inplace.prevversion=1534607778351415296&wt=javabin&version=2&distrib.inplace.update=true}{add=[13 (1534607779993485312)]} 0 0 30336 INFO (qtp1755078679-241) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607779993485312, prevVersion=1534607778351415296, prevPtr=636) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-2-001/cores/collection1/data/tlog/tlog.0000000000000000002 refcount=1} LogPtr(2343) map=1630836284, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000009, _version_=1534607779993485312, val1_i_dvo=3]) 30336 INFO (qtp1755078679-241) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.u.p.LogUpdateProcessorFactory [collection1] webapp=/_m path=/update params={update.distrib=FROMLEADER&distrib.from=http://127.0.0.1:37972/_m/collection1/&distrib.inplace.prevversion=1534607778351415296&wt=javabin&version=2&distrib.inplace.update=true}{add=[13 (1534607779993485312)]} 0 0 30396 INFO (qtp2143623462-227) [n:127.0.0.1:58295__m c:collection1 s:shard1 r:core_node3 x:collection1] o.a.s.c.S.Request [collection1] webapp=/_m path=/get params={qt=/get&ids=13&wt=javabin&version=2} status=0 QTime=0 30441 INFO (qtp1755078679-106) [n:127.0.0.1:38407__m c:collection1 s:shard1 r:core_node2 x:collection1] o.a.s.c.S.Request [collection1] webapp=/_m path=/get params={qt=/get&ids=13&wt=javabin&version=2} status=0 QTime=0 30469 INFO (qtp2010985731-180) [n:127.0.0.1:37972__m c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.p.LogUpdateProcessorFactory [collection1] webapp=/_m path=/update params={versions=true&wt=javabin&version=2}{add=[13 (1534607779993485312)]} 0 140 30556 INFO (WRITER10) [ ] o.a.s.c.TestStressInPlaceUpdates PARTIAL: Writing id=13, val=[3,3000000009], version=1534607778351415296, Prev was=[3,3000000006]. Returned version=1534607779993485312 BUT! Just before this logging about replacing 1534607778351415296 w/ 1534607779993485312 there is a log message mentioning version 1534607778351415296 that jumped out at me @ 30329: 30329 INFO (qtp2010985731-180) [n:127.0.0.1:37972__m c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.p.AtomicUpdateDocumentMerger Uncommitted doc is: SolrInputDocument(fields: [id=13, val2_l_dvo=3000000006, _version_=1534607778351415296, val1_i_dvo=3]) I don't know anything about this AtomicUpdateDocumentMerger class, but this logging smells particularly suspicious to me because it is the last time AtomicUpdateDocumentMerger logs anything about doc #13 is this an indication that the "state" AtomicUpdateDocumentMerger is tracking for doc #13 isn't getting updated when the (subsequently expected) version 1534607780231512064 is added? switching to beast-167.important.txt: the ERROR that failed the test was first logged by READER0 @ 32666: 32666 ERROR (READER0) [ ] o.a.s.c.TestStressInPlaceUpdates Realtime=true, ERROR, id=13 found={response={numFound=1,start=0,docs=[SolrDocument{id=13, title_s=[title13], val1_i_dvo=2, val2_l_dvo=2000000002, _version_=1534599975700267008, ratings=0.0, price=0}]}} model=[1534599977073901568, 2, 2000000004] this test didn't linger long enough to see if any interesting "out of order update" messages were logged by replicas after the failure. the expected version 1534599977073901568 is reported as the returned version by WRITER5 @ 32108: 32108 INFO (WRITER5) [ ] o.a.s.c.TestStressInPlaceUpdates PARTIAL: Writing id=13, val=[2,2000000004], version=1534599975700267008, Prev was=[2,2000000002]. Returned version=1534599977073901568 This is subsequent to the expected TLOG and LogUpdateProc messages we would expect for this update We again see no logging from AtomicUpdateDocumentMerger regarding the expected version 1534599977073901568 We DO see logging from AtomicUpdateDocumentMerger regarding the actual version return in the request that failed the assertion: 32082 INFO (qtp498157202-244) [n:127.0.0.1:55136__nge%2Fr c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.p.AtomicUpdateDocumentMerger Uncommitted doc is: SolrInputDocument(fields: [id=13, val2_l_dvo=2000000002, _version_=1534599975700267008]) This is again the last (and only) log message from AtomicUpdateDocumentMerger regarding doc #13 In my own hammering of TestStressInPlaceUpdates, i've encountered a handful of failures that haven't reproduced 100% reliably, but most of the failures i've seen have followed this pattern {red}The most recent version logged by AtomicUpdateDocumentMerger matches the 'actual' version logged in an ERROR assertion failure message{red} hoss.62D328FA1DEA57FD.fail.txt - follows pattern: 23263 INFO (qtp905061545-62) [n:127.0.0.1:43930_i%2Fop c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.p.AtomicUpdateDocumentMerger Uncommitted doc is: SolrInputDocument(fields: [id=4, val2_l_dvo=3000000009, _version_=1534612078380187648, val1_i_dvo=3]) ... 23543 ERROR (READER3) [ ] o.a.s.c.TestStressInPlaceUpdates Realtime=true, ERROR, id=4 found={response={numFound=1,start=0,docs=[SolrDocument{id=4, title_s=[title4], val1_i_dvo=3, val2_l_dvo=3000000009, _version_=1534612078380187648, ratings=0.0, price=0}]}} model=[1534612078708391936, 3, 3000000012] hoss.62D328FA1DEA57FD.fail2.txt - does NOT follow the pattern: 32481 INFO (qtp1749706169-67) [n:127.0.0.1:48694_i%2Fop c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.p.AtomicUpdateDocumentMerger Uncommitted doc is: SolrInputDocument(fields: [id=13, val2_l_dvo=23000000023, _version_=1534622450566823936, val1_i_dvo=23]) ... 32581 INFO (qtp1749706169-238) [n:127.0.0.1:48694_i%2Fop c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.p.AtomicUpdateDocumentMerger Uncommitted doc is: SolrInputDocument(fields: [id=13, val2_l_dvo=23000000046, _version_=1534622450634981376, val1_i_dvo=23]) ... 32661 ERROR (READER7) [ ] o.a.s.c.TestStressInPlaceUpdates Realtime=false, ERROR, id=13 found={response={numFound=1,start=0,docs=[SolrDocument{id=13, title_s=[title13], val1_i_dvo=23, val2_l_dvo=23000000023, _version_=1534622450566823936, ratings=0.0, price=0}]}} model=[1534622450634981376, 23, 23000000046] hoss.62D328FA1DEA57FD.fail3.txt - follows pattern: 33869 INFO (qtp1530691049-201) [n:127.0.0.1:33944_i%2Fop c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.p.AtomicUpdateDocumentMerger Uncommitted doc is: SolrInputDocument(fields: [id=1, val2_l_dvo=7000000035, _version_=1534622560585515008]) ... 34070 ERROR (READER1) [ ] o.a.s.c.TestStressInPlaceUpdates Realtime=true, ERROR, id=1 found={response={numFound=1,start=0,docs=[SolrDocument{id=1, title_s=[title1], val1_i_dvo=7, val2_l_dvo=7000000035, _version_=1534622560585515008, ratings=0.0, price=0}]}} model=[1534622560698761216, 7, 7000000042] hoss.D768DD9443A98DC.fail.txt - follows pattern: 29666 INFO (qtp2123501190-71) [n:127.0.0.1:45821_norsg%2Fy c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.p.AtomicUpdateDocumentMerger Uncommitted doc is: SolrInputDocument(fields: [id=16, val2_l_dvo=7000000014, _version_=1534604230269075456, val1_i_dvo=7]) ... 31018 ERROR (READER9) [ ] o.a.s.c.TestStressInPlaceUpdates Realtime=true, ERROR, id=16 found={response={numFound=1,start=0,docs=[SolrDocument{id=16, title_s=[title16], val1_i_dvo=7, val2_l_dvo=7000000014, _version_=1534604230269075456, ratings=0.0, price=0}]}} model=[1534604230432653312, 7, 7000000021] NOTE: this failure didn't reproduce for me reliably, see hoss.D768DD9443A98DC.pass.txt ... hoss.62D328FA1DEA57FD.fail2.txt is currently the odd duck out. my best guess (having not yet looked into AtomicUpdateDocumentMerger and what exactly it does and it's "Uncommitted doc is: ..." logging and what that refers to) is that some state tracked/updated by AtomicUpdateDocumentMerger is happening asyncronously and in the failure cases isn't "real" at the moments the test failures are happening – and that hoss.62D328FA1DEA57FD.fail2.txt may represent a situation where AtomicUpdateDocumentMerger is logging the version info in the "new" state before it's made it "real" and available for RTG. ...that's my theory anyway.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Thanks for your analysis, Hoss. I'll take a deeper look as soon as possible. A pattern I have observed with such failures (and these failures are the ones I was referring to in the past) that documents get in trouble immediately after or during a commit (i.e. between the commit start and end) happening in a parallel thread.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Thanks for your analysis, Hoss. I'll take a deeper look as soon as possible. A pattern I have observed with such failures (and these failures are the ones I was referring to in the past) that documents get in trouble immediately after or during a commit (i.e. between the commit start and end) happening in a parallel thread.
          Hide
          steve_rowe Steve Rowe added a comment -

          Beasting found 6 more failures, for a total of 9.

          I'm attaching a tar.gz file containing all 9 runs' logs (including those that are separately attached).

          Here are the repro lines (without the linedocs sysprop) for the 6 failing runs not previously mentioned on this issue:

          run 927:
          
          ant test  -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=D936BF4963826AEC -Dtests.slow=true -Dtests.locale=ja -Dtests.timezone=GMT -Dtests.asserts=true -Dtests.file.encoding=ISO-8859-1
          
          run 1246:
          
          ant test  -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=9717476E3589BED0 -Dtests.slow=true -Dtests.locale=pl -Dtests.timezone=Europe/Samara -Dtests.asserts=true -Dtests.file.encoding=UTF-8
          
          run 1543:
          
          ant test  -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=9D53690B34363B1C -Dtests.slow=true -Dtests.locale=es-NI -Dtests.timezone=America/Argentina/San_Juan -Dtests.asserts=true -Dtests.file.encoding=ISO-8859-1
          
          run 1558:
          
          ant test  -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=655C990D41E11FF7 -Dtests.slow=true -Dtests.locale=en-NZ -Dtests.timezone=MST -Dtests.asserts=true -Dtests.file.encoding=US-ASCII
          
          run 1604:
          
          ant test  -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=DD4A15BAF279FCB9 -Dtests.slow=true -Dtests.locale=de-AT -Dtests.timezone=Pacific/Pitcairn -Dtests.asserts=true -Dtests.file.encoding=UTF-8
          
          run 1854:
          
          ant test  -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=9D68461E2404622 -Dtests.slow=true -Dtests.locale=pl -Dtests.timezone=Brazil/West -Dtests.asserts=true -Dtests.file.encoding=US-ASCII
          

          5 of the above 6 runs reproduced for me on the first try, but I could not reproduce run 1246 (seed 9717476E3589BED0) - it succeeded in 4/4 attempts.

          Show
          steve_rowe Steve Rowe added a comment - Beasting found 6 more failures, for a total of 9. I'm attaching a tar.gz file containing all 9 runs' logs (including those that are separately attached). Here are the repro lines (without the linedocs sysprop) for the 6 failing runs not previously mentioned on this issue: run 927: ant test -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=D936BF4963826AEC -Dtests.slow=true -Dtests.locale=ja -Dtests.timezone=GMT -Dtests.asserts=true -Dtests.file.encoding=ISO-8859-1 run 1246: ant test -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=9717476E3589BED0 -Dtests.slow=true -Dtests.locale=pl -Dtests.timezone=Europe/Samara -Dtests.asserts=true -Dtests.file.encoding=UTF-8 run 1543: ant test -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=9D53690B34363B1C -Dtests.slow=true -Dtests.locale=es-NI -Dtests.timezone=America/Argentina/San_Juan -Dtests.asserts=true -Dtests.file.encoding=ISO-8859-1 run 1558: ant test -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=655C990D41E11FF7 -Dtests.slow=true -Dtests.locale=en-NZ -Dtests.timezone=MST -Dtests.asserts=true -Dtests.file.encoding=US-ASCII run 1604: ant test -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=DD4A15BAF279FCB9 -Dtests.slow=true -Dtests.locale=de-AT -Dtests.timezone=Pacific/Pitcairn -Dtests.asserts=true -Dtests.file.encoding=UTF-8 run 1854: ant test -Dtestcase=TestStressInPlaceUpdates -Dtests.method=stressTest -Dtests.seed=9D68461E2404622 -Dtests.slow=true -Dtests.locale=pl -Dtests.timezone=Brazil/West -Dtests.asserts=true -Dtests.file.encoding=US-ASCII 5 of the above 6 runs reproduced for me on the first try, but I could not reproduce run 1246 (seed 9717476E3589BED0) - it succeeded in 4/4 attempts.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          In the 587 file,
          between 30557 (when the expected update was written):

             [junit4]   2> 30557 INFO  (qtp2010985731-180) [n:127.0.0.1:37972__m c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607780231512064, prevVersion=1534607779993485312, prevPtr=2343) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-1-001/cores/collection1/data/tlog/tlog.0000000000000000004 refcount=1} LogPtr(2396) map=1977112331, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000012, _version_=1534607780231512064, val1_i_dvo=3])
          

          and the ERROR line at 34456,

             [junit4]   2> 34456 ERROR (READER2) [    ] o.a.s.c.TestStressInPlaceUpdates Realtime=true, ERROR, id=13 found={response={numFound=1,start=0,docs=[SolrDocument{id=13, title_s=[title13], val1_i_dvo=3, val2_l_dvo=3000000006, _version_=1534607778351415296, ratings=0.0, price=0}]}} model=[1534607780231512064, 3, 3000000012]
          

          ,
          there are two suspicious events:

          1. There was some reordering of updates (34422).
          2. There was a commit (start: 33306, 33363, 33398; end: 33871, 34209).

          I think either of those, or both, could be the reason for the inconsistency.

          I don't know anything about this AtomicUpdateDocumentMerger class, but this logging smells particularly suspicious to me because it is the last time AtomicUpdateDocumentMerger logs anything about doc #13

          The reason this is the last time AUDM logged anything about the doc #13 could be because soon after that line, the commit happened. And doc #13 was no longer found in the tlog, and hence there was no "uncommitted doc" with id=13.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - In the 587 file, between 30557 (when the expected update was written): [junit4] 2> 30557 INFO (qtp2010985731-180) [n:127.0.0.1:37972__m c:collection1 s:shard1 r:core_node1 x:collection1] o.a.s.u.UpdateLog TLOG: added id 13(ver=1534607780231512064, prevVersion=1534607779993485312, prevPtr=2343) to tlog{file=/tmp/beast-tmp-output/587/J0/temp/solr.cloud.TestStressInPlaceUpdates_FFC46C473EC471E6-001/shard-1-001/cores/collection1/data/tlog/tlog.0000000000000000004 refcount=1} LogPtr(2396) map=1977112331, actual doc=SolrInputDocument(fields: [id=13, val2_l_dvo=3000000012, _version_=1534607780231512064, val1_i_dvo=3]) and the ERROR line at 34456, [junit4] 2> 34456 ERROR (READER2) [ ] o.a.s.c.TestStressInPlaceUpdates Realtime= true , ERROR, id=13 found={response={numFound=1,start=0,docs=[SolrDocument{id=13, title_s=[title13], val1_i_dvo=3, val2_l_dvo=3000000006, _version_=1534607778351415296, ratings=0.0, price=0}]}} model=[1534607780231512064, 3, 3000000012] , there are two suspicious events: There was some reordering of updates (34422). There was a commit (start: 33306, 33363, 33398; end: 33871, 34209). I think either of those, or both, could be the reason for the inconsistency. I don't know anything about this AtomicUpdateDocumentMerger class, but this logging smells particularly suspicious to me because it is the last time AtomicUpdateDocumentMerger logs anything about doc #13 The reason this is the last time AUDM logged anything about the doc #13 could be because soon after that line, the commit happened. And doc #13 was no longer found in the tlog, and hence there was no "uncommitted doc" with id=13.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited

          Hoss Man pointed me to SOLR-8733, due to which returned versions are 0 unless the versioning is done locally (i.e. the update is sent to the correct shard leader). I found out that the stress test was affected by this issue. I have committed a fix to the stress test so as to workaround this problem by only sending updates to the leader of the shard. After having done that, I cannot reproduce the failures that Steve Rowe reported.

          Here's the corresponding commit: https://github.com/chatman/lucene-solr/commit/d61c4fc520111f721468edb236930180bd91d7eb

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited Hoss Man pointed me to SOLR-8733 , due to which returned versions are 0 unless the versioning is done locally (i.e. the update is sent to the correct shard leader). I found out that the stress test was affected by this issue. I have committed a fix to the stress test so as to workaround this problem by only sending updates to the leader of the shard. After having done that, I cannot reproduce the failures that Steve Rowe reported. Here's the corresponding commit: https://github.com/chatman/lucene-solr/commit/d61c4fc520111f721468edb236930180bd91d7eb
          Hide
          steve_rowe Steve Rowe added a comment -

          I beasted 2000 iterations of `TestStressInPlaceUpdates` against https://github.com/chatman/lucene-solr/commit/d61c4fc520111f721468edb236930180bd91d7eb and there were zero failures.

          Show
          steve_rowe Steve Rowe added a comment - I beasted 2000 iterations of `TestStressInPlaceUpdates` against https://github.com/chatman/lucene-solr/commit/d61c4fc520111f721468edb236930180bd91d7eb and there were zero failures.
          Hide
          noble.paul Noble Paul added a comment -

          Awesome

          Show
          noble.paul Noble Paul added a comment - Awesome
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited

          I ran the test all weekend and found the following failures which reproduce reliably:

          8CF844B8D2C14DFA
          AE673569D5853984
          

          It seems that the common pattern in these failures is that they fail when one of the docValues fields is Lucene54 and one of the fields is Memory. Here's an example (from the latter seed):

             [junit4]   2> NOTE: test params are: codec=Asserting(Lucene62): {title_s=PostingsFormat(name=MockRandom), id=Lucene50(blocksize=128)}, docValues:{val1_i_dvo=DocValuesFormat(name=Direct), _version_=DocValuesFormat(name=Direct), val2_l_dvo=DocValuesFormat(name=Memory), ratings=DocValuesFormat(name=Memory), price=DocValuesFormat(name=Lucene54)}, maxPointsInLeafNode=132, maxMBSortInHeap=7.882796951749762, sim=RandomSimilarity(queryNorm=true,coord=crazy): {}, locale=fr-CA, timezone=Pacific/Pago_Pago
          

          I found that the same pattern was present in all previously failing tests for which I had logs.

          As a logical next step, I suppressed both "Lucene54" and "Memory" codecs in the test and ran them. The failing tests passed, and so did lots and lots of other seeds. However, one of the tests failed after suppressing Lucene54 and Memory: seed F9C1398E563942D5 (this seed didn't fail before). Surprisingly, for this failing seed, I don't see the "NOTE" line mentioning per field codecs, but just the following info:

          NOTE: test params are: codec=FastCompressingStoredFields(storedFieldsFormat=CompressingStoredFieldsFormat(compressionMode=FAST, chunkSize=24740, maxDocsPerChunk=7, blockSize=236), termVectorsFormat=CompressingTermVectorsFormat(compressionMode=FAST, chunkSize=24740, blockSize=236)), sim=RandomSimilarity(queryNorm=true,coord=no): {}, locale=sv, timezone=America/Catamarca
          

          I'm looking into how to make this particular test fail in a more simple, reproducible test that does not require lots of threads etc.

          Hoss Man FYI.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited I ran the test all weekend and found the following failures which reproduce reliably: 8CF844B8D2C14DFA AE673569D5853984 It seems that the common pattern in these failures is that they fail when one of the docValues fields is Lucene54 and one of the fields is Memory. Here's an example (from the latter seed): [junit4] 2> NOTE: test params are: codec=Asserting(Lucene62): {title_s=PostingsFormat(name=MockRandom), id=Lucene50(blocksize=128)}, docValues:{val1_i_dvo=DocValuesFormat(name=Direct), _version_=DocValuesFormat(name=Direct), val2_l_dvo=DocValuesFormat(name=Memory), ratings=DocValuesFormat(name=Memory), price=DocValuesFormat(name=Lucene54)}, maxPointsInLeafNode=132, maxMBSortInHeap=7.882796951749762, sim=RandomSimilarity(queryNorm= true ,coord=crazy): {}, locale=fr-CA, timezone=Pacific/Pago_Pago I found that the same pattern was present in all previously failing tests for which I had logs. As a logical next step, I suppressed both "Lucene54" and "Memory" codecs in the test and ran them. The failing tests passed, and so did lots and lots of other seeds. However, one of the tests failed after suppressing Lucene54 and Memory: seed F9C1398E563942D5 (this seed didn't fail before). Surprisingly, for this failing seed, I don't see the "NOTE" line mentioning per field codecs, but just the following info: NOTE: test params are: codec=FastCompressingStoredFields(storedFieldsFormat=CompressingStoredFieldsFormat(compressionMode=FAST, chunkSize=24740, maxDocsPerChunk=7, blockSize=236), termVectorsFormat=CompressingTermVectorsFormat(compressionMode=FAST, chunkSize=24740, blockSize=236)), sim=RandomSimilarity(queryNorm= true ,coord=no): {}, locale=sv, timezone=America/Catamarca I'm looking into how to make this particular test fail in a more simple, reproducible test that does not require lots of threads etc. Hoss Man FYI.
          Hide
          hossman Hoss Man added a comment -

          I'm still very slowly trying to get up to speed on this ... i started out by reviewing the tests ishan wrote specifically for this issue, but once i realized there really weren't any pre-existing, non trivial, "distributed atomic updates" I put that on the backburner to work on SOLR-9159 – now that that test is solid and running on master & 6x, it's helped uncover an NPE in the new code when the latest patch is applied...

             [junit4]   2> 20685 ERROR (qtp1751829478-197) [n:127.0.0.1:58791_solr c:test_col s:shard1 r:core_node4 x:test_col_shard1_replica2] o.a.s.h.RequestHandlerBase org.apache.solr.common.SolrException: Exception writing document id 34 to the index; possible analysis error.
             [junit4]   2> 	at org.apache.solr.update.DirectUpdateHandler2.addDoc(DirectUpdateHandler2.java:188)
             [junit4]   2> 	at org.apache.solr.update.processor.RunUpdateProcessor.processAdd(RunUpdateProcessorFactory.java:68)
             [junit4]   2> 	at org.apache.solr.update.processor.UpdateRequestProcessor.processAdd(UpdateRequestProcessor.java:48)
             [junit4]   2> 	at org.apache.solr.update.processor.DistributedUpdateProcessor.doLocalAdd(DistributedUpdateProcessor.java:954)
             [junit4]   2> 	at org.apache.solr.update.processor.DistributedUpdateProcessor.versionAdd(DistributedUpdateProcessor.java:1145)
             [junit4]   2> 	at org.apache.solr.update.processor.DistributedUpdateProcessor.processAdd(DistributedUpdateProcessor.java:729)
             [junit4]   2> 	at org.apache.solr.update.processor.LogUpdateProcessorFactory$LogUpdateProcessor.processAdd(LogUpdateProcessorFactory.java:103)
             [junit4]   2> 	at org.apache.solr.handler.loader.JavabinLoader$1.update(JavabinLoader.java:98)
             [junit4]   2> 	at org.apache.solr.client.solrj.request.JavaBinUpdateRequestCodec$1.readOuterMostDocIterator(JavaBinUpdateRequestCodec.java:179)
             [junit4]   2> 	at org.apache.solr.client.solrj.request.JavaBinUpdateRequestCodec$1.readIterator(JavaBinUpdateRequestCodec.java:135)
             [junit4]   2> 	at org.apache.solr.common.util.JavaBinCodec.readVal(JavaBinCodec.java:274)
             [junit4]   2> 	at org.apache.solr.client.solrj.request.JavaBinUpdateRequestCodec$1.readNamedList(JavaBinUpdateRequestCodec.java:121)
             [junit4]   2> 	at org.apache.solr.common.util.JavaBinCodec.readVal(JavaBinCodec.java:239)
             [junit4]   2> 	at org.apache.solr.common.util.JavaBinCodec.unmarshal(JavaBinCodec.java:157)
             [junit4]   2> 	at org.apache.solr.client.solrj.request.JavaBinUpdateRequestCodec.unmarshal(JavaBinUpdateRequestCodec.java:186)
             [junit4]   2> 	at org.apache.solr.handler.loader.JavabinLoader.parseAndLoadDocs(JavabinLoader.java:108)
             [junit4]   2> 	at org.apache.solr.handler.loader.JavabinLoader.load(JavabinLoader.java:55)
             [junit4]   2> 	at org.apache.solr.handler.UpdateRequestHandler$1.load(UpdateRequestHandler.java:97)
             [junit4]   2> 	at org.apache.solr.handler.ContentStreamHandlerBase.handleRequestBody(ContentStreamHandlerBase.java:69)
             [junit4]   2> 	at org.apache.solr.handler.RequestHandlerBase.handleRequest(RequestHandlerBase.java:155)
             [junit4]   2> 	at org.apache.solr.core.SolrCore.execute(SolrCore.java:2036)
             [junit4]   2> 	at org.apache.solr.servlet.HttpSolrCall.execute(HttpSolrCall.java:658)
             [junit4]   2> 	at org.apache.solr.servlet.HttpSolrCall.call(HttpSolrCall.java:465)
             [junit4]   2> 	at org.apache.solr.servlet.SolrDispatchFilter.doFilter(SolrDispatchFilter.java:257)
             [junit4]   2> 	at org.apache.solr.servlet.SolrDispatchFilter.doFilter(SolrDispatchFilter.java:208)
             [junit4]   2> 	at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1676)
             [junit4]   2> 	at org.apache.solr.client.solrj.embedded.JettySolrRunner$DebugFilter.doFilter(JettySolrRunner.java:138)
             [junit4]   2> 	at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1676)
             [junit4]   2> 	at org.eclipse.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:581)
             [junit4]   2> 	at org.eclipse.jetty.server.session.SessionHandler.doHandle(SessionHandler.java:224)
             [junit4]   2> 	at org.eclipse.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1160)
             [junit4]   2> 	at org.eclipse.jetty.servlet.ServletHandler.doScope(ServletHandler.java:511)
             [junit4]   2> 	at org.eclipse.jetty.server.session.SessionHandler.doScope(SessionHandler.java:185)
             [junit4]   2> 	at org.eclipse.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1092)
             [junit4]   2> 	at org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:141)
             [junit4]   2> 	at org.eclipse.jetty.server.handler.gzip.GzipHandler.handle(GzipHandler.java:399)
             [junit4]   2> 	at org.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:134)
             [junit4]   2> 	at org.eclipse.jetty.server.Server.handle(Server.java:518)
             [junit4]   2> 	at org.eclipse.jetty.server.HttpChannel.handle(HttpChannel.java:308)
             [junit4]   2> 	at org.eclipse.jetty.server.HttpConnection.onFillable(HttpConnection.java:244)
             [junit4]   2> 	at org.eclipse.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:273)
             [junit4]   2> 	at org.eclipse.jetty.io.FillInterest.fillable(FillInterest.java:95)
             [junit4]   2> 	at org.eclipse.jetty.io.SelectChannelEndPoint$2.run(SelectChannelEndPoint.java:93)
             [junit4]   2> 	at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.produceAndRun(ExecuteProduceConsume.java:246)
             [junit4]   2> 	at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.run(ExecuteProduceConsume.java:156)
             [junit4]   2> 	at org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:654)
             [junit4]   2> 	at org.eclipse.jetty.util.thread.QueuedThreadPool$3.run(QueuedThreadPool.java:572)
             [junit4]   2> 	at java.lang.Thread.run(Thread.java:745)
             [junit4]   2> Caused by: java.lang.NullPointerException
             [junit4]   2> 	at org.apache.solr.update.DirectUpdateHandler2.doNormalUpdate(DirectUpdateHandler2.java:297)
             [junit4]   2> 	at org.apache.solr.update.DirectUpdateHandler2.addDoc0(DirectUpdateHandler2.java:221)
             [junit4]   2> 	at org.apache.solr.update.DirectUpdateHandler2.addDoc(DirectUpdateHandler2.java:176)
             [junit4]   2> 	... 47 more
             ...
             [junit4]   2> NOTE: reproduce with: ant test  -Dtestcase=TestStressCloudBlindAtomicUpdates -Dtests.method=test_dv -Dtests.seed=64C390303CA3F13A -Dtests.slow=true -Dtests.locale=el -Dtests.timezone=Indian/Cocos -Dtests.asserts=true -Dtests.file.encoding=UTF-8
          

          ...that sort of failure has happened every time i've tried running that new test with the current patch, so i don't think the seed matters.


          I also realized i had some notes from lsat week when i first started reviewing the tests that i aparently never posted here...

          Misc comments as i try to get up to wrap my head around some of these tests/code...

          • the ASL header should always be above the package & import statements
          • we should do everything we can to avoid adding any more kitch sink schema files – if an existing schema file isn't good enough, include only what's necessary to satisfy the test preconditions
            • tests should also verify their schema preconditions to ensure no one edits the schema to break assumptions (like docValues=true & stored=false)
              • schema API can be used to check this in cloud tests, or you can grab the IndexSchema directly from the SolrCore in single core tests
          • at first glance, SolrDocument.applyOlderUpdate looks like it's totally broken for SolrInputDocument objects that contain multiple values for a single field in the SolrInputDocument – is that usecase tested? where?
            • unit tests of this method in solrj independent of the usage in UpdateLog seems really important
          • these empty catch blogs are really bad, and overlook the possibility of serious bugs (either in the current code, or possibly being introduced in the future) by assuming any failures encountered here are the type of failure the code expects, w/o actually verifying it...
            try {
              addAndGetVersion(sdoc("id","20", "_version_", 123456, "ratings", map("inc", 1)), null);
              fail();
            } catch (Exception ex) {}
            

            ...it should be something like...

            SolrException exected = expectThrows(SolrException.class, () -> { addAndGetVersion(...)});
            assertEquals(/* something about expected */, expected.getSomething());
            
          • given that the whole point of this functionality is that the docValues are suppose to be updated "in place" i'm really suprised at how little is done to try and actaully assert that..
            • InPlaceUpdateDistribTest checks the internal lucene "docid", but i don't see similar checks in the other tests.
            • it seems like every one of these tests should really be looking at the segments in the index/replicas (via the Segments API for cloud, or more easily by grabbing the IndexReader direct from the SolrCore in the single node test) to really be absolutely sure that we didn't get a new segment unless we expected one. That would not only help prevent false positives in the doc values updating code, but could also help future proof if some settings/defaults change in the future that cause auto commits or something else to happen that this test doesn't expect.
              • or am i missunderstanding something about how the docvalue updating code works? ... IIUC we can tell the diff between new segments and existing segments whose docvalues have been updated, is my understanding incorrect?
            • any assertions that are not specific to checking docvalues are updated "in place" could be refactored so that we can test them for any field, not just pure DV fields.
              • for example: the exsiting bulkd of the test logic could be refactored into a helper method that takes in a field name – that helper method could then be called from different test methods that pass in a hardcoded field name. the test method that passes in a field name which is DV only, could have additional asserts that the segment files on disk do not change after the test passes.
          • it's hard to put a lot of faith in TestInPlaceUpdate and InPlaceUpdateDistribTest considering they use so few docs that they aren't likely to involve more then one index segment per shard (even in nightly or with test.multiplier) ... using things like "atLeast()" to ensure we have a non trivial number of docs in the index, and then indexing a random number of them before the docs you are are actually testing against (and the rest after) would help
            • likewise for TestStressInPlaceUpdates ... use atLeast for the ndocs (and maybe the num threads) so it's lower on basic runs, but higher for nightly runs or runs where tests.multiplier is set
          • is there a new test showing combinations/sequences of pure-dv updates followed by atomic updates of multiple fields, followed by pure-dv updates? to sanity check that (older style) atomic updates in the middle read the correct "updated" docvalues?

          ...the comment about refactoring the parts of the test that aren't specific to "in place" DV updates is also applicable to the new TestStressCloudBlindAtomicUpdates i added – in that class "test_dv" could be updated to assert that the segments on disk didn't change after the test is finished.

          Show
          hossman Hoss Man added a comment - I'm still very slowly trying to get up to speed on this ... i started out by reviewing the tests ishan wrote specifically for this issue, but once i realized there really weren't any pre-existing, non trivial, "distributed atomic updates" I put that on the backburner to work on SOLR-9159 – now that that test is solid and running on master & 6x, it's helped uncover an NPE in the new code when the latest patch is applied... [junit4] 2> 20685 ERROR (qtp1751829478-197) [n:127.0.0.1:58791_solr c:test_col s:shard1 r:core_node4 x:test_col_shard1_replica2] o.a.s.h.RequestHandlerBase org.apache.solr.common.SolrException: Exception writing document id 34 to the index; possible analysis error. [junit4] 2> at org.apache.solr.update.DirectUpdateHandler2.addDoc(DirectUpdateHandler2.java:188) [junit4] 2> at org.apache.solr.update.processor.RunUpdateProcessor.processAdd(RunUpdateProcessorFactory.java:68) [junit4] 2> at org.apache.solr.update.processor.UpdateRequestProcessor.processAdd(UpdateRequestProcessor.java:48) [junit4] 2> at org.apache.solr.update.processor.DistributedUpdateProcessor.doLocalAdd(DistributedUpdateProcessor.java:954) [junit4] 2> at org.apache.solr.update.processor.DistributedUpdateProcessor.versionAdd(DistributedUpdateProcessor.java:1145) [junit4] 2> at org.apache.solr.update.processor.DistributedUpdateProcessor.processAdd(DistributedUpdateProcessor.java:729) [junit4] 2> at org.apache.solr.update.processor.LogUpdateProcessorFactory$LogUpdateProcessor.processAdd(LogUpdateProcessorFactory.java:103) [junit4] 2> at org.apache.solr.handler.loader.JavabinLoader$1.update(JavabinLoader.java:98) [junit4] 2> at org.apache.solr.client.solrj.request.JavaBinUpdateRequestCodec$1.readOuterMostDocIterator(JavaBinUpdateRequestCodec.java:179) [junit4] 2> at org.apache.solr.client.solrj.request.JavaBinUpdateRequestCodec$1.readIterator(JavaBinUpdateRequestCodec.java:135) [junit4] 2> at org.apache.solr.common.util.JavaBinCodec.readVal(JavaBinCodec.java:274) [junit4] 2> at org.apache.solr.client.solrj.request.JavaBinUpdateRequestCodec$1.readNamedList(JavaBinUpdateRequestCodec.java:121) [junit4] 2> at org.apache.solr.common.util.JavaBinCodec.readVal(JavaBinCodec.java:239) [junit4] 2> at org.apache.solr.common.util.JavaBinCodec.unmarshal(JavaBinCodec.java:157) [junit4] 2> at org.apache.solr.client.solrj.request.JavaBinUpdateRequestCodec.unmarshal(JavaBinUpdateRequestCodec.java:186) [junit4] 2> at org.apache.solr.handler.loader.JavabinLoader.parseAndLoadDocs(JavabinLoader.java:108) [junit4] 2> at org.apache.solr.handler.loader.JavabinLoader.load(JavabinLoader.java:55) [junit4] 2> at org.apache.solr.handler.UpdateRequestHandler$1.load(UpdateRequestHandler.java:97) [junit4] 2> at org.apache.solr.handler.ContentStreamHandlerBase.handleRequestBody(ContentStreamHandlerBase.java:69) [junit4] 2> at org.apache.solr.handler.RequestHandlerBase.handleRequest(RequestHandlerBase.java:155) [junit4] 2> at org.apache.solr.core.SolrCore.execute(SolrCore.java:2036) [junit4] 2> at org.apache.solr.servlet.HttpSolrCall.execute(HttpSolrCall.java:658) [junit4] 2> at org.apache.solr.servlet.HttpSolrCall.call(HttpSolrCall.java:465) [junit4] 2> at org.apache.solr.servlet.SolrDispatchFilter.doFilter(SolrDispatchFilter.java:257) [junit4] 2> at org.apache.solr.servlet.SolrDispatchFilter.doFilter(SolrDispatchFilter.java:208) [junit4] 2> at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1676) [junit4] 2> at org.apache.solr.client.solrj.embedded.JettySolrRunner$DebugFilter.doFilter(JettySolrRunner.java:138) [junit4] 2> at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1676) [junit4] 2> at org.eclipse.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:581) [junit4] 2> at org.eclipse.jetty.server.session.SessionHandler.doHandle(SessionHandler.java:224) [junit4] 2> at org.eclipse.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1160) [junit4] 2> at org.eclipse.jetty.servlet.ServletHandler.doScope(ServletHandler.java:511) [junit4] 2> at org.eclipse.jetty.server.session.SessionHandler.doScope(SessionHandler.java:185) [junit4] 2> at org.eclipse.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1092) [junit4] 2> at org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:141) [junit4] 2> at org.eclipse.jetty.server.handler.gzip.GzipHandler.handle(GzipHandler.java:399) [junit4] 2> at org.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:134) [junit4] 2> at org.eclipse.jetty.server.Server.handle(Server.java:518) [junit4] 2> at org.eclipse.jetty.server.HttpChannel.handle(HttpChannel.java:308) [junit4] 2> at org.eclipse.jetty.server.HttpConnection.onFillable(HttpConnection.java:244) [junit4] 2> at org.eclipse.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:273) [junit4] 2> at org.eclipse.jetty.io.FillInterest.fillable(FillInterest.java:95) [junit4] 2> at org.eclipse.jetty.io.SelectChannelEndPoint$2.run(SelectChannelEndPoint.java:93) [junit4] 2> at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.produceAndRun(ExecuteProduceConsume.java:246) [junit4] 2> at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.run(ExecuteProduceConsume.java:156) [junit4] 2> at org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:654) [junit4] 2> at org.eclipse.jetty.util.thread.QueuedThreadPool$3.run(QueuedThreadPool.java:572) [junit4] 2> at java.lang.Thread.run(Thread.java:745) [junit4] 2> Caused by: java.lang.NullPointerException [junit4] 2> at org.apache.solr.update.DirectUpdateHandler2.doNormalUpdate(DirectUpdateHandler2.java:297) [junit4] 2> at org.apache.solr.update.DirectUpdateHandler2.addDoc0(DirectUpdateHandler2.java:221) [junit4] 2> at org.apache.solr.update.DirectUpdateHandler2.addDoc(DirectUpdateHandler2.java:176) [junit4] 2> ... 47 more ... [junit4] 2> NOTE: reproduce with: ant test -Dtestcase=TestStressCloudBlindAtomicUpdates -Dtests.method=test_dv -Dtests.seed=64C390303CA3F13A -Dtests.slow=true -Dtests.locale=el -Dtests.timezone=Indian/Cocos -Dtests.asserts=true -Dtests.file.encoding=UTF-8 ...that sort of failure has happened every time i've tried running that new test with the current patch, so i don't think the seed matters. I also realized i had some notes from lsat week when i first started reviewing the tests that i aparently never posted here... Misc comments as i try to get up to wrap my head around some of these tests/code... the ASL header should always be above the package & import statements we should do everything we can to avoid adding any more kitch sink schema files – if an existing schema file isn't good enough, include only what's necessary to satisfy the test preconditions tests should also verify their schema preconditions to ensure no one edits the schema to break assumptions (like docValues=true & stored=false) schema API can be used to check this in cloud tests, or you can grab the IndexSchema directly from the SolrCore in single core tests at first glance, SolrDocument.applyOlderUpdate looks like it's totally broken for SolrInputDocument objects that contain multiple values for a single field in the SolrInputDocument – is that usecase tested? where? unit tests of this method in solrj independent of the usage in UpdateLog seems really important these empty catch blogs are really bad, and overlook the possibility of serious bugs (either in the current code, or possibly being introduced in the future) by assuming any failures encountered here are the type of failure the code expects, w/o actually verifying it... try { addAndGetVersion(sdoc( "id" , "20" , "_version_" , 123456, "ratings" , map( "inc" , 1)), null ); fail(); } catch (Exception ex) {} ...it should be something like... SolrException exected = expectThrows(SolrException.class, () -> { addAndGetVersion(...)}); assertEquals(/* something about expected */, expected.getSomething()); given that the whole point of this functionality is that the docValues are suppose to be updated "in place" i'm really suprised at how little is done to try and actaully assert that.. InPlaceUpdateDistribTest checks the internal lucene "docid", but i don't see similar checks in the other tests. it seems like every one of these tests should really be looking at the segments in the index/replicas (via the Segments API for cloud, or more easily by grabbing the IndexReader direct from the SolrCore in the single node test) to really be absolutely sure that we didn't get a new segment unless we expected one. That would not only help prevent false positives in the doc values updating code, but could also help future proof if some settings/defaults change in the future that cause auto commits or something else to happen that this test doesn't expect. or am i missunderstanding something about how the docvalue updating code works? ... IIUC we can tell the diff between new segments and existing segments whose docvalues have been updated, is my understanding incorrect? any assertions that are not specific to checking docvalues are updated "in place" could be refactored so that we can test them for any field, not just pure DV fields. for example: the exsiting bulkd of the test logic could be refactored into a helper method that takes in a field name – that helper method could then be called from different test methods that pass in a hardcoded field name. the test method that passes in a field name which is DV only, could have additional asserts that the segment files on disk do not change after the test passes. it's hard to put a lot of faith in TestInPlaceUpdate and InPlaceUpdateDistribTest considering they use so few docs that they aren't likely to involve more then one index segment per shard (even in nightly or with test.multiplier) ... using things like "atLeast()" to ensure we have a non trivial number of docs in the index, and then indexing a random number of them before the docs you are are actually testing against (and the rest after) would help likewise for TestStressInPlaceUpdates ... use atLeast for the ndocs (and maybe the num threads) so it's lower on basic runs, but higher for nightly runs or runs where tests.multiplier is set is there a new test showing combinations/sequences of pure-dv updates followed by atomic updates of multiple fields, followed by pure-dv updates? to sanity check that (older style) atomic updates in the middle read the correct "updated" docvalues? ...the comment about refactoring the parts of the test that aren't specific to "in place" DV updates is also applicable to the new TestStressCloudBlindAtomicUpdates i added – in that class "test_dv" could be updated to assert that the segments on disk didn't change after the test is finished.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited

          Attaching fresh patch from the https://github.com/chatman/lucene-solr/tree/solr_5944 branch.

          I've added a test based on SolrTestCaseJ4, which fails, despite suppressing the Lucene54 and Memory codecs. I am now trying to reproduce this issue in a LuceneTestCase test.

          https://github.com/chatman/lucene-solr/commit/b80be98d9e1a98b97bee670a2d775f6acc2182c7

          Reproduce:

          -ea -Dtests.seed=1D9D6101E3D231FF -Dtests.verbose=true -Dtestcase=TestInPlaceUpdate
          

          Couple things I noticed that: (a) this happens when Compressing codec is used, along with (b) the "crazy" value of max buffered docs is used as per this part from the randomization in LuceneTestCase.java

              if (r.nextBoolean()) {
                if (rarely(r)) {
                  // crazy value
                  c.setMaxBufferedDocs(TestUtil.nextInt(r, 2, 15));
                } else {
                  // reasonable value
                  c.setMaxBufferedDocs(TestUtil.nextInt(r, 16, 1000));
                }
              }
          

          However, I can't reproduce the failure based on just those two factors; there must be something else that I'm unable to pinpoint.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited Attaching fresh patch from the https://github.com/chatman/lucene-solr/tree/solr_5944 branch. I've added a test based on SolrTestCaseJ4, which fails, despite suppressing the Lucene54 and Memory codecs. I am now trying to reproduce this issue in a LuceneTestCase test. https://github.com/chatman/lucene-solr/commit/b80be98d9e1a98b97bee670a2d775f6acc2182c7 Reproduce: -ea -Dtests.seed=1D9D6101E3D231FF -Dtests.verbose= true -Dtestcase=TestInPlaceUpdate Couple things I noticed that: (a) this happens when Compressing codec is used, along with (b) the "crazy" value of max buffered docs is used as per this part from the randomization in LuceneTestCase.java if (r.nextBoolean()) { if (rarely(r)) { // crazy value c.setMaxBufferedDocs(TestUtil.nextInt(r, 2, 15)); } else { // reasonable value c.setMaxBufferedDocs(TestUtil.nextInt(r, 16, 1000)); } } However, I can't reproduce the failure based on just those two factors; there must be something else that I'm unable to pinpoint.
          Hide
          hossman Hoss Man added a comment -

          minor tweaks to ishan's last patch...

          • updated to master (some compliation failures due to other jiras getting commited)
          • refactored TestInPlaceUpdate
            • testReplay3 & testReplay4 reproduce the previously mentioned failure that required using the external file.
            • you can still refer to external files i nteh same format via "testReplayFromFile" but only if a sys prop is set...
              ant test -Dtestcase=TestInPlaceUpdate -Dtests.seed=1D9D6101E3D231FF -Dtests.verbose=true -Dtests.method=testReplayFromFile -Dargs="-Dtests.inplace.file=/home/hossman/lucene/dev/operations3.txt"
              
          Show
          hossman Hoss Man added a comment - minor tweaks to ishan's last patch... updated to master (some compliation failures due to other jiras getting commited) refactored TestInPlaceUpdate testReplay3 & testReplay4 reproduce the previously mentioned failure that required using the external file. you can still refer to external files i nteh same format via "testReplayFromFile" but only if a sys prop is set... ant test -Dtestcase=TestInPlaceUpdate -Dtests.seed=1D9D6101E3D231FF -Dtests.verbose=true -Dtests.method=testReplayFromFile -Dargs="-Dtests.inplace.file=/home/hossman/lucene/dev/operations3.txt"
          Hide
          hossman Hoss Man added a comment -

          Ugh, testReplay4 in my last patch was a waste of time – somehoe i screwed something up with copy/paste.

          The point was to try and check if using "set" operations had any different effect on the test compared to using "inc" operations (like in testReplay3)

          This patch fixes that, so that both tests demonstrate the problem with the seed ishan mentioned (and aren't identical) ... but more interesting then that is the new testReplay5, testReplay6 & testReplay7 (still using hte same seed)...

          • testReplay5 - still uses "inc" for doc id=0, but uses "set" for every other doc in the index
            • this currently fails with an NPE in AtomicUpdateDocumentMerger.doInPlaceUpdateMerge(AtomicUpdateDocumentMerger.java:283)
          • testReplay6 was a quick hack to work around the NPE in testReplay5 - ensuring that the first time a doc is added, a regular "add" is done, but everytime after that "set" is used (except for id=0 where "inc" is still used)
            • this test currently passes – even though it should be effectively the same as testReplay3 and testReplay4
          • testReplay7 is the same as testReplay6 but (like testReplya4) does all the id=0 updates using "set" as well just to sanity check no diff in behavior
            • this also passes

          ...so based on the fact that testReplay6/7 pass, but testReplay3/4 fail, any ishan's observations about the setMaxBufferedDocs my suspicion is that the underlying problem probably has something to do with how frequently docs are flushed and trying to read back "updated" DV values when other docs were added with regular DV values i nte hsame field just before/after the DV update.

          (looking at TestNumericDocValuesUpdates i don't see many tests that mix calls to addDocument with calls to updateNumericDocValue – except testSegmentMerges & testManyReopensAndFields which update the DV of every document added up to that point)

          Show
          hossman Hoss Man added a comment - Ugh, testReplay4 in my last patch was a waste of time – somehoe i screwed something up with copy/paste. The point was to try and check if using "set" operations had any different effect on the test compared to using "inc" operations (like in testReplay3) This patch fixes that, so that both tests demonstrate the problem with the seed ishan mentioned (and aren't identical) ... but more interesting then that is the new testReplay5, testReplay6 & testReplay7 (still using hte same seed)... testReplay5 - still uses "inc" for doc id=0, but uses "set" for every other doc in the index this currently fails with an NPE in AtomicUpdateDocumentMerger.doInPlaceUpdateMerge(AtomicUpdateDocumentMerger.java:283) testReplay6 was a quick hack to work around the NPE in testReplay5 - ensuring that the first time a doc is added, a regular "add" is done, but everytime after that "set" is used (except for id=0 where "inc" is still used) this test currently passes – even though it should be effectively the same as testReplay3 and testReplay4 testReplay7 is the same as testReplay6 but (like testReplya4) does all the id=0 updates using "set" as well just to sanity check no diff in behavior this also passes ...so based on the fact that testReplay6/7 pass, but testReplay3/4 fail, any ishan's observations about the setMaxBufferedDocs my suspicion is that the underlying problem probably has something to do with how frequently docs are flushed and trying to read back "updated" DV values when other docs were added with regular DV values i nte hsame field just before/after the DV update. (looking at TestNumericDocValuesUpdates i don't see many tests that mix calls to addDocument with calls to updateNumericDocValue – except testSegmentMerges & testManyReopensAndFields which update the DV of every document added up to that point)
          Hide
          hossman Hoss Man added a comment -

          (I'm pretty sure) I was able to reproduce the root cause of the randomized failures in LUCENE-7301.

          Show
          hossman Hoss Man added a comment - (I'm pretty sure) I was able to reproduce the root cause of the randomized failures in LUCENE-7301 .
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          (I'm pretty sure) I was able to reproduce the root cause of the randomized failures in LUCENE-7301.

          Thanks Hoss for beating me to it!

          testReplay5 - still uses "inc" for doc id=0, but uses "set" for every other doc in the index

          this currently fails with an NPE in AtomicUpdateDocumentMerger.doInPlaceUpdateMerge(AtomicUpdateDocumentMerger.java:283)

          I think the problem there is that a "set" operation was attempted at a document that still doesn't exist in the index. I think such an operation works with atomic updates, but the underlying docValues API doesn't support updates of dv fields that don't exist yet. I will try to handle this better, instead of throwing NPE.

          I shall work on fixing your review comments regarding the tests, and increase their scope as you suggest. My idea behind the tests were (and naming could be improved): TestInPlaceUpdate just tests some basic cases in non-cloud mode, TestStressInPlaceUpdates tests lots of documents, lots of updates, lots of threads and cloud mode, InPlaceUpdateDistribTest for some basic operations/scenarios in cloud (including testing if same document was updated, or a new one was created). I was thinking that if we can get past the DV updates flushing issue (LUCENE-7301), we can focus well on improving scope of tests more. Thanks for your review!

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - (I'm pretty sure) I was able to reproduce the root cause of the randomized failures in LUCENE-7301 . Thanks Hoss for beating me to it! testReplay5 - still uses "inc" for doc id=0, but uses "set" for every other doc in the index this currently fails with an NPE in AtomicUpdateDocumentMerger.doInPlaceUpdateMerge(AtomicUpdateDocumentMerger.java:283) I think the problem there is that a "set" operation was attempted at a document that still doesn't exist in the index. I think such an operation works with atomic updates, but the underlying docValues API doesn't support updates of dv fields that don't exist yet. I will try to handle this better, instead of throwing NPE. I shall work on fixing your review comments regarding the tests, and increase their scope as you suggest. My idea behind the tests were (and naming could be improved): TestInPlaceUpdate just tests some basic cases in non-cloud mode, TestStressInPlaceUpdates tests lots of documents, lots of updates, lots of threads and cloud mode, InPlaceUpdateDistribTest for some basic operations/scenarios in cloud (including testing if same document was updated, or a new one was created). I was thinking that if we can get past the DV updates flushing issue ( LUCENE-7301 ), we can focus well on improving scope of tests more. Thanks for your review!
          Hide
          hossman Hoss Man added a comment -

          Ok ... more in depth comments reviewing the latest patch (ignoring some of the general higher level stuff i've previously commented on).

          (So far i've still focused on reviewing the tests, because we should make sure they're rock solid before any discussion of refacoting/improving/changing the code)


          • in general, all these tests seem to depend on autoCommit being disabled, and use a config that is setup that way, but don't actaully assert that it's true in case someone changes the configs in the future
            • TestInPlaceUpdate can get direct access to the SolrCore verify that for certain to
            • the distrib tests might be able to use one of hte new cnfig APIs to check this (i don't know off the top of my head)
              • at a minimum define a String constant for the config file name in TestInPlaceUpdate and refer to it in the other tests where the same config is expected with a comment explaining that we're assuming it has autoCommit disabled and that TestInPlaceUpdate will fail if it does not.
          • TestInPlaceUpdate
            • SuppressCodecs should be removed
            • should at least have class level javadocs explaining what's being tested
            • testUpdatingDocValues
              • 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...
                • currently version1 is compared to newVersion1 to assert that an update incrememnted the version, but in between those 2 updates are 4 other places where that document was updated – we have to assert it has the expected value (either the same as before, or new - and if new record it) after all of those addAndGetVersion calls, or we can't be sure where/why/how a bug exists if that existing comparison fails.
                • ideally we should be asserting the version of every doc when we query it right along side the assertion for it's updated "ratings" value
              • most of the use of "field(ratings)" can probbaly just be replaced with "ratings" now that DV are returnable – although it's nice to have both included in the test at least once to demo that both work, but when doing that there should be a comment making it clear why
            • testOnlyPartialUpdatesBetweenCommits
              • ditto comment about checking return value from addAndGetVersion
              • this also seems like a good place to to test if doing a redundent atomic update (either via set to the same value or via inc=0) returns a new version or not – should it?
            • DocInfo should be a private static class and have some javadocs
            • based on how testing has gone so far, and the discover of LUCENE-7301 it seems clear that adding even single thread, single node, randomized testing of lots of diff types of add/update calls would be good to add
              • we could refactor/improve the "checkReplay" function I added in the last patch to do more testing of a randomly generated Iterable of "commands" (commit, doc, doc+atomic mutation, etc...)
              • and of course: improve checkReplay to verify RTG against hte uncommited model as well
              • testReplayFromFile and getSDdoc should probably go away once we have more structured tests for doing this
            • createMap can be elimianted – callers can just use SolrTestCaseJ4.map(...)
            • In general the tests in this class should include more queries / sorting against the updated docvalues field after commits to ensure that the updated value is searchable & sortable
            • Likewise the test methods in this class should probably have a lot more RTG checks – with filter queries that constrain against the updated docvalues field, and checks of the expected version field – to ensure that is all working properly.
          • InPlaceUpdateDistribTest
            • SuppressCodecs should be removed
            • should at least have class level javadocs explaining what's being tested
            • 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
            • there should almost certianly be some "delete all docs and optimize" cleanup in between all of these tests
              • easy to do in an @Before method if we refactor to subclass SolrCloudTestCase
            • docValuesUpdateTest
              • should randomize numdocs
              • we need to find away to eliminate the hardcoded "Thread.sleep(500);" calls...
                • 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
            • ensureRtgWorksWithPartialUpdatesTest
              • even if we're only going to test one doc, we should ensure there are a random num docs in the index (some before the doc we're editing, and some after)
              • if we're testing RTG, then we should be testing the version returned from every /get call against the last version returned from every update
            • outOfOrderUpdatesIndividualReplicaTest
              • ditto comments about only one doc
              • ditto comments about testing the expected version in RTG requests
              • if we're sending updates direct to replicas to test how they handle out of order updates, then something better assert exactly where the leader is hosted and ensure we don't send to it by mistake
              • what's the point of using a threadpool and SendUpdateToReplicaTask here? why not just send the updates in a (randdomly assigned) determinisitc order?
                • 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 "delete all docs and optimize" followed up "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)
            • delayedReorderingFetchesMissingUpdateFromLeaderTest
              • ditto previous comments about using a threadpool and SendUpdateToReplicaTask
                • even more so considering the "Thread.sleep(100);" ... what's the pont of using a threadpool if we want the requests to be sequential?
              • 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
            • SendUpdateToReplicaTask
              • based on how it's used, i'm not really sure i see the point in this class, but assuming it continues to exist...
              • constructor takes in a Random, but method uses that global random() anyway.
                • should probably take in a seed, and construct it's own Random
                • random() ensures that each Thread gets it's own consistent Random instance – but in Callables like this each Thread having a consistent seed doesn't help the reproducibility since there's no garutee which Threed from an ThreadPool (Executor) will invoke call().
              • instead of returning true, this should be a Callable<UpdateResponse> and call() should return the results of the request so the caller can assert it was successful (via Future.get().getStatus())
            • getReplicaValue
              • using SolrTestCaseJ4.params(...) would make this method a lot shorter
              • based on where/how this method is used, i don't understand why it returns String instead of just Object
            • assertReplicaValue
              • should take in some sort of assertion message and/or build/append an assertion message using the clientId
              • if getReplicaValue returns an Object, this can take an "Object expected" param and eliminate abonch of toString & string concating throughout the test
            • simulatedUpdateRequest
              • if this method is going to assume that the only shard you ever want to similulate an update to is SHARD1 then the method name should be "simulatedUpdateRequestToShard1Replica"
              • better still - why not ask the DocRouter which shard this doc belongs in, and fetch the leader URL tha way?
            • most usage of "addFields" can just be replaced with a call to "sdoc(...)" to simplify code
            • replace createMap usage with SolrTestCaseJ4.map
            • why override tearDown if we're just calling super?
            • in general, i think this test would be a lot easier to read if there were well named variables for HttpSolrClient instances pointed at specific replicas (ie HttpSolrClient SHARD1_LEADER = ...; HttpSolrClient SHARD1_REPLICA1 = ...; etc...) and passed those around to the various methods instead of magic ints (ie: "1", "2") to refer to which index in the static clients list should be used for a given update.
          • TestStressInPlaceUpdates
            • ditto comments from InPlaceUpdateDistribTest about regarding @SupressCodecs, 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
            • stressTest
              • 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)
              • maxConcurrentCommits WTF?
                • has a comment that it should be less the # warming threads, but does that even make sense i na distrib test?
                • currently set to nWriteThreads – so what's the point, when/how could we ever possibly have more concurrent commits then the number of threads? doesn't that just mean that at the moment every write thread is allowed to commit if it wants to?
                • if there is a reason for it, then replaceing "numCommitting" with a new Semaphore(maxConcurrentCommits) would probably make more sense
              • why is the "hardCommit start" logic calling both commit(); and clients.get(chosenClientIndex).commit(); ?
              • 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
              • there's a bunch of "TODO" blocks realted to deletes that still need implemented
              • 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
              • why is //fail("Were were results: "+response); commented out?
              • there's a lot of "instanceof ArrayList" checks that make no sense to me since the object came from getFirstValue
            • DocInfo
              • should be a private static class and have some javadocs
              • or sould be a public class in it's own file, w/javadocs, and re-used in the various tests that want ot reuse it
            • 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
              • why are we synchronizing on cloudClient but updating with leaderClient?
                • if the point is just to ensure all udpates happem synchronously regardless of client, then we should just define some public static final Object UPDATE_SYNC = new Object("sync lock for updates"); and use that
            • getClientForLeader
              • i know this method is currently just a workaround for SOLR-8733, noting that in the method javadocs seems important
              • if we refactor this test to use multiple shards before SOLR-8733 gets resolved, this method can take in a uniqueKey, and consult the DocRouter to pick the correct shard/node.
            • replace createMap usage with SolrTestCaseJ4.map
            • why override tearDown if we're just calling super?
          • SolrDocument
            • applyOlderUpdate
              • as mentioned before, i don't think this method is correct when it comes to multivalued fields, and should have more unit tests of various permutations to be sure
              • this functionality should probably be moved to a private helper method in UpdateLog (it doesn't do anything that requires it have access to the internals of the SolrDocument)
              • no matter where it lives, it should have some javadocs
          Show
          hossman Hoss Man added a comment - Ok ... more in depth comments reviewing the latest patch (ignoring some of the general higher level stuff i've previously commented on). (So far i've still focused on reviewing the tests, because we should make sure they're rock solid before any discussion of refacoting/improving/changing the code) in general, all these tests seem to depend on autoCommit being disabled, and use a config that is setup that way, but don't actaully assert that it's true in case someone changes the configs in the future TestInPlaceUpdate can get direct access to the SolrCore verify that for certain to the distrib tests might be able to use one of hte new cnfig APIs to check this (i don't know off the top of my head) at a minimum define a String constant for the config file name in TestInPlaceUpdate and refer to it in the other tests where the same config is expected with a comment explaining that we're assuming it has autoCommit disabled and that TestInPlaceUpdate will fail if it does not. TestInPlaceUpdate SuppressCodecs should be removed should at least have class level javadocs explaining what's being tested testUpdatingDocValues 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... currently version1 is compared to newVersion1 to assert that an update incrememnted the version, but in between those 2 updates are 4 other places where that document was updated – we have to assert it has the expected value (either the same as before, or new - and if new record it) after all of those addAndGetVersion calls, or we can't be sure where/why/how a bug exists if that existing comparison fails. ideally we should be asserting the version of every doc when we query it right along side the assertion for it's updated "ratings" value most of the use of "field(ratings)" can probbaly just be replaced with "ratings" now that DV are returnable – although it's nice to have both included in the test at least once to demo that both work, but when doing that there should be a comment making it clear why testOnlyPartialUpdatesBetweenCommits ditto comment about checking return value from addAndGetVersion this also seems like a good place to to test if doing a redundent atomic update (either via set to the same value or via inc=0) returns a new version or not – should it? DocInfo should be a private static class and have some javadocs based on how testing has gone so far, and the discover of LUCENE-7301 it seems clear that adding even single thread, single node, randomized testing of lots of diff types of add/update calls would be good to add we could refactor/improve the "checkReplay" function I added in the last patch to do more testing of a randomly generated Iterable of "commands" (commit, doc, doc+atomic mutation, etc...) and of course: improve checkReplay to verify RTG against hte uncommited model as well testReplayFromFile and getSDdoc should probably go away once we have more structured tests for doing this createMap can be elimianted – callers can just use SolrTestCaseJ4.map(...) In general the tests in this class should include more queries / sorting against the updated docvalues field after commits to ensure that the updated value is searchable & sortable Likewise the test methods in this class should probably have a lot more RTG checks – with filter queries that constrain against the updated docvalues field, and checks of the expected version field – to ensure that is all working properly. InPlaceUpdateDistribTest SuppressCodecs should be removed should at least have class level javadocs explaining what's being tested 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 there should almost certianly be some "delete all docs and optimize" cleanup in between all of these tests easy to do in an @Before method if we refactor to subclass SolrCloudTestCase docValuesUpdateTest should randomize numdocs we need to find away to eliminate the hardcoded "Thread.sleep(500);" calls... 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 ensureRtgWorksWithPartialUpdatesTest even if we're only going to test one doc, we should ensure there are a random num docs in the index (some before the doc we're editing, and some after) if we're testing RTG, then we should be testing the version returned from every /get call against the last version returned from every update outOfOrderUpdatesIndividualReplicaTest ditto comments about only one doc ditto comments about testing the expected version in RTG requests if we're sending updates direct to replicas to test how they handle out of order updates, then something better assert exactly where the leader is hosted and ensure we don't send to it by mistake what's the point of using a threadpool and SendUpdateToReplicaTask here? why not just send the updates in a (randdomly assigned) determinisitc order? 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 "delete all docs and optimize" followed up "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) delayedReorderingFetchesMissingUpdateFromLeaderTest ditto previous comments about using a threadpool and SendUpdateToReplicaTask even more so considering the "Thread.sleep(100);" ... what's the pont of using a threadpool if we want the requests to be sequential? 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 SendUpdateToReplicaTask based on how it's used, i'm not really sure i see the point in this class, but assuming it continues to exist... constructor takes in a Random, but method uses that global random() anyway. should probably take in a seed, and construct it's own Random random() ensures that each Thread gets it's own consistent Random instance – but in Callables like this each Thread having a consistent seed doesn't help the reproducibility since there's no garutee which Threed from an ThreadPool (Executor) will invoke call(). instead of returning true, this should be a Callable<UpdateResponse> and call() should return the results of the request so the caller can assert it was successful (via Future.get().getStatus()) getReplicaValue using SolrTestCaseJ4.params(...) would make this method a lot shorter based on where/how this method is used, i don't understand why it returns String instead of just Object assertReplicaValue should take in some sort of assertion message and/or build/append an assertion message using the clientId if getReplicaValue returns an Object, this can take an "Object expected" param and eliminate abonch of toString & string concating throughout the test simulatedUpdateRequest if this method is going to assume that the only shard you ever want to similulate an update to is SHARD1 then the method name should be "simulatedUpdateRequestToShard1Replica" better still - why not ask the DocRouter which shard this doc belongs in, and fetch the leader URL tha way? most usage of "addFields" can just be replaced with a call to "sdoc(...)" to simplify code replace createMap usage with SolrTestCaseJ4.map why override tearDown if we're just calling super? in general, i think this test would be a lot easier to read if there were well named variables for HttpSolrClient instances pointed at specific replicas (ie HttpSolrClient SHARD1_LEADER = ...; HttpSolrClient SHARD1_REPLICA1 = ...; etc...) and passed those around to the various methods instead of magic ints (ie: "1", "2") to refer to which index in the static clients list should be used for a given update. TestStressInPlaceUpdates ditto comments from InPlaceUpdateDistribTest about regarding @SupressCodecs, 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 stressTest 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) maxConcurrentCommits WTF? has a comment that it should be less the # warming threads, but does that even make sense i na distrib test? currently set to nWriteThreads – so what's the point, when/how could we ever possibly have more concurrent commits then the number of threads? doesn't that just mean that at the moment every write thread is allowed to commit if it wants to? if there is a reason for it, then replaceing "numCommitting" with a new Semaphore(maxConcurrentCommits) would probably make more sense why is the "hardCommit start" logic calling both commit(); and clients.get(chosenClientIndex).commit(); ? 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 there's a bunch of "TODO" blocks realted to deletes that still need implemented 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 why is //fail("Were were results: "+response); commented out? there's a lot of "instanceof ArrayList" checks that make no sense to me since the object came from getFirstValue DocInfo should be a private static class and have some javadocs or sould be a public class in it's own file, w/javadocs, and re-used in the various tests that want ot reuse it 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 why are we synchronizing on cloudClient but updating with leaderClient? if the point is just to ensure all udpates happem synchronously regardless of client, then we should just define some public static final Object UPDATE_SYNC = new Object("sync lock for updates"); and use that getClientForLeader i know this method is currently just a workaround for SOLR-8733 , noting that in the method javadocs seems important if we refactor this test to use multiple shards before SOLR-8733 gets resolved, this method can take in a uniqueKey, and consult the DocRouter to pick the correct shard/node. replace createMap usage with SolrTestCaseJ4.map why override tearDown if we're just calling super? SolrDocument applyOlderUpdate as mentioned before, i don't think this method is correct when it comes to multivalued fields, and should have more unit tests of various permutations to be sure this functionality should probably be moved to a private helper method in UpdateLog (it doesn't do anything that requires it have access to the internals of the SolrDocument) no matter where it lives, it should have some javadocs
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Updated the patch. Fixed some issues from Hoss' comments. Some nocommits are remaining. I'll reply to Hoss' suggestions in-line shortly.
          Couple of error handling fixes, but mostly changes to test suites.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Updated the patch. Fixed some issues from Hoss' comments. Some nocommits are remaining. I'll reply to Hoss' suggestions in-line shortly. Couple of error handling fixes, but mostly changes to test suites.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Fixed two bugs:
          1. Added a check while writing a document to exclude anything to do with the id field.
          2. Added an exception when a "set" or "inc" operation is attempted at a non-existent document.

          Review comments:

          * in general, all these tests seem to depend on autoCommit being disabled, and use a config that is setup that way, but don't actaully assert that it's true in case someone changes the configs in the future

          TODO

          * TestInPlaceUpdate

          Renamed this test to TestInPlaceUpdatesStandalone.

          ** SuppressCodecs should be removed

          Removed all non 3x, 4x codec suppression. They need to be suppressed as per a comment from Mikhail. https://issues.apache.org/jira/browse/LUCENE-5189?focusedCommentId=13958205&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13958205

          ** should at least have class level javadocs explaining what's being tested

          TODO

            • testUpdatingDocValues

              *** for addAndGetVersion calls where we don't care about the returned version, don't bother assigning it to a variable (distracting)

              Fixed

              *** for addAndGetVersion calls where we do care about the returned version, we need check it for every update to that doc...

              TODO

              **** currently version1 is compared to newVersion1 to assert that an update incrememnted the version, but in between those 2 updates are 4 other places where that document was updated – we have to assert it has the expected value (either the same as before, or new - and if new record it) after all of those addAndGetVersion calls, or we can't be sure where/why/how a bug exists if that existing comparison fails.

              TODO

              **** ideally we should be asserting the version of every doc when we query it right along side the assertion for it's updated "ratings" value

              TODO

              *** most of the use of "field(ratings)" can probbaly just be replaced with "ratings" now that DV are returnable – although it's nice to have both included in the test at least once to demo that both work, but when doing that there should be a comment making it clear why

              Fixed

            • testOnlyPartialUpdatesBetweenCommits
              • ditto comment about checking return value from addAndGetVersion

                *** this also seems like a good place to to test if doing a redundent atomic update (either via set to the same value or via inc=0) returns a new version or not – should it?

                As of now, both will generate a new version. I think "inc" 0 should be dropped, and "set" same value should be versioned. I'll check if behaviour in this patch is at par with regular atomic updates; and if so, will open a separate issue for this later.

                ** DocInfo should be a private static class and have some javadocs

                Fixed

                ** based on how testing has gone so far, and the discover of LUCENE-7301 it seems clear that adding even single thread, single node, randomized testing of lots of diff types of add/update calls would be good to add

                I think we can do the same in TestStressInPlaceUpdates, by randomly setting number of writer threads to 1 sometimes.

                *** we could refactor/improve the "checkReplay" function I added in the last patch to do more testing of a randomly generated Iterable of "commands" (commit, doc, doc+atomic mutation, etc...)

                TODO

                *** and of course: improve checkReplay to verify RTG against hte uncommited model as well

                I couldn't find a way to do this for the TestInPlaceUpdate (now called TestInPlaceUpdatesStandalone in this patch). This is based on SolrTestCaseJ4.

          *** testReplayFromFile and getSDdoc should probably go away once we have more structured tests for doing this

          Fixed

          ** createMap can be elimianted – callers can just use SolrTestCaseJ4.map(...)

          Fixed

          ** In general the tests in this class should include more queries / sorting against the updated docvalues field after commits to ensure that the updated value is searchable & sortable

          TODO

          ** Likewise the test methods in this class should probably have a lot more RTG checks – with filter queries that constrain against the updated docvalues field, and checks of the expected version field – to ensure that is all working properly.

          Couldn't figure out how to do RTGs with this test, but will check RTGs + filter queries in the TestInPlaceUpdatesDistrib test (which was formerly InPlaceUpdateDistribTest)

          * InPlaceUpdateDistribTest

          Renamed to TestInPlaceUpdatesDistrib now

          ** SuppressCodecs should be removed

          3x and 4x codec suppressions cannot be removed.

          ** should at least have class level javadocs explaining what's being tested

          TODO

          ** 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

          TODO

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

          TODO

          ** there should almost certianly be some "delete all docs and optimize" cleanup in between all of these tests

          Added this to the beginning of every test. I couldn't see an easy way to refactor to SolrCloudTestCase immediately, but I can look more later.

            • docValuesUpdateTest

              *** should randomize numdocs

              Fixed

              *** we need to find away to eliminate the hardcoded "Thread.sleep(500);" calls...

              Fixed (I think it is the correct fix, but I'll review it again, testing with a fast machine and beasting it).

              **** 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.

              TODO

              **** 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

              TODO

            • ensureRtgWorksWithPartialUpdatesTest

              *** even if we're only going to test one doc, we should ensure there are a random num docs in the index (some before the doc we're editing, and some after)

              TODO

              *** if we're testing RTG, then we should be testing the version returned from every /get call against the last version returned from every update

              TODO

            • outOfOrderUpdatesIndividualReplicaTest

              *** ditto comments about only one doc

              TODO

              *** ditto comments about testing the expected version in RTG requests

              TODO

              *** if we're sending updates direct to replicas to test how they handle out of order updates, then something better assert exactly where the leader is hosted and ensure we don't send to it by mistake

              TODO

              *** what's the point of using a threadpool and SendUpdateToReplicaTask here? why not just send the updates in a (randdomly assigned) determinisitc order?

              TODO

              **** if we are going to use an ExecutorService, then the result of awaitTermination has to be checked

              TODO

              • since this tests puts replicas out of sync, a "delete all docs and optimize" followed up "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)
                TODO
            • delayedReorderingFetchesMissingUpdateFromLeaderTest

              *** ditto previous comments about using a threadpool and SendUpdateToReplicaTask

              TODO

              *** 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.

              TODO.

            • getReplicaValue

              *** using SolrTestCaseJ4.params(...) would make this method a lot shorter

              Fixed

              *** based on where/how this method is used, i don't understand why it returns String instead of just Object

              Fixed

              ** assertReplicaValue

              *** should take in some sort of assertion message and/or build/append an assertion message using the clientId

              Fixed

              *** if getReplicaValue returns an Object, this can take an "Object expected" param and eliminate abonch of toString & string concating throughout the test

              Fixed

            • simulatedUpdateRequest

              *** if this method is going to assume that the only shard you ever want to similulate an update to is SHARD1 then the method name should be "simulatedUpdateRequestToShard1Replica"

              TODO

              *** better still - why not ask the DocRouter which shard this doc belongs in, and fetch the leader URL tha way?

              TODO

              ** most usage of "addFields" can just be replaced with a call to "sdoc(...)" to simplify code

              Fixed

              ** replace createMap usage with SolrTestCaseJ4.map

              Fixed

              ** why override tearDown if we're just calling super?

              Fixed

              ** in general, i think this test would be a lot easier to read if there were well named variables for HttpSolrClient instances pointed at specific replicas (ie HttpSolrClient SHARD1_LEADER = ...; HttpSolrClient SHARD1_REPLICA1 = ...; etc...) and passed those around to the various methods instead of magic ints (ie: "1", "2") to refer to which index in the static clients list should be used for a given update.

              Fixed: LEADER and NON_LEADERS now

          • TestStressInPlaceUpdates
            TODO
          • SolrDocument

            ** applyOlderUpdate

            Fixed, moved it to UpdateLog. I think Noble had moved this from UpdateLog to SolrDocument; and if so, I'll check with him once (after I'm done with other changes here marked as TODO).

            *** as mentioned before, i don't think this method is correct when it comes to multivalued fields, and should have more unit tests of various permutations to be sure

            TODO

            *** no matter where it lives, it should have some javadocs

            Fixed

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Fixed two bugs: 1. Added a check while writing a document to exclude anything to do with the id field. 2. Added an exception when a "set" or "inc" operation is attempted at a non-existent document. Review comments: * in general, all these tests seem to depend on autoCommit being disabled, and use a config that is setup that way, but don't actaully assert that it's true in case someone changes the configs in the future TODO * TestInPlaceUpdate Renamed this test to TestInPlaceUpdatesStandalone. ** SuppressCodecs should be removed Removed all non 3x, 4x codec suppression. They need to be suppressed as per a comment from Mikhail. https://issues.apache.org/jira/browse/LUCENE-5189?focusedCommentId=13958205&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13958205 ** should at least have class level javadocs explaining what's being tested TODO testUpdatingDocValues *** for addAndGetVersion calls where we don't care about the returned version, don't bother assigning it to a variable (distracting) Fixed *** for addAndGetVersion calls where we do care about the returned version, we need check it for every update to that doc... TODO **** currently version1 is compared to newVersion1 to assert that an update incrememnted the version, but in between those 2 updates are 4 other places where that document was updated – we have to assert it has the expected value (either the same as before, or new - and if new record it) after all of those addAndGetVersion calls, or we can't be sure where/why/how a bug exists if that existing comparison fails. TODO **** ideally we should be asserting the version of every doc when we query it right along side the assertion for it's updated "ratings" value TODO *** most of the use of "field(ratings)" can probbaly just be replaced with "ratings" now that DV are returnable – although it's nice to have both included in the test at least once to demo that both work, but when doing that there should be a comment making it clear why Fixed testOnlyPartialUpdatesBetweenCommits ditto comment about checking return value from addAndGetVersion *** this also seems like a good place to to test if doing a redundent atomic update (either via set to the same value or via inc=0) returns a new version or not – should it? As of now, both will generate a new version. I think "inc" 0 should be dropped, and "set" same value should be versioned. I'll check if behaviour in this patch is at par with regular atomic updates; and if so, will open a separate issue for this later. ** DocInfo should be a private static class and have some javadocs Fixed ** based on how testing has gone so far, and the discover of LUCENE-7301 it seems clear that adding even single thread, single node, randomized testing of lots of diff types of add/update calls would be good to add I think we can do the same in TestStressInPlaceUpdates, by randomly setting number of writer threads to 1 sometimes. *** we could refactor/improve the "checkReplay" function I added in the last patch to do more testing of a randomly generated Iterable of "commands" (commit, doc, doc+atomic mutation, etc...) TODO *** and of course: improve checkReplay to verify RTG against hte uncommited model as well I couldn't find a way to do this for the TestInPlaceUpdate (now called TestInPlaceUpdatesStandalone in this patch). This is based on SolrTestCaseJ4. *** testReplayFromFile and getSDdoc should probably go away once we have more structured tests for doing this Fixed ** createMap can be elimianted – callers can just use SolrTestCaseJ4.map(...) Fixed ** In general the tests in this class should include more queries / sorting against the updated docvalues field after commits to ensure that the updated value is searchable & sortable TODO ** Likewise the test methods in this class should probably have a lot more RTG checks – with filter queries that constrain against the updated docvalues field, and checks of the expected version field – to ensure that is all working properly. Couldn't figure out how to do RTGs with this test, but will check RTGs + filter queries in the TestInPlaceUpdatesDistrib test (which was formerly InPlaceUpdateDistribTest) * InPlaceUpdateDistribTest Renamed to TestInPlaceUpdatesDistrib now ** SuppressCodecs should be removed 3x and 4x codec suppressions cannot be removed. ** should at least have class level javadocs explaining what's being tested TODO ** 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 TODO ** in general, the "pick a random client" logic should be refactored so that sometimes it randomly picks a CloudSolrClient TODO ** there should almost certianly be some "delete all docs and optimize" cleanup in between all of these tests Added this to the beginning of every test. I couldn't see an easy way to refactor to SolrCloudTestCase immediately, but I can look more later. docValuesUpdateTest *** should randomize numdocs Fixed *** we need to find away to eliminate the hardcoded "Thread.sleep(500);" calls... Fixed (I think it is the correct fix, but I'll review it again, testing with a fast machine and beasting it). **** 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. TODO **** 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 TODO ensureRtgWorksWithPartialUpdatesTest *** even if we're only going to test one doc, we should ensure there are a random num docs in the index (some before the doc we're editing, and some after) TODO *** if we're testing RTG, then we should be testing the version returned from every /get call against the last version returned from every update TODO outOfOrderUpdatesIndividualReplicaTest *** ditto comments about only one doc TODO *** ditto comments about testing the expected version in RTG requests TODO *** if we're sending updates direct to replicas to test how they handle out of order updates, then something better assert exactly where the leader is hosted and ensure we don't send to it by mistake TODO *** what's the point of using a threadpool and SendUpdateToReplicaTask here? why not just send the updates in a (randdomly assigned) determinisitc order? TODO **** if we are going to use an ExecutorService, then the result of awaitTermination has to be checked TODO since this tests puts replicas out of sync, a "delete all docs and optimize" followed up "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) TODO delayedReorderingFetchesMissingUpdateFromLeaderTest *** ditto previous comments about using a threadpool and SendUpdateToReplicaTask TODO *** 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. TODO. getReplicaValue *** using SolrTestCaseJ4.params(...) would make this method a lot shorter Fixed *** based on where/how this method is used, i don't understand why it returns String instead of just Object Fixed ** assertReplicaValue *** should take in some sort of assertion message and/or build/append an assertion message using the clientId Fixed *** if getReplicaValue returns an Object, this can take an "Object expected" param and eliminate abonch of toString & string concating throughout the test Fixed simulatedUpdateRequest *** if this method is going to assume that the only shard you ever want to similulate an update to is SHARD1 then the method name should be "simulatedUpdateRequestToShard1Replica" TODO *** better still - why not ask the DocRouter which shard this doc belongs in, and fetch the leader URL tha way? TODO ** most usage of "addFields" can just be replaced with a call to "sdoc(...)" to simplify code Fixed ** replace createMap usage with SolrTestCaseJ4.map Fixed ** why override tearDown if we're just calling super? Fixed ** in general, i think this test would be a lot easier to read if there were well named variables for HttpSolrClient instances pointed at specific replicas (ie HttpSolrClient SHARD1_LEADER = ...; HttpSolrClient SHARD1_REPLICA1 = ...; etc...) and passed those around to the various methods instead of magic ints (ie: "1", "2") to refer to which index in the static clients list should be used for a given update. Fixed: LEADER and NON_LEADERS now TestStressInPlaceUpdates TODO SolrDocument ** applyOlderUpdate Fixed, moved it to UpdateLog. I think Noble had moved this from UpdateLog to SolrDocument; and if so, I'll check with him once (after I'm done with other changes here marked as TODO). *** as mentioned before, i don't think this method is correct when it comes to multivalued fields, and should have more unit tests of various permutations to be sure TODO *** no matter where it lives, it should have some javadocs Fixed
          Hide
          hossman Hoss Man added a comment -

          Removed all non 3x, 4x codec suppression. They need to be suppressed as per a comment from Mikhail. ...

          that comment is over 2 years old, from a time when those codecs existed but did not support updating doc values.

          those codecs no longer exist (on either master or branch_6x) – even if someone had na existing index with segments from those codecs, they would not be supported by any Solr 6.x version because they are more then 1 major version old – we only have to worry about Lucene5* codecs and higher.

          As of now, both will generate a new version. I think "inc" 0 should be dropped, and "set" same value should be versioned. I'll check if behaviour in this patch is at par with regular atomic updates; and if so, will open a separate issue for this later.

          yeah, sorry – my point was: "whatever the current, non-patched, behavior is for the version returned from these types of updates, we need to assert that behavior is true here." – we should not be changing any semantics here, absolutely open a distinct issue for that if you think it makes sense as a future improvement.

          I think we can do the same in TestStressInPlaceUpdates, by randomly setting number of writer threads to 1 sometimes.

          Isn't that still a cloud based test with multiple nodes/shards? Even with only 1 writer thread it's going ot be harder to debug then doing more randomized testing in a single node test (via something like checkReplay as in my previous suggestion)

          I couldn't find a way to do this (check RTG against uncommitted model) for the TestInPlaceUpdate (now called TestInPlaceUpdatesStandalone in this patch). This is based on SolrTestCaseJ4.

          SolrTestCaseJ4.addAndGetVersion(...)

          Show
          hossman Hoss Man added a comment - Removed all non 3x, 4x codec suppression. They need to be suppressed as per a comment from Mikhail. ... that comment is over 2 years old, from a time when those codecs existed but did not support updating doc values. those codecs no longer exist (on either master or branch_6x) – even if someone had na existing index with segments from those codecs, they would not be supported by any Solr 6.x version because they are more then 1 major version old – we only have to worry about Lucene5* codecs and higher. As of now, both will generate a new version. I think "inc" 0 should be dropped, and "set" same value should be versioned. I'll check if behaviour in this patch is at par with regular atomic updates; and if so, will open a separate issue for this later. yeah, sorry – my point was: "whatever the current, non-patched, behavior is for the version returned from these types of updates, we need to assert that behavior is true here." – we should not be changing any semantics here, absolutely open a distinct issue for that if you think it makes sense as a future improvement. I think we can do the same in TestStressInPlaceUpdates, by randomly setting number of writer threads to 1 sometimes. Isn't that still a cloud based test with multiple nodes/shards? Even with only 1 writer thread it's going ot be harder to debug then doing more randomized testing in a single node test (via something like checkReplay as in my previous suggestion) I couldn't find a way to do this (check RTG against uncommitted model) for the TestInPlaceUpdate (now called TestInPlaceUpdatesStandalone in this patch). This is based on SolrTestCaseJ4. SolrTestCaseJ4.addAndGetVersion(...)
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited

          New patch. Fixed all comments in TestInPlaceUpdatesDistrib (formerly InPlaceUpdateDistribTest).

          what's the point of using a threadpool and SendUpdateToReplicaTask here? why not just send the updates in a (randdomly assigned) determinisitc order?

          Essentially, I need a way to send three updates to the replica asynchronously. To achieve the effect of asynchronous updates, I used a threadpool here. Three updates sent one after the other, each being a blocking call, wouldn't have simulated the leader -> replica interaction sufficiently.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited New patch. Fixed all comments in TestInPlaceUpdatesDistrib (formerly InPlaceUpdateDistribTest). what's the point of using a threadpool and SendUpdateToReplicaTask here? why not just send the updates in a (randdomly assigned) determinisitc order? Essentially, I need a way to send three updates to the replica asynchronously. To achieve the effect of asynchronous updates, I used a threadpool here. Three updates sent one after the other, each being a blocking call, wouldn't have simulated the leader -> replica interaction sufficiently.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited

          New patch. Fixes to TestInPlaceUpdatesStandalone (formerly TestInPlaceUpdate).

          TODO items in the test are:

          1. Randomize the replay based tests.
          2. Add new tests that validate sorting, searching, filter queries involving the updated DV.
          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited New patch. Fixes to TestInPlaceUpdatesStandalone (formerly TestInPlaceUpdate). TODO items in the test are: Randomize the replay based tests. Add new tests that validate sorting, searching, filter queries involving the updated DV.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          New patch.

          1. Increasing the timeout of the threadpool termination in TestInPlaceUpdatesDistrib from 2s to 5s, and added assertion message.
          2. Fixing some review comments in the TestStressInPlaceUpdates (more remains to be fixed).
          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - New patch. Increasing the timeout of the threadpool termination in TestInPlaceUpdatesDistrib from 2s to 5s, and added assertion message. Fixing some review comments in the TestStressInPlaceUpdates (more remains to be fixed).
          Hide
          hossman Hoss Man added a comment -

          I don't understand this comment – particularly in light of the changes you've made to the test since...

          what's the point of using a threadpool and SendUpdateToReplicaTask here? why not just send the updates in a (randdomly assigned) determinisitc order?

          Essentially, I need a way to send three updates to the replica asynchronously. To achieve the effect of asynchronous updates, I used a threadpool here. Three updates sent one after the other, each being a blocking call, wouldn't have simulated the leader -> replica interaction sufficiently.

          When i posted that particular question it was about outOfOrderUpdatesIndividualReplicaTest – the code in question at teh time looked like this...

          // re-order the updates for replica2
          List<UpdateRequest> reorderedUpdates = new ArrayList<>(updates);
          Collections.shuffle(reorderedUpdates, random());
          for (UpdateRequest update : reorderedUpdates) {
            SendUpdateToReplicaTask task = new SendUpdateToReplicaTask(update, clients.get(1), random());
            threadpool.submit(task);
          }
          

          ...My impression, based on the entirety of that method, was that the intent of the test was to bypass the normal distributed update logic and send carefully crafted "simulated" updates direct to each replica, such that one repliica got the (simulated from leader) updates "in order" and another replica got the (simulated from leader) updates "out of order"

          • if the point was for replica2 to get the (simulated from leader) updates "out of order" then why shuffle them - why not explicitly put them in the "wrong" order?
          • if the goal was send them asynchronously, and try to get them to happen as concurrently as possible (as you indicated above in your answer to my question) then what was the point of the "shuffle" ?

          Looking at the modified version of that code in your latest patch doesn't really help clarify things for me...

          // re-order the updates for NONLEADER 0
          List<UpdateRequest> reorderedUpdates = new ArrayList<>(updates);
          Collections.shuffle(reorderedUpdates, random());
          List<Future<UpdateResponse>> updateResponses = new ArrayList<>();
          for (UpdateRequest update : reorderedUpdates) {
            AsyncUpdateWithRandomCommit task = new AsyncUpdateWithRandomCommit(update, NONLEADERS.get(0), seed);
            updateResponses.add(threadpool.submit(task));
            // send the updates with a slight delay in between so that they arrive in the intended order
            Thread.sleep(10);
          }
          

          In the context of your answer, that it's intentional for the updates to be async...

          • why shuffle them?
          • why is there now a sleep call with an explicit comment "...so that they arrive in the intended order" ... if there is an "intended" order why would you want them to be async?

          the other SendUpdateToReplicaTask/AsyncUpdateWithRandomCommit usages exhibit the same behavior of a "sleep" in between {{ threadpool.submit(task); }} calls with similar comments about wanting to "...ensure requests are sequential..." hence my question about why threadpools are being used at all.

          Show
          hossman Hoss Man added a comment - I don't understand this comment – particularly in light of the changes you've made to the test since... what's the point of using a threadpool and SendUpdateToReplicaTask here? why not just send the updates in a (randdomly assigned) determinisitc order? Essentially, I need a way to send three updates to the replica asynchronously. To achieve the effect of asynchronous updates, I used a threadpool here. Three updates sent one after the other, each being a blocking call, wouldn't have simulated the leader -> replica interaction sufficiently. When i posted that particular question it was about outOfOrderUpdatesIndividualReplicaTest – the code in question at teh time looked like this... // re-order the updates for replica2 List<UpdateRequest> reorderedUpdates = new ArrayList<>(updates); Collections.shuffle(reorderedUpdates, random()); for (UpdateRequest update : reorderedUpdates) { SendUpdateToReplicaTask task = new SendUpdateToReplicaTask(update, clients.get(1), random()); threadpool.submit(task); } ...My impression, based on the entirety of that method, was that the intent of the test was to bypass the normal distributed update logic and send carefully crafted "simulated" updates direct to each replica, such that one repliica got the (simulated from leader) updates "in order" and another replica got the (simulated from leader) updates "out of order" if the point was for replica2 to get the (simulated from leader) updates "out of order" then why shuffle them - why not explicitly put them in the "wrong" order? if the goal was send them asynchronously, and try to get them to happen as concurrently as possible (as you indicated above in your answer to my question) then what was the point of the "shuffle" ? Looking at the modified version of that code in your latest patch doesn't really help clarify things for me... // re-order the updates for NONLEADER 0 List<UpdateRequest> reorderedUpdates = new ArrayList<>(updates); Collections.shuffle(reorderedUpdates, random()); List<Future<UpdateResponse>> updateResponses = new ArrayList<>(); for (UpdateRequest update : reorderedUpdates) { AsyncUpdateWithRandomCommit task = new AsyncUpdateWithRandomCommit(update, NONLEADERS.get(0), seed); updateResponses.add(threadpool.submit(task)); // send the updates with a slight delay in between so that they arrive in the intended order Thread .sleep(10); } In the context of your answer, that it's intentional for the updates to be async... why shuffle them? why is there now a sleep call with an explicit comment "...so that they arrive in the intended order" ... if there is an "intended" order why would you want them to be async? the other SendUpdateToReplicaTask/AsyncUpdateWithRandomCommit usages exhibit the same behavior of a "sleep" in between {{ threadpool.submit(task); }} calls with similar comments about wanting to "...ensure requests are sequential..." hence my question about why threadpools are being used at all.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited

          When update2 (say a partial update) arrives before update1 (say a full update, on which update2 depends), then the call for indexing update2 is a blocking call (which finishes either after update1 is indexed, or timeout is reached).

          The intention was to:

          1. shuffle the updates (so that the 3 updates are in one of the 6 possible permutations, one of those permutations being in-order)
          2. send them out in sequence of the shuffle
          3. have them arrive at Solr in the intended order (as intended in steps 1 and 2). However, since an out of order update waits for the dependent update and blocks the call until such a dependent update arrives (or timeout is reached), the intention is to have these calls non-blocking.

          So, I wanted to send updates out sequentially (deliberately re-ordered, through a shuffle), but asynchronously (so as to keep those calls non-blocking).

          ...My impression, based on the entirety of that method, was that the intent of the test was to bypass the normal distributed update logic and send carefully crafted "simulated" updates direct to each replica, such that one repliica got the (simulated from leader) updates "in order" and another replica got the (simulated from leader) updates "out of order"

          That is exactly my intention.

          if the point was for replica2 to get the (simulated from leader) updates "out of order" then why shuffle them - why not explicitly put them in the "wrong" order?

          There could be possibly 6 permutations in terms of the mutual ordering of the 3 updates, so I used shuffle instead of choosing a particular "wrong" ordering. Of course, one of those 6 permutations is the "right" order, so that case is not consistent with the name of the test; I can make a fix to exclude that case.

          if the goal was send them asynchronously, and try to get them to happen as concurrently as possible (as you indicated above in your answer to my question) then what was the point of the "shuffle" ?

          I think I was trying: (a) asynchronously (so that out of order update doesn't block out the next update request that sends a dependent order), (b) intention was not really to test for race conditions (i.e. not really "as concurrently as possible", but maybe I don't understand the phrase correctly), but just to be concurrent enough so that a dependent update arrives before an out of order update times out.

          why is there now a sleep call with an explicit comment "...so that they arrive in the intended order" ... if there is an "intended" order why would you want them to be async?

          The point of this was to avoid situations where the shuffled list (and intended order for that testcase) was, say, "update1, update3, update2", but it actually arrived at the Solr server in the order "update1, update2, update3" due to parallel threads sending the updates at nearly the same time.

          if there is an "intended" order why would you want them to be async?

          So that the calls are non-blocking. The first out of order partial update request will block the call until timeout/dependent update is indexed.

          Do you think this makes sense? I am open to revise this entire logic if you suggest.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited When update2 (say a partial update) arrives before update1 (say a full update, on which update2 depends), then the call for indexing update2 is a blocking call (which finishes either after update1 is indexed, or timeout is reached). The intention was to: shuffle the updates (so that the 3 updates are in one of the 6 possible permutations, one of those permutations being in-order) send them out in sequence of the shuffle have them arrive at Solr in the intended order (as intended in steps 1 and 2). However, since an out of order update waits for the dependent update and blocks the call until such a dependent update arrives (or timeout is reached), the intention is to have these calls non-blocking. So, I wanted to send updates out sequentially (deliberately re-ordered, through a shuffle), but asynchronously (so as to keep those calls non-blocking). ...My impression, based on the entirety of that method, was that the intent of the test was to bypass the normal distributed update logic and send carefully crafted "simulated" updates direct to each replica, such that one repliica got the (simulated from leader) updates "in order" and another replica got the (simulated from leader) updates "out of order" That is exactly my intention. if the point was for replica2 to get the (simulated from leader) updates "out of order" then why shuffle them - why not explicitly put them in the "wrong" order? There could be possibly 6 permutations in terms of the mutual ordering of the 3 updates, so I used shuffle instead of choosing a particular "wrong" ordering. Of course, one of those 6 permutations is the "right" order, so that case is not consistent with the name of the test; I can make a fix to exclude that case. if the goal was send them asynchronously, and try to get them to happen as concurrently as possible (as you indicated above in your answer to my question) then what was the point of the "shuffle" ? I think I was trying: (a) asynchronously (so that out of order update doesn't block out the next update request that sends a dependent order), (b) intention was not really to test for race conditions (i.e. not really "as concurrently as possible", but maybe I don't understand the phrase correctly), but just to be concurrent enough so that a dependent update arrives before an out of order update times out. why is there now a sleep call with an explicit comment "...so that they arrive in the intended order" ... if there is an "intended" order why would you want them to be async? The point of this was to avoid situations where the shuffled list (and intended order for that testcase) was, say, "update1, update3, update2", but it actually arrived at the Solr server in the order "update1, update2, update3" due to parallel threads sending the updates at nearly the same time. if there is an "intended" order why would you want them to be async? So that the calls are non-blocking. The first out of order partial update request will block the call until timeout/dependent update is indexed. Do you think this makes sense? I am open to revise this entire logic if you suggest.
          Hide
          hossman Hoss Man added a comment -

          When update2 (say a partial update) arrives before update1 (say a full update, on which update2 depends), then the call for indexing update2 is a blocking call (which finishes either after update1 is indexed, or timeout is reached).

          Ahhh... now it makes sense to me. The part I wasn't getting before was that update2 blocks on the replica until it sees the update1 it is dependent on.

          I feel like there is probably a way we could write a more sophisticate "grey box" type test for this leveraging callbacks in the DebugFilter, but I'm having trouble working out what that would really look like.

          I think the hueristic approach you're taking here is generall fine for now (as a way to try to run the updates in a given order even though we know there are no garuntees) but i have a few suggestions to improve things:

          • 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
          • replace the comments on the sleep calls to make it clear that while we can't garuntee/trust what order the updates are executed in since multiple threads are involved, we're trying to bias the thread scheduling to run them in the order submitted
            • (the wording right now seems definitive and makes the code look clearly suspicious)
          • 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...

          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
            }
          }
          

          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...

          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());
          
          Show
          hossman Hoss Man added a comment - When update2 (say a partial update) arrives before update1 (say a full update, on which update2 depends), then the call for indexing update2 is a blocking call (which finishes either after update1 is indexed, or timeout is reached). Ahhh... now it makes sense to me. The part I wasn't getting before was that update2 blocks on the replica until it sees the update1 it is dependent on. I feel like there is probably a way we could write a more sophisticate "grey box" type test for this leveraging callbacks in the DebugFilter, but I'm having trouble working out what that would really look like. I think the hueristic approach you're taking here is generall fine for now (as a way to try to run the updates in a given order even though we know there are no garuntees) but i have a few suggestions to improve things: 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 replace the comments on the sleep calls to make it clear that while we can't garuntee/trust what order the updates are executed in since multiple threads are involved, we're trying to bias the thread scheduling to run them in the order submitted (the wording right now seems definitive and makes the code look clearly suspicious) 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... 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 } } 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... 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());
          Hide
          erickerickson Erick Erickson added a comment -

          I had a question come up today that I wanted to ask for posterity. What gets returned between the time we update one of these and a commit occurs? The old value or the new one? I'd assumed the old one but wanted to be sure.

          And explicitly this only applies to fields for which indexed=false I see, which answers another of the questions that came up.

          Show
          erickerickson Erick Erickson added a comment - I had a question come up today that I wanted to ask for posterity. What gets returned between the time we update one of these and a commit occurs? The old value or the new one? I'd assumed the old one but wanted to be sure. And explicitly this only applies to fields for which indexed=false I see, which answers another of the questions that came up.
          Hide
          hossman Hoss Man added a comment -

          I had a question come up today that I wanted to ask for posterity. What gets returned between the time we update one of these and a commit occurs? The old value or the new one? I'd assumed the old one but wanted to be sure.

          in theory, it's exactly identical to the existing behavior of an atomic update: searching returns only the committed values, an RTG will return the "new" (uncommitted) value.

          Show
          hossman Hoss Man added a comment - I had a question come up today that I wanted to ask for posterity. What gets returned between the time we update one of these and a commit occurs? The old value or the new one? I'd assumed the old one but wanted to be sure. in theory, it's exactly identical to the existing behavior of an atomic update: searching returns only the committed values, an RTG will return the "new" (uncommitted) value.
          Hide
          hossman Hoss Man added a comment -

          I know Ishan has been working on improving the tests based on my last batch of feedback – since then I've been reviewing the non test changes in the lastest patch.

          Here's my notes about specific class/methods as I reviewed them in individually....

          JettySolrRunner
          • javadocs, javadocs, javadocs
          XMLLoader + JavabinLoader
          • why is this param checks logic duplicated in these classes?
          • why not put this in DUP (which already has access to the request params) when it's doing it's "FROMLEADER" logic?
          AddUpdateCommand
          • these variables (like all variables) should have javadocs explaining what they are and what they mean
            • people skimming a class shouldn't have to grep the code for a variable name to understand it's purpose
          • having 2 variables here seems like it might be error prone? what does it mean if prevVersion < 0 && isInPlaceUpdate == true ? or 0 < prevVersion && isInPlaceUpdate == false ?
            • would it make more sense to use a single long prevVersion variable and have a public boolean isInPlaceUpdate() that simply does {{return (0 < prevVersion); }} ?
          TransactionLog
          • javadocs for both the new write method and the existig write method
            • explain what "prevPointer" means and note in the 2 arg method what the effective default "prevPoint" is.
          • we should really have some "int" constants for refering to the List indexes involved in these records, so instead of code like entry.get(3) sprinkled in various classes like UpdateLog and PeerSync it can be smething more readable like entry.get(PREV_VERSION_IDX)
          UpdateLog
          • javadocs for both the new LogPtr constructure and the existing constructor
            • explain what "prevPointer" means and note in the 2 arg constructure what the effective default "prevPoint" is.
          • add(AddUpdateCommand, boolean)
            • this new code for doing lookups in map, prevMap and preMap2 seems weird to me (but admitedly i'm not really an expert on UpdateLog in general and how these maps are used
            • what primarily concerns me is what the expected behavior is if the "id" isn't found in any of these maps – it looks like prevPointer defaults to "-1" regardless of whether this is an inplace update ... is that intentional? ... is it possible there are older records we will miss and need to flag that?
            • ie: do we need to worry about distinguising here between "not an in place update, therefore prePointer=-1" vs "is an in place update, but we can't find the prevPointer" ??
            • assuming this code is correct, it might be a little easier to read if it were refactored into something like:
              // nocommit: jdocs
              private synchronized long getPrevPointerForUpdate(AddUpdateCommand cmd) {
                // note: sync required to ensure maps aren't changed out form under us
                if (cmd.isInPlaceUpdate) {
                  BytesRef indexedId = cmd.getIndexedId();
                  for (Map<BytesRef,TransactionLog> currentMap : Arrays.asList(map, prevMap, prevMap2)) {
                    LogPtr prevEntry = currentMap.get(indexedId);
                    if (null != prevEntry) {
                      return prevEntry.pointer;
                    }
                  }
                }
                return -1; // default when not inplace, or if we can't find a previous entry
              }
              
          • applyPartialUpdates
            • it seems like this method would be a really good candidate for some direct unit testing?
              • ie: construct a synthetic UpdateLog, and confirm applyPartialUpdates does the right thing
            • the sync block in this method, and how the resulting lookupLogs list is used subsequently, doesn't seem safe to me – particularly the way getEntryFromTLog calls incref/decref on each TransactionLog as it loops over that list...
              • what prevents some other thread from decref'ing one of these TransactionLog objects (and possibly auto-closing it) in between the sync block and the incref in getEntryFromTLog?
                • (most existing usages of TransactionLog.incref() seem to be in blocks that sync on the UpdateLog – and the ones that aren't in sync blocks look sketchy to me as well)
              • in general i'm wondering if lookupLogs should be created outside of the while loop, so that there is a consistent set of "logs" for the duration of the method call ... what happens right now if some other thread changes tlog/prevMapLog/prevMapLog2 in between iterations of the while loop?
            • shouldn't we make some sanity check assertions about the results of getEntryFromTLog? – there's an INVALID_STATE if it's not an ADD or a list of 5 elements, but what about actually asserting that it's either an ADD or an UPDATE_INPLACE? ... what about asserting the doc's uniqueKey matches?
              • (because unless i'm missing something, it's possible for 2 docs to have the same version, so if there is a glitch in the pointer we can't just trust the version check can we?)
            • partialUpdateEntry seems like a missleading variable name ... can't it be either a full document, or partial update (not in place), or an UPDATE_INPLACE partial update?
            • if there is only 1 way to break out of this while loop, then the method would probably be easier to make sense of if the applyOlderUpdate and return 0 calls replaced the break statement
            • semi-related: while (true) is generally a red flag: it seems like might be better if it was refactored inot a while (0 <= prevPointer) loop?
          • getEntryFromTLog
            • I don't really see the point of using get(i) over and over .. why not a simple for (TransactionLog log : lookupLogs) ?
            • why is the Exception & Error handling here just a debug log? shouldn't that be a pretty hard fail?
            • as mentioned above re: applyPartialUpdates, isn't is possible for 2 diff docs to have the same version? if we get unlucky and those 2 docs, with identical versions, at the same position in diff TransactionLog files then isn't a sanity check of the doc ID in applyPartialUpdates too late? ... if applyPartialUpdates tries again it's just going to keep getting the same (wrong) document. It seems like this method actaully needs to know the ID it's looking for, and "skip" any entries thta don't match, checking the next (older) TransactionLog
          • lookupPartialUpdates
            • what is the purpose/intent of this method? ... it seems to be unused.
          • doReplay
            • switch statement ordering...
              • in theory, switch statement cases should be ordered from most likeley to least likely (it's a microoptimization, but in heavily executed loops it might be worth it)
              • so i wouldn't inject UPDATE_INPLACE at the begining of the switch – it should definitely come after ADD, probably best to put it at the end of the list
            • why is entry.get(2) commented out? and why does it say "idBytes" ? ... isn't slot #2 the prevPointer? copy paste confusion from "ADD" ?
              • if slot#2 really isn't needed in this code, get rid of the missleading comment about idBytes and replace it with an explicit comment that prevVersion isn't needed for reply.
          PeerSync
          • ditto comments about switch statement ordering from above comments about UpdateLog
          • a lot of code here looks duplicated straight from UpdateLog.doReplay
            • I realize that's true for the other case values as well, but bad code shouldn't be emulated
            • lets refactor this duplicated code into a new public static AddUpdateCommand updateCommandFromTlog(List tlogEntry) method in UpdateLog and re-use it here.
          • log.info looks wrong here ... especially inside of an if (debug) block ... pretty sure this should be log.debug like the other case blocks
          DirectUpdateHandler2
          • I don't really understand why we need any schema/DocValue checks here?
          • If cmd.isInPlaceUpdate is true, then doesn't that mean the update is by definition an in place update that only contains values for DV fields? ... wasn't it the responsibility of the upstream code that set that value to true to ensure that?
            • if not, why can't it be? (ie: why can't we move the schema checks upstream, and out of the sync block here?
          • if that's not what cmd.isInPlaceUpdate means, then why isn't there any error handling here in the event that non-DV field/values are found in the luceneDocument? ... doesn't that mean we need to fall back to the original writer.updateDocument call?
          • If it is neccessary to do SchemaField validation here for some reason, then shouldn't it be the exact same validation done in AtomicUpdateDocumentMerger.isSupportedForInPlaceUpdate ?
          AtomicUpdateDocumentMerger
          • isSupportedForInPlaceUpdate
            • shouldn't this method either be static or final? the rules don't change if someone subclasses AtomicUpdateDocumentMerger do they?
            • why isn't TrieDateField also valid? ... could this just be checking for instanceof TrieField ?
              • particularly suspicious since doInPlaceUpdateMerge does in fact check instanceof TrieField
            • if the intent is truely to only support "numerics" then, why not instanceof NumericValueFieldType ?
            • shouldn't we also check "not-indexed" and "not-stored" here as well?
          • doInPlaceUpdateMerge
            • why does this method have 3 args? can't all of the neccessary info be deterined from the AddUpdateCommand ?
            • this method seems like another good candidate for some explicit unit testing...
              • build up an index & tlog with some explicitly crated non trivial docs/updates, then call this method with a variety of inputs and assert the expected modifications to the AddUpdateCommand (or assert no modifications if they aren't viable in place update candaites
              • then hard commit everything in the tlog and assert that all the same calls return the exact same output/modifications.
            • we should probably continue to assume the common case is not to need (in-place) updates ... just regular adds. in which case anything we can do to short circut out faster – before any checks that require stuff like SchemaField – wouldn't reordering the checks in the loop to something like this be equivilent to what we have currently but faster in the common case? ...
              for (SolrInputField f : sdoc) {
                final String fname = f.getName();
                if (idField.getName().equals(fname)) {
                  continue;
                }
                if (! f.getValue() instanceof Map) {
                  // doc contains field value that is not an update, therefore definitely not an inplace update
                  return false;
                }
                if (!isSupportedForInPlaceUpdate(schema.getField(fname))) {
                  // doc contains update to a field that does not support in place updates
                  return false;
                }
              }
              
              • Even if i've overloked something and that code isn't better, i think in general the "is this sdoc a candidate for in place updating" logic should be refactored into a public static helper method that has some unit tests.
            • this method calls RealTimeGetComponent.getInputDocumentFromTlog but doesn't check for the special DELETED return value...
              • definitely smells like a bug ... there are many assumptions made about uncommittedDoc as long as it's non-null
              • based on this, now i really want to see more testing of in place updates mixed with document deletions:
                • some explicit single node testing of "add docX, delete docX, do a 'set':'42' on docX"
                • introduce some randomized deleteById calls into the randomized single/multi node tests
            • this method calls RealTimeGetComponent.getVersionFromTlog which has docs that say "If null is returned, it could still be in the latest index."
              • i don't see any code accounting for that possibility, cmd.prevVersion is just blindly assigned the null in that case ... which could lead to an NPE since it's declared as public long prevVersion
              • the fact that this hasn't caused an NPE in testing is a red flag that there is a code path not being tested here ... but at a glance i can't really tell what it is?
            • In general, I guess I don't really understand why this method is jumping through as many hoops as it is with the RTG code?
              • it seems to duplicate a lot of functionality already in RealTimeGetComponent.getInputDocument ... why not just use that method?
              • if the concern is avoiding the searcher.doc(docid) call to get all stored fields, perhaps RealTimeGetComponent.getInputDocument could be refactored to make it easier to re-use most of the functionality here? ... not sure what would make the most sense off the top of my head.
              • at a minimum it seems like using SolrIndexSearcher.decorateDocValueFields(...) would make more sense then doing it ourselves as we loop over the fields – we can even pass in the explicit list of field names we know we care about based on the SolrInputDocument (or even just the field names we know use "inc" if that's all we really need)
                • (Or is there something i'm missing about why using decorateDocValueFields would be a mistake here?)
            • in the switch statement, shouldn't we be using the doSet and doInc methods to actaully cary out the operations?
              • that would simplify the "inc" case a lot
            • the default on the switch statement looks sketchy to me ... i understand that only "inc" and "set" are supported, but why does this method only warn if it sees something else? shouldn't this be a hard failure?
              • for that matter: shouldn't the instanceof Map check when looping over the fields at the begining of the method short circut out if the Map contains an operation that isn't one of the supported "in place update" operations?
              • in fact: if we pre-checked the Maps only contained "set" and "inc", and used something like decorateDocValueFields (or did the equivilent ourselves in a smaller loop) then couldn't we also simplify this method a lot by just delegating to the existing merge(SolrInputDocument,SolrInputDocument) method?
            • these assumptions seem sketchy, if that's the only reason for these "else" blocks then let's include some asert fieldName.equals(...) calls to prove it...
                        } else { // for version field, which is a docvalue but there's no set/inc operation
                          ...
                        }
                      } else { // for id field
                        ...
              
              • in particluar i'm curious about the VERSION_FIELD...
                • this method is only called from one place – DistributedUpdateProcessor.getUpdatedDocument – and in the existing code of that method, when a SolrInputDocument is fetched from RealTimeGetComponent, the VERSION_FIELD is explicitly removed from it before using it & returning.
                • should this method also be explicitly removing the VERSION_FIELD field? and/or should the caller (getUpdatedDocument) be removing it consistently before returning?
          RealTimeGetComponent
          • process
            • I like this SearcherInfo refactoring, but a few suggestions:
              • it should be promoted into a (private) static (inner) class ... no need for a new class instance every time RealTimeGetComponent.process is called.
              • add a one arg constructor and pass the SolrCore to that.
              • javadocs, javadocs, javadocs .... note that it's not thread safe
              • let's make searcherHolder and searcher private, and replace direct searcher access with:
                public SolrIndexSearcher getSearcher() {
                  assert null != searcher : "init not called!";
                  return searcher;
                }
                
            • in the switch statement, it seems like there is a lot of code duplicated between the ADD and UPDATE_INPLACE cases
              • why 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 ?
          • resolveFullDocument
            • see comments above about modifying this method to call toSolrDocument itself rather then expecting as input, and return early if the entry is an UpdateLog.ADD
            • let's put an assert 0==lastPrevPointer in this else block in case someone improves/breaks ulog.applyPartialUpdates to return -2 in the future...
                  } else { // i.e. lastPrevPointer==0
              
            • since ulog.applyPartialUpdates(...) is a No-Op when prevPointer == -1, can't we remove the redundent calls to mergePartialDocWithFullDocFromIndex & reopenRealtimeSearcherAndGet before and after calling ulog.applyPartialUpdates(...) ... ie:
                  long prevPointer = (long) logEntry.get(2);
                  long prevVersion = (long) logEntry.get(3);
                  // this is a No-Op if prevPointer is already negative, otherwise...
                  // get the last full document from ulog
                  prevPointer = ulog.applyPartialUpdates(idBytes, prevPointer, prevVersion, partialDoc);
                  if (-1 == prevPointer) {
                    ...
                  } else if (0 < prevPointer) {
                    ...
                  } else {
                    assert 0 == prevPointer;
                    ...
                  }
              
              • If there is some reason i'm not seeing why it's important to call mergePartialDocWithFullDocFromIndex & reopenRealtimeSearcherAndGet before calling ulog.applyPartialUpdates, then perhaps we should at least refactor the "if mergedDoc == null, return reopenRealtimeSearcherAndGet" logic into mergePartialDocWithFullDocFromIndex since that's the only way it's ever used.
          • mergePartialDocWithFullDocFromIndex
            • since this is a private method, what's the expected usage of docidFromIndex ? ... it's never used, so can we refactor it away?
            • see previous comment about refactoring this method to automatically return reopenRealtimeSearcherAndGet(...) when it would otherwise return null
          • reopenRealtimeSearcherAndGet
            • javadocs, javadocs, javadocs
            • since this method is only used in conjunction with mergePartialDocWithFullDocFromIndex, if the code is refactored so that mergePartialDocWithFullDocFromIndex calls this method directly (see suggestion above), we could make a small micro-optimization by changing the method sig to take in a Term to (re)use rather then passing in idBytes and calling core.getLatestSchema().getUniqueKeyField() twice.
            • re: the INVALID_STATE ... is that really a fatal error, or should this method be accounting for the possibility of a doc that has been completley deleted (or was never in the index) in a diff way?
          • getInputDocumentFromTlog
            • lets put an explicit comment here noting that we are intentionally falling through to the Updatelog.ADD case
          • getVersionFromTlog
            • based on it's usage, should this method be changed to return -1 instead of null? ... not clear to me from the given caller usage ... if so should be be declared to return long instead of Long ?
          DistributedUpdateProcessor
          • Similar question from AddUpdateCommand: do we really need 2 distinct params here, or would it be cleaner / less error-prone to have a single distrib.inplace.prevversion which indicates we're doing an inplace update if it's a positive # ?
          • versionAdd
            • "Something weird has happened" ... perhaps waitForDependentUpdates should return the lastFoundVersion so we can use it in this error msg? ... "Dependent version not found after waiting: ..."
            • "TODO: Should we call waitForDependentUpdates() again?" ... i don't see how that would help? if we get to this point it definitely seems like a "WTF?" fail fast situation.
            • the way the existing code in this method has been refactored into an "else" block (see comment: // non inplace update, i.e. full document update) makes sense, but the way the Long lastVersion declaration was refactored out of that block to reuse with the "if isInPlaceUpdate" side of things is a bit confusing and doesn't seem to actaully simplify anything...
              • It's not used after the if/else block, so there's no reason to declare it before the "if" statement
              • by definition it must be null in the "else" case, so if (lastVersion == null) will also be true in that code path
              • it seems simpiler to just let both the "if" and "else" branches declare/define their own "Long lastVersion" and not risk any confusion about why that variable needs to be "shared"
          • waitForDependentUpdates
            • javadocs, javadocs, javadocs
            • fetchFromLeader is always true? why not eliminate arg?
            • I'm not a concurrency expert, but is this control flow with the sync/wait actaully safe? ... my understanding was the conditional check you're waiting on should always be a loop inside the sync block?
            • even if there are no spurious returns from wait, logging at "info" level every 100ms is excessive ... logging that often at trace seems excessive.
              • why don't we log an info once at the start of method (since our of order updates should be rare) and once at debug anytime lastFoundVersion changes? (diff log message if/when lastFoundVersion is == or > prev)
              • the "Waiting attempt" counter "i" doesn't seem like useful info to log given how wait(...) works
            • "...Dropping current update." - that log msg seems missleading, this method doesn't do anything to drop the current update, it just assumes the current update will be droped later
            • i don't really see the point of the boolean foundDependentUpdate variable... why not change the only place where it's set to true to return immediately?
            • fetchMissingUpdateFromLeader can return null, but that possibility isn't handled here.
            • if (uc instanceof AddUpdateCommand) ... what if it's not?
              • currently it's just silently ignored
              • is this a viable scenerio that needs accounted for, or an exceptional scenerio that should have error checking?
              • looks like maybe it's just a confusing way to do a null check?
            • ((System.nanoTime()-startTime)/1000000 ) ... that's a missleading thing to include in the Exception
              • we didn't wait that long, we waited at most 5 seconds – who knows how long we spent calling fetchMissingUpdateFromLeader & executing it.
          • fetchMissingUpdateFromLeader
            • javadocs, javadocs, javadocs
            • should we really be constructing a new HttpSolrClient on the fly like this?
            • is this on the fly SolrClient + GenericSolrRequest going to work if/when the security auth/acl features are in use in the solr cluster?
            • this seems to assume the node that forwarded us the current request (via get(DISTRIB_FROM) is still the leader – but what if there was a leader election?
              • if the leader failed, and a new one was elected, isn't that a pretty viable/likeley reason why waitForDependentUpdates timed-out and needed to call fetchMissingUpdateFromLeader in the first place?
            • e.printStackTrace(); ... huge red flag that serious error handling is missing here.
            • This method seems like it expects the possibility that missingUpdates will contain more then one entry, and if it does contain more then one entry it will convert/copy all of them into the updates list – but then it will completley ignore all but the first one.
              • if we don't expect more then 1, why not assert that?
              • if we expect more then one, and they are all important, why don't we return List<UpdateCommand> ?
              • if we expect more then one, but only care about one in particularly – why loop over all of them?
                • how do we know for sure which one in the list is the one we care about?
            • ditto comments about PeerSync & UpdateLog and creating a public static AddUpdateCommand updateCommandFromTlog(List tlogEntry) somwhere that can be reused here as well
            • switch statement should have some sort of default case ... even if i'ts just to throw an error because anything but an ADD or UPDATE_INPLACE is impossible
            • need to future proof this code against the posibility of other stuff down the road

          And here are some general overall comments / impressions I got while reviewing the code and then edited up once i was all finished...

          • Given that this patch requires some non-trivial changes to the types of records that go in the update log, and requires corisponding changes to PeerSync, it seems like there should definitely be some very explicit testing of log reply and peer sync
            • ie: TestReplay and PeerSyncTest should be updated to include a variety of scenerios involving in-place updates
          • after seeing how complex & hairy some of the new code has to be around the diff handling of "in-place atomic updates", vs existing "atomic updates" (that aren't in place) It seems like we should definitely have more test code that mixes and matches diff types of "updates"
            • static, non randomized, examples of explicit tests we should definitely have...
              • a doc gets a sequence of atomic updates each containing multiple "in place" inc/set ops on diff fields
              • a doc gets a sequence of atomic updates each containing multiple inc/set ops, where a single update may have a mix of "in place" vs "not in place" eligable ops (due to diff fields having diff docvalue/stored settings)
            • our randomized (cloud and non-cloud) testing of in-place updates should also include updates to the canidate docs that may ocasionally not be viable "in-place" updates (because they involved updates to additional non-dv fields)
            • in all of these tests we should be checking that both the RTG and regualr search results make sense
          • we also need a lot more testing of various deleteById and deleteByQuery commands mixed with in-place atomic updates
            • both deleteByQuerys against the DV fields used in the in-place atomic updates as well as DBQs against other fields in the documents
            • test the results of (uncommited) RTG as well as searches when these deletes are intermixed with in-place atomic updates
            • test the results of peer sync and reply when deletes are mixed with in-place atomic updates.
            • test that we correctly get 409 error codes when trying to do in-place updates w/optimistic concurrency after a delete (and vice versa: DBQ/dID afte in-place update)
          • all logging needs heavily audited
            • there's a lot of logging happening at the info and debug level that should probably be much lower.
            • likewise there may be a few existing info or debug msgs that might be good candidates for warn or error level msgs.
          • uniqueKey and Exception/log msgs
            • there is a lot of log msgs or Exception msgs that cite a version# when reporting a problem, but don't include the uniqueKey of the document involved
            • these messages aren't going to be remotely useful to end users w/o also including the (human readable) uniqueKey of the document involved.
          • it feels like we now have a really large number of methods involved in the merging/combining/converting of documents to apply atomic updates ("in place" or otherwise) ... either for use in RTG, or for use when writing updates to disk, or from reading from the tlog, etc...
            • the ones that jump out at me w/o digging very hard...
              RealTimeGetComponent.resolveFullDocument
              RealTimeGetComponent.toSolrDoc
              RealTimeGetComponent.mergePartialDocWithFullDocFromIndex
              UpdateLog.applyPartialUpdates
              UpdateLog.applyOlderUpdate
              AtomicUpdateDocumentMerger.merge
              AtomicUpdateDocumentMerger.doInPlaceUpdateMerge
              
            • i can't help but wonder if there is room for consolidation?
            • in many cases these "merge" methods actually delegate to other "merge" methods, before/after applying additional logic – in which case at a minimum using @link or @see tags in the javadocs to make this (intentional) relationship/similarity more obvious would be helpful.
            • in cases where methods do not delegate to eachother, or have any relationship w/eachother, having @link mentions of eachother in the javadocs to compare/constrast why they are different would be equally helpful.
            • and who knows – perhaps in the process of writing these docs we'll find good oportunities to refactor/consolidate
          Show
          hossman Hoss Man added a comment - I know Ishan has been working on improving the tests based on my last batch of feedback – since then I've been reviewing the non test changes in the lastest patch. Here's my notes about specific class/methods as I reviewed them in individually.... JettySolrRunner javadocs, javadocs, javadocs XMLLoader + JavabinLoader why is this param checks logic duplicated in these classes? why not put this in DUP (which already has access to the request params) when it's doing it's "FROMLEADER" logic? AddUpdateCommand these variables (like all variables) should have javadocs explaining what they are and what they mean people skimming a class shouldn't have to grep the code for a variable name to understand it's purpose having 2 variables here seems like it might be error prone? what does it mean if prevVersion < 0 && isInPlaceUpdate == true ? or 0 < prevVersion && isInPlaceUpdate == false ? would it make more sense to use a single long prevVersion variable and have a public boolean isInPlaceUpdate() that simply does {{return (0 < prevVersion); }} ? TransactionLog javadocs for both the new write method and the existig write method explain what "prevPointer" means and note in the 2 arg method what the effective default "prevPoint" is. we should really have some "int" constants for refering to the List indexes involved in these records, so instead of code like entry.get(3) sprinkled in various classes like UpdateLog and PeerSync it can be smething more readable like entry.get(PREV_VERSION_IDX) UpdateLog javadocs for both the new LogPtr constructure and the existing constructor explain what "prevPointer" means and note in the 2 arg constructure what the effective default "prevPoint" is. add(AddUpdateCommand, boolean) this new code for doing lookups in map , prevMap and preMap2 seems weird to me (but admitedly i'm not really an expert on UpdateLog in general and how these maps are used what primarily concerns me is what the expected behavior is if the "id" isn't found in any of these maps – it looks like prevPointer defaults to "-1" regardless of whether this is an inplace update ... is that intentional? ... is it possible there are older records we will miss and need to flag that? ie: do we need to worry about distinguising here between "not an in place update, therefore prePointer=-1" vs "is an in place update, but we can't find the prevPointer" ?? assuming this code is correct, it might be a little easier to read if it were refactored into something like: // nocommit: jdocs private synchronized long getPrevPointerForUpdate(AddUpdateCommand cmd) { // note: sync required to ensure maps aren't changed out form under us if (cmd.isInPlaceUpdate) { BytesRef indexedId = cmd.getIndexedId(); for (Map<BytesRef,TransactionLog> currentMap : Arrays.asList(map, prevMap, prevMap2)) { LogPtr prevEntry = currentMap.get(indexedId); if ( null != prevEntry) { return prevEntry.pointer; } } } return -1; // default when not inplace, or if we can't find a previous entry } applyPartialUpdates it seems like this method would be a really good candidate for some direct unit testing? ie: construct a synthetic UpdateLog, and confirm applyPartialUpdates does the right thing the sync block in this method, and how the resulting lookupLogs list is used subsequently, doesn't seem safe to me – particularly the way getEntryFromTLog calls incref/decref on each TransactionLog as it loops over that list... what prevents some other thread from decref'ing one of these TransactionLog objects (and possibly auto-closing it) in between the sync block and the incref in getEntryFromTLog? (most existing usages of TransactionLog.incref() seem to be in blocks that sync on the UpdateLog – and the ones that aren't in sync blocks look sketchy to me as well) in general i'm wondering if lookupLogs should be created outside of the while loop, so that there is a consistent set of "logs" for the duration of the method call ... what happens right now if some other thread changes tlog/prevMapLog/prevMapLog2 in between iterations of the while loop? shouldn't we make some sanity check assertions about the results of getEntryFromTLog? – there's an INVALID_STATE if it's not an ADD or a list of 5 elements, but what about actually asserting that it's either an ADD or an UPDATE_INPLACE? ... what about asserting the doc's uniqueKey matches? (because unless i'm missing something, it's possible for 2 docs to have the same version, so if there is a glitch in the pointer we can't just trust the version check can we?) partialUpdateEntry seems like a missleading variable name ... can't it be either a full document, or partial update (not in place), or an UPDATE_INPLACE partial update? if there is only 1 way to break out of this while loop, then the method would probably be easier to make sense of if the applyOlderUpdate and return 0 calls replaced the break statement semi-related: while (true) is generally a red flag: it seems like might be better if it was refactored inot a while (0 <= prevPointer) loop? getEntryFromTLog I don't really see the point of using get(i) over and over .. why not a simple for (TransactionLog log : lookupLogs) ? why is the Exception & Error handling here just a debug log? shouldn't that be a pretty hard fail? as mentioned above re: applyPartialUpdates , isn't is possible for 2 diff docs to have the same version? if we get unlucky and those 2 docs, with identical versions, at the same position in diff TransactionLog files then isn't a sanity check of the doc ID in applyPartialUpdates too late? ... if applyPartialUpdates tries again it's just going to keep getting the same (wrong) document. It seems like this method actaully needs to know the ID it's looking for, and "skip" any entries thta don't match, checking the next (older) TransactionLog lookupPartialUpdates what is the purpose/intent of this method? ... it seems to be unused. doReplay switch statement ordering... in theory, switch statement cases should be ordered from most likeley to least likely (it's a microoptimization, but in heavily executed loops it might be worth it) so i wouldn't inject UPDATE_INPLACE at the begining of the switch – it should definitely come after ADD, probably best to put it at the end of the list why is entry.get(2) commented out? and why does it say "idBytes" ? ... isn't slot #2 the prevPointer? copy paste confusion from "ADD" ? if slot#2 really isn't needed in this code, get rid of the missleading comment about idBytes and replace it with an explicit comment that prevVersion isn't needed for reply. PeerSync ditto comments about switch statement ordering from above comments about UpdateLog a lot of code here looks duplicated straight from UpdateLog.doReplay I realize that's true for the other case values as well, but bad code shouldn't be emulated lets refactor this duplicated code into a new public static AddUpdateCommand updateCommandFromTlog(List tlogEntry) method in UpdateLog and re-use it here. log.info looks wrong here ... especially inside of an if (debug) block ... pretty sure this should be log.debug like the other case blocks DirectUpdateHandler2 I don't really understand why we need any schema/DocValue checks here? If cmd.isInPlaceUpdate is true, then doesn't that mean the update is by definition an in place update that only contains values for DV fields? ... wasn't it the responsibility of the upstream code that set that value to true to ensure that? if not, why can't it be? (ie: why can't we move the schema checks upstream, and out of the sync block here? if that's not what cmd.isInPlaceUpdate means, then why isn't there any error handling here in the event that non-DV field/values are found in the luceneDocument? ... doesn't that mean we need to fall back to the original writer.updateDocument call? If it is neccessary to do SchemaField validation here for some reason, then shouldn't it be the exact same validation done in AtomicUpdateDocumentMerger.isSupportedForInPlaceUpdate ? AtomicUpdateDocumentMerger isSupportedForInPlaceUpdate shouldn't this method either be static or final? the rules don't change if someone subclasses AtomicUpdateDocumentMerger do they? why isn't TrieDateField also valid? ... could this just be checking for instanceof TrieField ? particularly suspicious since doInPlaceUpdateMerge does in fact check instanceof TrieField if the intent is truely to only support "numerics" then, why not instanceof NumericValueFieldType ? shouldn't we also check "not-indexed" and "not-stored" here as well? doInPlaceUpdateMerge why does this method have 3 args? can't all of the neccessary info be deterined from the AddUpdateCommand ? this method seems like another good candidate for some explicit unit testing... build up an index & tlog with some explicitly crated non trivial docs/updates, then call this method with a variety of inputs and assert the expected modifications to the AddUpdateCommand (or assert no modifications if they aren't viable in place update candaites then hard commit everything in the tlog and assert that all the same calls return the exact same output/modifications. we should probably continue to assume the common case is not to need (in-place) updates ... just regular adds. in which case anything we can do to short circut out faster – before any checks that require stuff like SchemaField – wouldn't reordering the checks in the loop to something like this be equivilent to what we have currently but faster in the common case? ... for (SolrInputField f : sdoc) { final String fname = f.getName(); if (idField.getName().equals(fname)) { continue ; } if (! f.getValue() instanceof Map) { // doc contains field value that is not an update, therefore definitely not an inplace update return false ; } if (!isSupportedForInPlaceUpdate(schema.getField(fname))) { // doc contains update to a field that does not support in place updates return false ; } } Even if i've overloked something and that code isn't better, i think in general the "is this sdoc a candidate for in place updating" logic should be refactored into a public static helper method that has some unit tests. this method calls RealTimeGetComponent.getInputDocumentFromTlog but doesn't check for the special DELETED return value... definitely smells like a bug ... there are many assumptions made about uncommittedDoc as long as it's non-null based on this, now i really want to see more testing of in place updates mixed with document deletions: some explicit single node testing of "add docX, delete docX, do a 'set':'42' on docX" introduce some randomized deleteById calls into the randomized single/multi node tests this method calls RealTimeGetComponent.getVersionFromTlog which has docs that say "If null is returned, it could still be in the latest index." i don't see any code accounting for that possibility, cmd.prevVersion is just blindly assigned the null in that case ... which could lead to an NPE since it's declared as public long prevVersion the fact that this hasn't caused an NPE in testing is a red flag that there is a code path not being tested here ... but at a glance i can't really tell what it is? In general, I guess I don't really understand why this method is jumping through as many hoops as it is with the RTG code? it seems to duplicate a lot of functionality already in RealTimeGetComponent.getInputDocument ... why not just use that method? if the concern is avoiding the searcher.doc(docid) call to get all stored fields, perhaps RealTimeGetComponent.getInputDocument could be refactored to make it easier to re-use most of the functionality here? ... not sure what would make the most sense off the top of my head. at a minimum it seems like using SolrIndexSearcher.decorateDocValueFields(...) would make more sense then doing it ourselves as we loop over the fields – we can even pass in the explicit list of field names we know we care about based on the SolrInputDocument (or even just the field names we know use "inc" if that's all we really need) (Or is there something i'm missing about why using decorateDocValueFields would be a mistake here?) in the switch statement, shouldn't we be using the doSet and doInc methods to actaully cary out the operations? that would simplify the "inc" case a lot the default on the switch statement looks sketchy to me ... i understand that only "inc" and "set" are supported, but why does this method only warn if it sees something else? shouldn't this be a hard failure? for that matter: shouldn't the instanceof Map check when looping over the fields at the begining of the method short circut out if the Map contains an operation that isn't one of the supported "in place update" operations? in fact: if we pre-checked the Maps only contained "set" and "inc", and used something like decorateDocValueFields (or did the equivilent ourselves in a smaller loop) then couldn't we also simplify this method a lot by just delegating to the existing merge(SolrInputDocument,SolrInputDocument) method? these assumptions seem sketchy, if that's the only reason for these "else" blocks then let's include some asert fieldName.equals(...) calls to prove it... } else { // for version field, which is a docvalue but there's no set/inc operation ... } } else { // for id field ... in particluar i'm curious about the VERSION_FIELD ... this method is only called from one place – DistributedUpdateProcessor.getUpdatedDocument – and in the existing code of that method, when a SolrInputDocument is fetched from RealTimeGetComponent , the VERSION_FIELD is explicitly removed from it before using it & returning. should this method also be explicitly removing the VERSION_FIELD field? and/or should the caller ( getUpdatedDocument ) be removing it consistently before returning? RealTimeGetComponent process I like this SearcherInfo refactoring, but a few suggestions: it should be promoted into a (private) static (inner) class ... no need for a new class instance every time RealTimeGetComponent.process is called. add a one arg constructor and pass the SolrCore to that. javadocs, javadocs, javadocs .... note that it's not thread safe let's make searcherHolder and searcher private, and replace direct searcher access with: public SolrIndexSearcher getSearcher() { assert null != searcher : "init not called!" ; return searcher; } in the switch statement, it seems like there is a lot of code duplicated between the ADD and UPDATE_INPLACE cases why 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 ? resolveFullDocument see comments above about modifying this method to call toSolrDocument itself rather then expecting as input, and return early if the entry is an UpdateLog.ADD let's put an assert 0==lastPrevPointer in this else block in case someone improves/breaks ulog.applyPartialUpdates to return -2 in the future... } else { // i.e. lastPrevPointer==0 since ulog.applyPartialUpdates(...) is a No-Op when prevPointer == -1 , can't we remove the redundent calls to mergePartialDocWithFullDocFromIndex & reopenRealtimeSearcherAndGet before and after calling ulog.applyPartialUpdates(...) ... ie: long prevPointer = ( long ) logEntry.get(2); long prevVersion = ( long ) logEntry.get(3); // this is a No-Op if prevPointer is already negative, otherwise... // get the last full document from ulog prevPointer = ulog.applyPartialUpdates(idBytes, prevPointer, prevVersion, partialDoc); if (-1 == prevPointer) { ... } else if (0 < prevPointer) { ... } else { assert 0 == prevPointer; ... } If there is some reason i'm not seeing why it's important to call mergePartialDocWithFullDocFromIndex & reopenRealtimeSearcherAndGet before calling ulog.applyPartialUpdates , then perhaps we should at least refactor the "if mergedDoc == null, return reopenRealtimeSearcherAndGet" logic into mergePartialDocWithFullDocFromIndex since that's the only way it's ever used. mergePartialDocWithFullDocFromIndex since this is a private method, what's the expected usage of docidFromIndex ? ... it's never used, so can we refactor it away? see previous comment about refactoring this method to automatically return reopenRealtimeSearcherAndGet(...) when it would otherwise return null reopenRealtimeSearcherAndGet javadocs, javadocs, javadocs since this method is only used in conjunction with mergePartialDocWithFullDocFromIndex , if the code is refactored so that mergePartialDocWithFullDocFromIndex calls this method directly (see suggestion above), we could make a small micro-optimization by changing the method sig to take in a Term to (re)use rather then passing in idBytes and calling core.getLatestSchema().getUniqueKeyField() twice. re: the INVALID_STATE ... is that really a fatal error, or should this method be accounting for the possibility of a doc that has been completley deleted (or was never in the index) in a diff way? getInputDocumentFromTlog lets put an explicit comment here noting that we are intentionally falling through to the Updatelog.ADD case getVersionFromTlog based on it's usage, should this method be changed to return -1 instead of null? ... not clear to me from the given caller usage ... if so should be be declared to return long instead of Long ? DistributedUpdateProcessor Similar question from AddUpdateCommand: do we really need 2 distinct params here, or would it be cleaner / less error-prone to have a single distrib.inplace.prevversion which indicates we're doing an inplace update if it's a positive # ? versionAdd "Something weird has happened" ... perhaps waitForDependentUpdates should return the lastFoundVersion so we can use it in this error msg? ... "Dependent version not found after waiting: ..." "TODO: Should we call waitForDependentUpdates() again?" ... i don't see how that would help? if we get to this point it definitely seems like a "WTF?" fail fast situation. the way the existing code in this method has been refactored into an "else" block (see comment: // non inplace update, i.e. full document update ) makes sense, but the way the Long lastVersion declaration was refactored out of that block to reuse with the "if isInPlaceUpdate" side of things is a bit confusing and doesn't seem to actaully simplify anything... It's not used after the if/else block, so there's no reason to declare it before the "if" statement by definition it must be null in the "else" case, so if (lastVersion == null) will also be true in that code path it seems simpiler to just let both the "if" and "else" branches declare/define their own "Long lastVersion" and not risk any confusion about why that variable needs to be "shared" waitForDependentUpdates javadocs, javadocs, javadocs fetchFromLeader is always true? why not eliminate arg? I'm not a concurrency expert, but is this control flow with the sync/wait actaully safe? ... my understanding was the conditional check you're waiting on should always be a loop inside the sync block? even if there are no spurious returns from wait, logging at "info" level every 100ms is excessive ... logging that often at trace seems excessive. why don't we log an info once at the start of method (since our of order updates should be rare) and once at debug anytime lastFoundVersion changes? (diff log message if/when lastFoundVersion is == or > prev) the "Waiting attempt" counter "i" doesn't seem like useful info to log given how wait(...) works "...Dropping current update." - that log msg seems missleading, this method doesn't do anything to drop the current update, it just assumes the current update will be droped later i don't really see the point of the boolean foundDependentUpdate variable... why not change the only place where it's set to true to return immediately? fetchMissingUpdateFromLeader can return null, but that possibility isn't handled here. if (uc instanceof AddUpdateCommand) ... what if it's not? currently it's just silently ignored is this a viable scenerio that needs accounted for, or an exceptional scenerio that should have error checking? looks like maybe it's just a confusing way to do a null check? ((System.nanoTime()-startTime)/1000000 ) ... that's a missleading thing to include in the Exception we didn't wait that long, we waited at most 5 seconds – who knows how long we spent calling fetchMissingUpdateFromLeader & executing it. fetchMissingUpdateFromLeader javadocs, javadocs, javadocs should we really be constructing a new HttpSolrClient on the fly like this? is this on the fly SolrClient + GenericSolrRequest going to work if/when the security auth/acl features are in use in the solr cluster? this seems to assume the node that forwarded us the current request (via get(DISTRIB_FROM) is still the leader – but what if there was a leader election? if the leader failed, and a new one was elected, isn't that a pretty viable/likeley reason why waitForDependentUpdates timed-out and needed to call fetchMissingUpdateFromLeader in the first place? e.printStackTrace(); ... huge red flag that serious error handling is missing here. This method seems like it expects the possibility that missingUpdates will contain more then one entry, and if it does contain more then one entry it will convert/copy all of them into the updates list – but then it will completley ignore all but the first one. if we don't expect more then 1, why not assert that? if we expect more then one, and they are all important, why don't we return List<UpdateCommand> ? if we expect more then one, but only care about one in particularly – why loop over all of them? how do we know for sure which one in the list is the one we care about? ditto comments about PeerSync & UpdateLog and creating a public static AddUpdateCommand updateCommandFromTlog(List tlogEntry) somwhere that can be reused here as well switch statement should have some sort of default case ... even if i'ts just to throw an error because anything but an ADD or UPDATE_INPLACE is impossible need to future proof this code against the posibility of other stuff down the road And here are some general overall comments / impressions I got while reviewing the code and then edited up once i was all finished... Given that this patch requires some non-trivial changes to the types of records that go in the update log, and requires corisponding changes to PeerSync , it seems like there should definitely be some very explicit testing of log reply and peer sync ie: TestReplay and PeerSyncTest should be updated to include a variety of scenerios involving in-place updates after seeing how complex & hairy some of the new code has to be around the diff handling of "in-place atomic updates", vs existing "atomic updates" (that aren't in place) It seems like we should definitely have more test code that mixes and matches diff types of "updates" static, non randomized, examples of explicit tests we should definitely have... a doc gets a sequence of atomic updates each containing multiple "in place" inc/set ops on diff fields a doc gets a sequence of atomic updates each containing multiple inc/set ops, where a single update may have a mix of "in place" vs "not in place" eligable ops (due to diff fields having diff docvalue/stored settings) our randomized (cloud and non-cloud) testing of in-place updates should also include updates to the canidate docs that may ocasionally not be viable "in-place" updates (because they involved updates to additional non-dv fields) in all of these tests we should be checking that both the RTG and regualr search results make sense we also need a lot more testing of various deleteById and deleteByQuery commands mixed with in-place atomic updates both deleteByQuerys against the DV fields used in the in-place atomic updates as well as DBQs against other fields in the documents test the results of (uncommited) RTG as well as searches when these deletes are intermixed with in-place atomic updates test the results of peer sync and reply when deletes are mixed with in-place atomic updates. test that we correctly get 409 error codes when trying to do in-place updates w/optimistic concurrency after a delete (and vice versa: DBQ/dID afte in-place update) all logging needs heavily audited there's a lot of logging happening at the info and debug level that should probably be much lower. likewise there may be a few existing info or debug msgs that might be good candidates for warn or error level msgs. uniqueKey and Exception/log msgs there is a lot of log msgs or Exception msgs that cite a version# when reporting a problem, but don't include the uniqueKey of the document involved these messages aren't going to be remotely useful to end users w/o also including the (human readable) uniqueKey of the document involved. it feels like we now have a really large number of methods involved in the merging/combining/converting of documents to apply atomic updates ("in place" or otherwise) ... either for use in RTG, or for use when writing updates to disk, or from reading from the tlog, etc... the ones that jump out at me w/o digging very hard... RealTimeGetComponent.resolveFullDocument RealTimeGetComponent.toSolrDoc RealTimeGetComponent.mergePartialDocWithFullDocFromIndex UpdateLog.applyPartialUpdates UpdateLog.applyOlderUpdate AtomicUpdateDocumentMerger.merge AtomicUpdateDocumentMerger.doInPlaceUpdateMerge i can't help but wonder if there is room for consolidation? in many cases these "merge" methods actually delegate to other "merge" methods, before/after applying additional logic – in which case at a minimum using @link or @see tags in the javadocs to make this (intentional) relationship/similarity more obvious would be helpful. in cases where methods do not delegate to eachother, or have any relationship w/eachother, having @link mentions of eachother in the javadocs to compare/constrast why they are different would be equally helpful. and who knows – perhaps in the process of writing these docs we'll find good oportunities to refactor/consolidate
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          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.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - 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 .
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          New patch:

          1. Addressed most of code level comments from Hoss. I think the readability of the code has now improved and also more robust exception handling. (Thanks Hoss). I shall reply inline to all of your suggestions, perhaps along with my next patch.
          2. Nocommits remain, I think these are all related to Javadocs.
          3. Stress test now has DBQ and DBI. It seems to pass 1000 runs successfully.
          4. PeerSync, TestReplay etc. and a few more unit tests remain.
          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - New patch: Addressed most of code level comments from Hoss. I think the readability of the code has now improved and also more robust exception handling. (Thanks Hoss). I shall reply inline to all of your suggestions, perhaps along with my next patch. Nocommits remain, I think these are all related to Javadocs. Stress test now has DBQ and DBI. It seems to pass 1000 runs successfully. PeerSync, TestReplay etc. and a few more unit tests remain.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -
          1. Added Javadocs, fixed some nocommits
          2. Addressed an issue due to which in-place updating of non-existing DVs was throwing exceptions. For this, it was needed to know which fields have already been added to the index, so that if an update is needed to non-existent DV, then we can resort to a traditional full document atomic update. This check could've been easy if access to IW.globalFieldNumberMap was possible publicly. Instead resorted to checking with the RT searcher's list of DVs, and if field not found there then getting the document from tlog (RTG) and checking if the field exists in that document.
          3. Added some simple tests to PeerSyncTest and TestRecovery for in-place updates.
          4. TODO: A few more tests remain, few nocommits remain (mostly test code related).
          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Added Javadocs, fixed some nocommits Addressed an issue due to which in-place updating of non-existing DVs was throwing exceptions. For this, it was needed to know which fields have already been added to the index, so that if an update is needed to non-existent DV, then we can resort to a traditional full document atomic update. This check could've been easy if access to IW.globalFieldNumberMap was possible publicly. Instead resorted to checking with the RT searcher's list of DVs, and if field not found there then getting the document from tlog (RTG) and checking if the field exists in that document. Added some simple tests to PeerSyncTest and TestRecovery for in-place updates. TODO: A few more tests remain, few nocommits remain (mostly test code related).
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited

          New patch fixing all nocommits. Still a few additional tests, which Hoss mentioned, are TODO. Here's a stab at replying to Hoss' comments (Maybe I'll keep updating this comment itself as and when I fix some of the TODO items here):

          JettySolrRunner
          • javadocs, javadocs, javadocs [FIXED]
          XMLLoader + JavabinLoader
          • why is this param checks logic duplicated in these classes? [Not sure what you mean here, I just set the prevVersion to the cmd here now]
          • why not put this in DUP (which already has access to the request params) when it's doing it's "FROMLEADER" logic? [Since commitWithin and overwrite was being set here, I thought this is an appropriate place to set the prevVersion to the cmd]
          AddUpdateCommand
          • these variables (like all variables) should have javadocs explaining what they are and what they mean [FIXED]
            • people skimming a class shouldn't have to grep the code for a variable name to understand it's purpose
          • having 2 variables here seems like it might be error prone? what does it mean if prevVersion < 0 && isInPlaceUpdate == true ? or 0 < prevVersion && isInPlaceUpdate == false ? [FIXED: Now just have one variable]
            • would it make more sense to use a single long prevVersion variable and have a public boolean isInPlaceUpdate() that simply does {{return (0 < prevVersion); }} ? [FIXED]
          TransactionLog
          • javadocs for both the new write method and the existig write method [FIXED]
            • explain what "prevPointer" means and note in the 2 arg method what the effective default "prevPoint" is.
          • we should really have some "int" constants for refering to the List indexes involved in these records, so instead of code like entry.get(3) sprinkled in various classes like UpdateLog and PeerSync it can be smething more readable like entry.get(PREV_VERSION_IDX) [FIXED]
          UpdateLog
          • javadocs for both the new LogPtr constructure and the existing constructor [FIXED]
            • explain what "prevPointer" means and note in the 2 arg constructure what the effective default "prevPoint" is. [FIXED]
          • add(AddUpdateCommand, boolean)
            • this new code for doing lookups in map, prevMap and preMap2 seems weird to me (but admitedly i'm not really an expert on UpdateLog in general and how these maps are used
            • what primarily concerns me is what the expected behavior is if the "id" isn't found in any of these maps – it looks like prevPointer defaults to "-1" regardless of whether this is an inplace update ... is that intentional? ... is it possible there are older records we will miss and need to flag that? [Yes, this was intentional, and I think it doesn't make any difference. If an "id" isn't found in any of these maps, it would mean that the previous update was committed and should be looked up in the index. ]
            • ie: do we need to worry about distinguising here between "not an in place update, therefore prePointer=-1" vs "is an in place update, but we can't find the prevPointer" ?? [I think we don't need to worry. Upon receiving a prevPointer=-1 by whoever reads this LogPtr, it should be clear why it was -1: if the command's {{flags|UpdateLog.UPDATE_INPLACE}} is set, then this command is an in-place update whose previous update is in the index and not in the tlog; if that flag is not set, it is not an in-place update at all, and don't bother about the prevPointer value at all (which is -1 as a dummy value).]
            • assuming this code is correct, it might be a little easier to read if it were refactored into something like:
              // nocommit: jdocs
              private synchronized long getPrevPointerForUpdate(AddUpdateCommand cmd) {
                // note: sync required to ensure maps aren't changed out form under us
                if (cmd.isInPlaceUpdate) {
                  BytesRef indexedId = cmd.getIndexedId();
                  for (Map<BytesRef,TransactionLog> currentMap : Arrays.asList(map, prevMap, prevMap2)) {
                    LogPtr prevEntry = currentMap.get(indexedId);
                    if (null != prevEntry) {
                      return prevEntry.pointer;
                    }
                  }
                }
                return -1; // default when not inplace, or if we can't find a previous entry
              }
              

              [FIXED: Refactored into something similar to above]

          • applyPartialUpdates
            • it seems like this method would be a really good candidate for some direct unit testing? [Added test to UpdateLogTest]
              • ie: construct a synthetic UpdateLog, and confirm applyPartialUpdates does the right thing
            • the sync block in this method, and how the resulting lookupLogs list is used subsequently, doesn't seem safe to me – particularly the way getEntryFromTLog calls incref/decref on each TransactionLog as it loops over that list...
              • what prevents some other thread from decref'ing one of these TransactionLog objects (and possibly auto-closing it) in between the sync block and the incref in getEntryFromTLog?
                • (most existing usages of TransactionLog.incref() seem to be in blocks that sync on the UpdateLog – and the ones that aren't in sync blocks look sketchy to me as well)
              • in general i'm wondering if lookupLogs should be created outside of the while loop, so that there is a consistent set of "logs" for the duration of the method call ... what happens right now if some other thread changes tlog/prevMapLog/prevMapLog2 in between iterations of the while loop?
            • shouldn't we make some sanity check assertions about the results of getEntryFromTLog? – there's an INVALID_STATE if it's not an ADD or a list of 5 elements, but what about actually asserting that it's either an ADD or an UPDATE_INPLACE? ... [FIXED] what about asserting the doc's uniqueKey matches? [We could do that, but I think it is not necessary]
              • (because unless i'm missing something, it's possible for 2 docs to have the same version, so if there is a glitch in the pointer we can't just trust the version check can we?) [I think we can trust a document to be of the same id if the version matches. It is possible for 2 docs to have same version, but then they would have to be from two different shards altogether. Since all of this processing is happening within a particular replica (which obviously belongs to only one shard), I think we can get away safely without asserting the id and just relying on the version.]
            • partialUpdateEntry seems like a missleading variable name ... can't it be either a full document, or partial update (not in place), or an UPDATE_INPLACE partial update? [FIXED: calling it entry now]
            • if there is only 1 way to break out of this while loop, then the method would probably be easier to make sense of if the applyOlderUpdate and return 0 calls replaced the break statement [FIXED]
            • semi-related: while (true) is generally a red flag: it seems like might be better if it was refactored inot a while (0 <= prevPointer) loop? [FIXED]
          • getEntryFromTLog [FIXED]
            • I don't really see the point of using get(i) over and over .. why not a simple for (TransactionLog log : lookupLogs) ? [FIXED]
            • why is the Exception & Error handling here just a debug log? shouldn't that be a pretty hard fail? [This shouldn't be a hard fail. This is basically a seek operation on a transaction log at the specified position. 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.]
            • as mentioned above re: applyPartialUpdates, isn't is possible for 2 diff docs to have the same version? if we get unlucky and those 2 docs, with identical versions, at the same position in diff TransactionLog files then isn't a sanity check of the doc ID in applyPartialUpdates too late? ... if applyPartialUpdates tries again it's just going to keep getting the same (wrong) document. It seems like this method actaully needs to know the ID it's looking for, and "skip" any entries thta don't match, checking the next (older) TransactionLog [I think this is not a likely scenario, since in a given replica, a version should be unique to a document (I think we have bigger problems if this assumption isn't true).]
          • lookupPartialUpdates
            • what is the purpose/intent of this method? ... it seems to be unused. [FIXED: Removed]
          • doReplay
            • switch statement ordering... [FIXED: I'll add it to my knowledge!]
              • in theory, switch statement cases should be ordered from most likeley to least likely (it's a microoptimization, but in heavily executed loops it might be worth it)
              • so i wouldn't inject UPDATE_INPLACE at the begining of the switch – it should definitely come after ADD, probably best to put it at the end of the list
            • why is entry.get(2) commented out? and why does it say "idBytes" ? ... isn't slot #2 the prevPointer? copy paste confusion from "ADD" ? [FIXED: True, it was copy-paste confusion from ADD. Removed the commented out line.]
              • if slot#2 really isn't needed in this code, get rid of the missleading comment about idBytes and replace it with an explicit comment that prevVersion isn't needed for reply. [FIXED: I have removed the spurious commented out lines, refactored that part into a updateCommandFromTlog() method. Does it address your concern here?]
          PeerSync
          • ditto comments about switch statement ordering from above comments about UpdateLog [FIXED]
          • a lot of code here looks duplicated straight from UpdateLog.doReplay
            • I realize that's true for the other case values as well, but bad code shouldn't be emulated
            • lets refactor this duplicated code into a new public static AddUpdateCommand updateCommandFromTlog(List tlogEntry) method in UpdateLog and re-use it here. [FIXED]
          • log.info looks wrong here ... especially inside of an if (debug) block ... pretty sure this should be log.debug like the other case blocks [FIXED]
          DirectUpdateHandler2
          • I don't really understand why we need any schema/DocValue checks here? [This was unnecessary and I've removed it. I have done a somewhat related refactoring to the AddUpdateCommand.getLuceneDocument(boolean isInPlace) to now only generate a Lucene document that has docValues. This was needed because the lucene document that was originally being returned had copy fields targets of id field, default fields, multiple Field per field (due to FieldType.createFields()) etc., which are not needed for in-place updates.]
          • If cmd.isInPlaceUpdate is true, then doesn't that mean the update is by definition an in place update that only contains values for DV fields? ... wasn't it the responsibility of the upstream code that set that value to true to ensure that? [True, it was. However, copy field targets of id field, default fields etc. were added in this doNormalAdd() method itself due to cmd.getLuceneDocument(); I have overloaded that method to tackle the needs of in-place updates and filter out such unnecessary fields being added to the lucene doc]
            • if not, why can't it be? (ie: why can't we move the schema checks upstream, and out of the sync block here?
          • if that's not what cmd.isInPlaceUpdate means, then why isn't there any error handling here in the event that non-DV field/values are found in the luceneDocument? ... doesn't that mean we need to fall back to the original writer.updateDocument call?
          • If it is neccessary to do SchemaField validation here for some reason, then shouldn't it be the exact same validation done in AtomicUpdateDocumentMerger.isSupportedForInPlaceUpdate ? [FIXED: We should do all schema validation there only, I have removed everything from here, except for some filtering logic at cmd.getLuceneDocument()]
          AtomicUpdateDocumentMerger
          • isSupportedForInPlaceUpdate
            • shouldn't this method either be static or final? the rules don't change if someone subclasses AtomicUpdateDocumentMerger do they?
            • why isn't TrieDateField also valid? ... could this just be checking for instanceof TrieField ? [I wasn't sure how to deal with inc for dates, so left dates out of this for simplicity for now]
              • particularly suspicious since doInPlaceUpdateMerge does in fact check instanceof TrieField [FIXED this and moved this check to isSupportedFieldForInPlaceUpdate()]
            • if the intent is truely to only support "numerics" then, why not instanceof NumericValueFieldType ?
            • shouldn't we also check "not-indexed" and "not-stored" here as well? [This logic was in doInPlaceUpdateMerge in the previous patch; refactored it into the isSupportedFieldForInPlaceUpdate() method now]
          • doInPlaceUpdateMerge
            • why does this method have 3 args? can't all of the neccessary info be deterined from the AddUpdateCommand ? [FIXED]
            • this method seems like another good candidate for some explicit unit testing... [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.]
              • build up an index & tlog with some explicitly crated non trivial docs/updates, then call this method with a variety of inputs and assert the expected modifications to the AddUpdateCommand (or assert no modifications if they aren't viable in place update candaites
              • then hard commit everything in the tlog and assert that all the same calls return the exact same output/modifications.
            • we should probably continue to assume the common case is not to need (in-place) updates ... just regular adds. in which case anything we can do to short circut out faster – before any checks that require stuff like SchemaField – wouldn't reordering the checks in the loop to something like this be equivilent to what we have currently but faster in the common case? ...
              for (SolrInputField f : sdoc) {
                final String fname = f.getName();
                if (idField.getName().equals(fname)) {
                  continue;
                }
                if (! f.getValue() instanceof Map) {
                  // doc contains field value that is not an update, therefore definitely not an inplace update
                  return false;
                }
                if (!isSupportedForInPlaceUpdate(schema.getField(fname))) {
                  // doc contains update to a field that does not support in place updates
                  return false;
                }
              }
              

              [FIXED: Moved it to isInPlaceUpdate() method, though]

              • Even if i've overloked something and that code isn't better, i think in general the "is this sdoc a candidate for in place updating" logic should be refactored into a public static helper method that has some unit tests. [FIXED, moved it to isInPlaceUpdate() method.] [Fixed, added test to TestInPlaceUpdatesCopyField]
            • this method calls RealTimeGetComponent.getInputDocumentFromTlog but doesn't check for the special DELETED return value... [FIXED]
              • definitely smells like a bug ... there are many assumptions made about uncommittedDoc as long as it's non-null
              • based on this, now i really want to see more testing of in place updates mixed with document deletions:
                • some explicit single node testing of "add docX, delete docX, do a 'set':'42' on docX"
                • introduce some randomized deleteById calls into the randomized single/multi node tests
            • this method calls RealTimeGetComponent.getVersionFromTlog which has docs that say "If null is returned, it could still be in the latest index." [The javadocs on that method was stale. The method actually returns from index if not found in tlog]
              • i don't see any code accounting for that possibility, cmd.prevVersion is just blindly assigned the null in that case ... which could lead to an NPE since it's declared as public long prevVersion
              • the fact that this hasn't caused an NPE in testing is a red flag that there is a code path not being tested here ... but at a glance i can't really tell what it is?
            • In general, I guess I don't really understand why this method is jumping through as many hoops as it is with the RTG code? [FIXED: Refactored this to modify the RTGC.getInputDocument to now accept a param whether to avoid stored fields or not. This refactoring is a great improvement in terms of readability!]
              • it seems to duplicate a lot of functionality already in RealTimeGetComponent.getInputDocument ... why not just use that method?
              • if the concern is avoiding the searcher.doc(docid) call to get all stored fields, perhaps RealTimeGetComponent.getInputDocument could be refactored to make it easier to re-use most of the functionality here? ... not sure what would make the most sense off the top of my head.
              • at a minimum it seems like using SolrIndexSearcher.decorateDocValueFields(...) would make more sense then doing it ourselves as we loop over the fields – we can even pass in the explicit list of field names we know we care about based on the SolrInputDocument (or even just the field names we know use "inc" if that's all we really need)
                • (Or is there something i'm missing about why using decorateDocValueFields would be a mistake here?)
            • in the switch statement, shouldn't we be using the doSet and doInc methods to actaully cary out the operations?
              • that would simplify the "inc" case a lot
            • the default on the switch statement looks sketchy to me ... i understand that only "inc" and "set" are supported, but why does this method only warn if it sees something else? shouldn't this be a hard failure? [FIXED]
              • for that matter: shouldn't the instanceof Map check when looping over the fields at the begining of the method short circut out if the Map contains an operation that isn't one of the supported "in place update" operations? [FIXED]
              • in fact: if we pre-checked the Maps only contained "set" and "inc", and used something like decorateDocValueFields (or did the equivilent ourselves in a smaller loop) then couldn't we also simplify this method a lot by just delegating to the existing merge(SolrInputDocument,SolrInputDocument) method? [FIXED]
            • these assumptions seem sketchy, if that's the only reason for these "else" blocks then let's include some asert fieldName.equals(...) calls to prove it...
                        } else { // for version field, which is a docvalue but there's no set/inc operation
                          ...
                        }
                      } else { // for id field
                        ...
              

              [FIXED]

              • in particluar i'm curious about the VERSION_FIELD... [FIXED: I've not added or removed any VERSION_FIELD. If it exists in the cmd.sdoc, it will be copied over to the cmd.partialDoc via the merge() call.]
                • this method is only called from one place – DistributedUpdateProcessor.getUpdatedDocument – and in the existing code of that method, when a SolrInputDocument is fetched from RealTimeGetComponent, the VERSION_FIELD is explicitly removed from it before using it & returning.
                • should this method also be explicitly removing the VERSION_FIELD field? and/or should the caller (getUpdatedDocument) be removing it consistently before returning?
          RealTimeGetComponent
          • process
            • I like this SearcherInfo refactoring, but a few suggestions: [FIXED]
              • it should be promoted into a (private) static (inner) class ... no need for a new class instance every time RealTimeGetComponent.process is called.
              • add a one arg constructor and pass the SolrCore to that.
              • javadocs, javadocs, javadocs .... note that it's not thread safe
              • let's make searcherHolder and searcher private, and replace direct searcher access with:
                public SolrIndexSearcher getSearcher() {
                  assert null != searcher : "init not called!";
                  return searcher;
                }
                
            • in the switch statement, it seems like there is a lot of code duplicated between the ADD and UPDATE_INPLACE cases
              • why 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 ? [FIXED: but kept the call to toSolrDoc() call outside the resolveFullDocument]
          • resolveFullDocument
            • see comments above about modifying this method to call toSolrDocument itself rather then expecting as input, and return early if the entry is an UpdateLog.ADD
            • let's put an assert 0==lastPrevPointer in this else block in case someone improves/breaks ulog.applyPartialUpdates to return -2 in the future...
                  } else { // i.e. lastPrevPointer==0
              

              [FIXED]

            • since ulog.applyPartialUpdates(...) is a No-Op when prevPointer == -1, can't we remove the redundent calls to mergePartialDocWithFullDocFromIndex & reopenRealtimeSearcherAndGet before and after calling ulog.applyPartialUpdates(...) ... ie:
                  long prevPointer = (long) logEntry.get(2);
                  long prevVersion = (long) logEntry.get(3);
                  // this is a No-Op if prevPointer is already negative, otherwise...
                  // get the last full document from ulog
                  prevPointer = ulog.applyPartialUpdates(idBytes, prevPointer, prevVersion, partialDoc);
                  if (-1 == prevPointer) {
                    ...
                  } else if (0 < prevPointer) {
                    ...
                  } else {
                    assert 0 == prevPointer;
                    ...
                  }
              

              [FIXED]

              • If there is some reason i'm not seeing why it's important to call mergePartialDocWithFullDocFromIndex & reopenRealtimeSearcherAndGet before calling ulog.applyPartialUpdates, then perhaps we should at least refactor the "if mergedDoc == null, return reopenRealtimeSearcherAndGet" logic into mergePartialDocWithFullDocFromIndex since that's the only way it's ever used.
          • mergePartialDocWithFullDocFromIndex
            • since this is a private method, what's the expected usage of docidFromIndex ? ... it's never used, so can we refactor it away? [FIXED, removed]
            • see previous comment about refactoring this method to automatically return reopenRealtimeSearcherAndGet(...) when it would otherwise return null [FIXED]
          • reopenRealtimeSearcherAndGet
            • javadocs, javadocs, javadocs [FIXED]
            • since this method is only used in conjunction with mergePartialDocWithFullDocFromIndex, if the code is refactored so that mergePartialDocWithFullDocFromIndex calls this method directly (see suggestion above), we could make a small micro-optimization by changing the method sig to take in a Term to (re)use rather then passing in idBytes and calling core.getLatestSchema().getUniqueKeyField() twice. [FIXED]
            • re: the INVALID_STATE ... is that really a fatal error, or should this method be accounting for the possibility of a doc that has been completley deleted (or was never in the index) in a diff way? [I think this is fatal, since if the doc was deleted, then there shouldn't have been an attempt to resolve to a previous document by that id. I think this should never be triggered.]
          • getInputDocumentFromTlog
            • lets put an explicit comment here noting that we are intentionally falling through to the Updatelog.ADD case [FIXED]
          • getVersionFromTlog
            • based on it's usage, should this method be changed to return -1 instead of null? ... not clear to me from the given caller usage ... if so should be be declared to return long instead of Long ? [I'm inclined to keep it to Long/null instead of long/-1, since versionInfo.getVersionFromIndex() is also Long/null]
          DistributedUpdateProcessor
          • Similar question from AddUpdateCommand: do we really need 2 distinct params here, or would it be cleaner / less error-prone to have a single distrib.inplace.prevversion which indicates we're doing an inplace update if it's a positive # ? [FIXED]
          • versionAdd
            • "Something weird has happened" ... perhaps waitForDependentUpdates should return the lastFoundVersion so we can use it in this error msg? ... "Dependent version not found after waiting: ..." [FIXED]
            • "TODO: Should we call waitForDependentUpdates() again?" ... i don't see how that would help? if we get to this point it definitely seems like a "WTF?" fail fast situation. [FIXED, agreed there's no value to trying again. Removed the comment]
            • the way the existing code in this method has been refactored into an "else" block (see comment: // non inplace update, i.e. full document update) makes sense, but the way the Long lastVersion declaration was refactored out of that block to reuse with the "if isInPlaceUpdate" side of things is a bit confusing and doesn't seem to actaully simplify anything... [FIXED]
              • It's not used after the if/else block, so there's no reason to declare it before the "if" statement
              • by definition it must be null in the "else" case, so if (lastVersion == null) will also be true in that code path
              • it seems simpiler to just let both the "if" and "else" branches declare/define their own "Long lastVersion" and not risk any confusion about why that variable needs to be "shared"
          • waitForDependentUpdates
            • javadocs, javadocs, javadocs [FIXED]
            • fetchFromLeader is always true? why not eliminate arg? [FIXED, this was remnants from a previous patch where I was trying to make this configurable]
            • I'm not a concurrency expert, but is this control flow with the sync/wait actaully safe? ... my understanding was the conditional check you're waiting on should always be a loop inside the sync block? [I consulted Noble on exactly this, and he suggested that this is fine]
            • even if there are no spurious returns from wait, logging at "info" level every 100ms is excessive ... logging that often at trace seems excessive. [FIXED]
              • why don't we log an info once at the start of method (since our of order updates should be rare) and once at debug anytime lastFoundVersion changes? (diff log message if/when lastFoundVersion is == or > prev)
              • the "Waiting attempt" counter "i" doesn't seem like useful info to log given how wait(...) works
            • "...Dropping current update." - that log msg seems missleading, this method doesn't do anything to drop the current update, it just assumes the current update will be droped later [FIXED]
            • i don't really see the point of the boolean foundDependentUpdate variable... why not change the only place where it's set to true to return immediately? [FIXED]
            • fetchMissingUpdateFromLeader can return null, but that possibility isn't handled here. [FIXED]
            • if (uc instanceof AddUpdateCommand) ... what if it's not? [FIXED]
              • currently it's just silently ignored
              • is this a viable scenerio that needs accounted for, or an exceptional scenerio that should have error checking?
              • looks like maybe it's just a confusing way to do a null check?
            • ((System.nanoTime()-startTime)/1000000 ) ... that's a missleading thing to include in the Exception [FIXED]
              • we didn't wait that long, we waited at most 5 seconds – who knows how long we spent calling fetchMissingUpdateFromLeader & executing it.
          • fetchMissingUpdateFromLeader
            • javadocs, javadocs, javadocs [FIXED]
            • should we really be constructing a new HttpSolrClient on the fly like this? [This should be fine, since the underlying HttpClient is created from HttpClientUtil.createClient, and hence security features should all work. However, I've changed this to now use the UpdateShardHandler's httpClient.]
            • is this on the fly SolrClient + GenericSolrRequest going to work if/when the security auth/acl features are in use in the solr cluster?
            • this seems to assume the node that forwarded us the current request (via get(DISTRIB_FROM) is still the leader – but what if there was a leader election? [FIXED: I've now added a check to the cluster state and now fetching from current leader]
              • if the leader failed, and a new one was elected, isn't that a pretty viable/likeley reason why waitForDependentUpdates timed-out and needed to call fetchMissingUpdateFromLeader in the first place?
            • e.printStackTrace(); ... huge red flag that serious error handling is missing here. [FIXED. This one was bad indeed.]
            • This method seems like it expects the possibility that missingUpdates will contain more then one entry, and if it does contain more then one entry it will convert/copy all of them into the updates list – but then it will completley ignore all but the first one. [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.]
              • if we don't expect more then 1, why not assert that?
              • if we expect more then one, and they are all important, why don't we return List<UpdateCommand> ?
              • if we expect more then one, but only care about one in particularly – why loop over all of them?
                • how do we know for sure which one in the list is the one we care about?
            • ditto comments about PeerSync & UpdateLog and creating a public static AddUpdateCommand updateCommandFromTlog(List tlogEntry) somwhere that can be reused here as well [FIXED]
            • switch statement should have some sort of default case ... even if i'ts just to throw an error because anything but an ADD or UPDATE_INPLACE is impossible [FIXED]
            • need to future proof this code against the posibility of other stuff down the road [Not sure what needs to be done more here]

          And here are some general overall comments / impressions I got while reviewing the code and then edited up once i was all finished...

          • Given that this patch requires some non-trivial changes to the types of records that go in the update log, and requires corisponding changes to PeerSync, it seems like there should definitely be some very explicit testing of log reply and peer sync [FIXED]
            • ie: TestReplay and PeerSyncTest should be updated to include a variety of scenerios involving in-place updates
          • after seeing how complex & hairy some of the new code has to be around the diff handling of "in-place atomic updates", vs existing "atomic updates" (that aren't in place) It seems like we should definitely have more test code that mixes and matches diff types of "updates" [Added such a test to TestInPlaceUpdatesStandalone.]
            • static, non randomized, examples of explicit tests we should definitely have...
              • a doc gets a sequence of atomic updates each containing multiple "in place" inc/set ops on diff fields
              • a doc gets a sequence of atomic updates each containing multiple inc/set ops, where a single update may have a mix of "in place" vs "not in place" eligable ops (due to diff fields having diff docvalue/stored settings)
            • our randomized (cloud and non-cloud) testing of in-place updates should also include updates to the canidate docs that may ocasionally not be viable "in-place" updates (because they involved updates to additional non-dv fields)
            • in all of these tests we should be checking that both the RTG and regualr search results make sense
          • we also need a lot more testing of various deleteById and deleteByQuery commands mixed with in-place atomic updates [FIXED]
            • both deleteByQuerys against the DV fields used in the in-place atomic updates as well as DBQs against other fields in the documents
            • test the results of (uncommited) RTG as well as searches when these deletes are intermixed with in-place atomic updates
            • test the results of peer sync and reply when deletes are mixed with in-place atomic updates.
            • test that we correctly get 409 error codes when trying to do in-place updates w/optimistic concurrency after a delete (and vice versa: DBQ/dID afte in-place update)
          • all logging needs heavily audited [FIXED: I've gone through one round of this, but I'll go through again]
            • there's a lot of logging happening at the info and debug level that should probably be much lower.
            • likewise there may be a few existing info or debug msgs that might be good candidates for warn or error level msgs.
          • uniqueKey and Exception/log msgs [FIXED: I've added the id to many places now. I'll go through another round to see if some places are still missing the id field]
            • there is a lot of log msgs or Exception msgs that cite a version# when reporting a problem, but don't include the uniqueKey of the document involved
            • these messages aren't going to be remotely useful to end users w/o also including the (human readable) uniqueKey of the document involved.
          • it feels like we now have a really large number of methods involved in the merging/combining/converting of documents to apply atomic updates ("in place" or otherwise) ... either for use in RTG, or for use when writing updates to disk, or from reading from the tlog, etc... [TODO: I think these methods are in their suitable places for now, but maybe there could be scope for consolidation.]
            • the ones that jump out at me w/o digging very hard...
              RealTimeGetComponent.resolveFullDocument
              RealTimeGetComponent.toSolrDoc
              RealTimeGetComponent.mergePartialDocWithFullDocFromIndex
              UpdateLog.applyPartialUpdates
              UpdateLog.applyOlderUpdate
              AtomicUpdateDocumentMerger.merge
              AtomicUpdateDocumentMerger.doInPlaceUpdateMerge
              
            • i can't help but wonder if there is room for consolidation?
            • in many cases these "merge" methods actually delegate to other "merge" methods, before/after applying additional logic – in which case at a minimum using @link or @see tags in the javadocs to make this (intentional) relationship/similarity more obvious would be helpful.
            • in cases where methods do not delegate to eachother, or have any relationship w/eachother, having @link mentions of eachother in the javadocs to compare/constrast why they are different would be equally helpful.
            • and who knows – perhaps in the process of writing these docs we'll find good oportunities to refactor/consolidate
          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited New patch fixing all nocommits. Still a few additional tests, which Hoss mentioned, are TODO. Here's a stab at replying to Hoss' comments (Maybe I'll keep updating this comment itself as and when I fix some of the TODO items here): JettySolrRunner javadocs, javadocs, javadocs [FIXED] XMLLoader + JavabinLoader why is this param checks logic duplicated in these classes? [Not sure what you mean here, I just set the prevVersion to the cmd here now] why not put this in DUP (which already has access to the request params) when it's doing it's "FROMLEADER" logic? [Since commitWithin and overwrite was being set here, I thought this is an appropriate place to set the prevVersion to the cmd] AddUpdateCommand these variables (like all variables) should have javadocs explaining what they are and what they mean [FIXED] people skimming a class shouldn't have to grep the code for a variable name to understand it's purpose having 2 variables here seems like it might be error prone? what does it mean if prevVersion < 0 && isInPlaceUpdate == true ? or 0 < prevVersion && isInPlaceUpdate == false ? [FIXED: Now just have one variable] would it make more sense to use a single long prevVersion variable and have a public boolean isInPlaceUpdate() that simply does {{return (0 < prevVersion); }} ? [FIXED] TransactionLog javadocs for both the new write method and the existig write method [FIXED] explain what "prevPointer" means and note in the 2 arg method what the effective default "prevPoint" is. we should really have some "int" constants for refering to the List indexes involved in these records, so instead of code like entry.get(3) sprinkled in various classes like UpdateLog and PeerSync it can be smething more readable like entry.get(PREV_VERSION_IDX) [FIXED] UpdateLog javadocs for both the new LogPtr constructure and the existing constructor [FIXED] explain what "prevPointer" means and note in the 2 arg constructure what the effective default "prevPoint" is. [FIXED] add(AddUpdateCommand, boolean) this new code for doing lookups in map , prevMap and preMap2 seems weird to me (but admitedly i'm not really an expert on UpdateLog in general and how these maps are used what primarily concerns me is what the expected behavior is if the "id" isn't found in any of these maps – it looks like prevPointer defaults to "-1" regardless of whether this is an inplace update ... is that intentional? ... is it possible there are older records we will miss and need to flag that? [Yes, this was intentional, and I think it doesn't make any difference. If an "id" isn't found in any of these maps, it would mean that the previous update was committed and should be looked up in the index. ] ie: do we need to worry about distinguising here between "not an in place update, therefore prePointer=-1" vs "is an in place update, but we can't find the prevPointer" ?? [I think we don't need to worry. Upon receiving a prevPointer=-1 by whoever reads this LogPtr, it should be clear why it was -1: if the command's {{flags|UpdateLog.UPDATE_INPLACE}} is set, then this command is an in-place update whose previous update is in the index and not in the tlog; if that flag is not set, it is not an in-place update at all, and don't bother about the prevPointer value at all (which is -1 as a dummy value).] assuming this code is correct, it might be a little easier to read if it were refactored into something like: // nocommit: jdocs private synchronized long getPrevPointerForUpdate(AddUpdateCommand cmd) { // note: sync required to ensure maps aren't changed out form under us if (cmd.isInPlaceUpdate) { BytesRef indexedId = cmd.getIndexedId(); for (Map<BytesRef,TransactionLog> currentMap : Arrays.asList(map, prevMap, prevMap2)) { LogPtr prevEntry = currentMap.get(indexedId); if ( null != prevEntry) { return prevEntry.pointer; } } } return -1; // default when not inplace, or if we can't find a previous entry } [FIXED: Refactored into something similar to above] applyPartialUpdates it seems like this method would be a really good candidate for some direct unit testing? [Added test to UpdateLogTest] ie: construct a synthetic UpdateLog, and confirm applyPartialUpdates does the right thing the sync block in this method, and how the resulting lookupLogs list is used subsequently, doesn't seem safe to me – particularly the way getEntryFromTLog calls incref/decref on each TransactionLog as it loops over that list... what prevents some other thread from decref'ing one of these TransactionLog objects (and possibly auto-closing it) in between the sync block and the incref in getEntryFromTLog? (most existing usages of TransactionLog.incref() seem to be in blocks that sync on the UpdateLog – and the ones that aren't in sync blocks look sketchy to me as well) in general i'm wondering if lookupLogs should be created outside of the while loop, so that there is a consistent set of "logs" for the duration of the method call ... what happens right now if some other thread changes tlog/prevMapLog/prevMapLog2 in between iterations of the while loop? shouldn't we make some sanity check assertions about the results of getEntryFromTLog? – there's an INVALID_STATE if it's not an ADD or a list of 5 elements, but what about actually asserting that it's either an ADD or an UPDATE_INPLACE? ... [FIXED] what about asserting the doc's uniqueKey matches? [We could do that, but I think it is not necessary] (because unless i'm missing something, it's possible for 2 docs to have the same version, so if there is a glitch in the pointer we can't just trust the version check can we?) [I think we can trust a document to be of the same id if the version matches. It is possible for 2 docs to have same version, but then they would have to be from two different shards altogether. Since all of this processing is happening within a particular replica (which obviously belongs to only one shard), I think we can get away safely without asserting the id and just relying on the version.] partialUpdateEntry seems like a missleading variable name ... can't it be either a full document, or partial update (not in place), or an UPDATE_INPLACE partial update? [FIXED: calling it entry now] if there is only 1 way to break out of this while loop, then the method would probably be easier to make sense of if the applyOlderUpdate and return 0 calls replaced the break statement [FIXED] semi-related: while (true) is generally a red flag: it seems like might be better if it was refactored inot a while (0 <= prevPointer) loop? [FIXED] getEntryFromTLog [FIXED] I don't really see the point of using get(i) over and over .. why not a simple for (TransactionLog log : lookupLogs) ? [FIXED] why is the Exception & Error handling here just a debug log? shouldn't that be a pretty hard fail? [This shouldn't be a hard fail. This is basically a seek operation on a transaction log at the specified position. 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.] as mentioned above re: applyPartialUpdates , isn't is possible for 2 diff docs to have the same version? if we get unlucky and those 2 docs, with identical versions, at the same position in diff TransactionLog files then isn't a sanity check of the doc ID in applyPartialUpdates too late? ... if applyPartialUpdates tries again it's just going to keep getting the same (wrong) document. It seems like this method actaully needs to know the ID it's looking for, and "skip" any entries thta don't match, checking the next (older) TransactionLog [I think this is not a likely scenario, since in a given replica, a version should be unique to a document (I think we have bigger problems if this assumption isn't true).] lookupPartialUpdates what is the purpose/intent of this method? ... it seems to be unused. [FIXED: Removed] doReplay switch statement ordering... [FIXED: I'll add it to my knowledge!] in theory, switch statement cases should be ordered from most likeley to least likely (it's a microoptimization, but in heavily executed loops it might be worth it) so i wouldn't inject UPDATE_INPLACE at the begining of the switch – it should definitely come after ADD, probably best to put it at the end of the list why is entry.get(2) commented out? and why does it say "idBytes" ? ... isn't slot #2 the prevPointer? copy paste confusion from "ADD" ? [FIXED: True, it was copy-paste confusion from ADD. Removed the commented out line.] if slot#2 really isn't needed in this code, get rid of the missleading comment about idBytes and replace it with an explicit comment that prevVersion isn't needed for reply. [FIXED: I have removed the spurious commented out lines, refactored that part into a updateCommandFromTlog() method. Does it address your concern here?] PeerSync ditto comments about switch statement ordering from above comments about UpdateLog [FIXED] a lot of code here looks duplicated straight from UpdateLog.doReplay I realize that's true for the other case values as well, but bad code shouldn't be emulated lets refactor this duplicated code into a new public static AddUpdateCommand updateCommandFromTlog(List tlogEntry) method in UpdateLog and re-use it here. [FIXED] log.info looks wrong here ... especially inside of an if (debug) block ... pretty sure this should be log.debug like the other case blocks [FIXED] DirectUpdateHandler2 I don't really understand why we need any schema/DocValue checks here? [This was unnecessary and I've removed it. I have done a somewhat related refactoring to the AddUpdateCommand.getLuceneDocument(boolean isInPlace) to now only generate a Lucene document that has docValues. This was needed because the lucene document that was originally being returned had copy fields targets of id field, default fields, multiple Field per field (due to FieldType.createFields()) etc., which are not needed for in-place updates.] If cmd.isInPlaceUpdate is true, then doesn't that mean the update is by definition an in place update that only contains values for DV fields? ... wasn't it the responsibility of the upstream code that set that value to true to ensure that? [True, it was. However, copy field targets of id field, default fields etc. were added in this doNormalAdd() method itself due to cmd.getLuceneDocument(); I have overloaded that method to tackle the needs of in-place updates and filter out such unnecessary fields being added to the lucene doc] if not, why can't it be? (ie: why can't we move the schema checks upstream, and out of the sync block here? if that's not what cmd.isInPlaceUpdate means, then why isn't there any error handling here in the event that non-DV field/values are found in the luceneDocument? ... doesn't that mean we need to fall back to the original writer.updateDocument call? If it is neccessary to do SchemaField validation here for some reason, then shouldn't it be the exact same validation done in AtomicUpdateDocumentMerger.isSupportedForInPlaceUpdate ? [FIXED: We should do all schema validation there only, I have removed everything from here, except for some filtering logic at cmd.getLuceneDocument()] AtomicUpdateDocumentMerger isSupportedForInPlaceUpdate shouldn't this method either be static or final? the rules don't change if someone subclasses AtomicUpdateDocumentMerger do they? why isn't TrieDateField also valid? ... could this just be checking for instanceof TrieField ? [I wasn't sure how to deal with inc for dates, so left dates out of this for simplicity for now] particularly suspicious since doInPlaceUpdateMerge does in fact check instanceof TrieField [FIXED this and moved this check to isSupportedFieldForInPlaceUpdate()] if the intent is truely to only support "numerics" then, why not instanceof NumericValueFieldType ? shouldn't we also check "not-indexed" and "not-stored" here as well? [This logic was in doInPlaceUpdateMerge in the previous patch; refactored it into the isSupportedFieldForInPlaceUpdate() method now] doInPlaceUpdateMerge why does this method have 3 args? can't all of the neccessary info be deterined from the AddUpdateCommand ? [FIXED] this method seems like another good candidate for some explicit unit testing... [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.] build up an index & tlog with some explicitly crated non trivial docs/updates, then call this method with a variety of inputs and assert the expected modifications to the AddUpdateCommand (or assert no modifications if they aren't viable in place update candaites then hard commit everything in the tlog and assert that all the same calls return the exact same output/modifications. we should probably continue to assume the common case is not to need (in-place) updates ... just regular adds. in which case anything we can do to short circut out faster – before any checks that require stuff like SchemaField – wouldn't reordering the checks in the loop to something like this be equivilent to what we have currently but faster in the common case? ... for (SolrInputField f : sdoc) { final String fname = f.getName(); if (idField.getName().equals(fname)) { continue ; } if (! f.getValue() instanceof Map) { // doc contains field value that is not an update, therefore definitely not an inplace update return false ; } if (!isSupportedForInPlaceUpdate(schema.getField(fname))) { // doc contains update to a field that does not support in place updates return false ; } } [FIXED: Moved it to isInPlaceUpdate() method, though] Even if i've overloked something and that code isn't better, i think in general the "is this sdoc a candidate for in place updating" logic should be refactored into a public static helper method that has some unit tests. [FIXED, moved it to isInPlaceUpdate() method.] [Fixed, added test to TestInPlaceUpdatesCopyField] this method calls RealTimeGetComponent.getInputDocumentFromTlog but doesn't check for the special DELETED return value... [FIXED] definitely smells like a bug ... there are many assumptions made about uncommittedDoc as long as it's non-null based on this, now i really want to see more testing of in place updates mixed with document deletions: some explicit single node testing of "add docX, delete docX, do a 'set':'42' on docX" introduce some randomized deleteById calls into the randomized single/multi node tests this method calls RealTimeGetComponent.getVersionFromTlog which has docs that say "If null is returned, it could still be in the latest index." [The javadocs on that method was stale. The method actually returns from index if not found in tlog] i don't see any code accounting for that possibility, cmd.prevVersion is just blindly assigned the null in that case ... which could lead to an NPE since it's declared as public long prevVersion the fact that this hasn't caused an NPE in testing is a red flag that there is a code path not being tested here ... but at a glance i can't really tell what it is? In general, I guess I don't really understand why this method is jumping through as many hoops as it is with the RTG code? [FIXED: Refactored this to modify the RTGC.getInputDocument to now accept a param whether to avoid stored fields or not. This refactoring is a great improvement in terms of readability!] it seems to duplicate a lot of functionality already in RealTimeGetComponent.getInputDocument ... why not just use that method? if the concern is avoiding the searcher.doc(docid) call to get all stored fields, perhaps RealTimeGetComponent.getInputDocument could be refactored to make it easier to re-use most of the functionality here? ... not sure what would make the most sense off the top of my head. at a minimum it seems like using SolrIndexSearcher.decorateDocValueFields(...) would make more sense then doing it ourselves as we loop over the fields – we can even pass in the explicit list of field names we know we care about based on the SolrInputDocument (or even just the field names we know use "inc" if that's all we really need) (Or is there something i'm missing about why using decorateDocValueFields would be a mistake here?) in the switch statement, shouldn't we be using the doSet and doInc methods to actaully cary out the operations? that would simplify the "inc" case a lot the default on the switch statement looks sketchy to me ... i understand that only "inc" and "set" are supported, but why does this method only warn if it sees something else? shouldn't this be a hard failure? [FIXED] for that matter: shouldn't the instanceof Map check when looping over the fields at the begining of the method short circut out if the Map contains an operation that isn't one of the supported "in place update" operations? [FIXED] in fact: if we pre-checked the Maps only contained "set" and "inc", and used something like decorateDocValueFields (or did the equivilent ourselves in a smaller loop) then couldn't we also simplify this method a lot by just delegating to the existing merge(SolrInputDocument,SolrInputDocument) method? [FIXED] these assumptions seem sketchy, if that's the only reason for these "else" blocks then let's include some asert fieldName.equals(...) calls to prove it... } else { // for version field, which is a docvalue but there's no set/inc operation ... } } else { // for id field ... [FIXED] in particluar i'm curious about the VERSION_FIELD ... [FIXED: I've not added or removed any VERSION_FIELD. If it exists in the cmd.sdoc, it will be copied over to the cmd.partialDoc via the merge() call.] this method is only called from one place – DistributedUpdateProcessor.getUpdatedDocument – and in the existing code of that method, when a SolrInputDocument is fetched from RealTimeGetComponent , the VERSION_FIELD is explicitly removed from it before using it & returning. should this method also be explicitly removing the VERSION_FIELD field? and/or should the caller ( getUpdatedDocument ) be removing it consistently before returning? RealTimeGetComponent process I like this SearcherInfo refactoring, but a few suggestions: [FIXED] it should be promoted into a (private) static (inner) class ... no need for a new class instance every time RealTimeGetComponent.process is called. add a one arg constructor and pass the SolrCore to that. javadocs, javadocs, javadocs .... note that it's not thread safe let's make searcherHolder and searcher private, and replace direct searcher access with: public SolrIndexSearcher getSearcher() { assert null != searcher : "init not called!" ; return searcher; } in the switch statement, it seems like there is a lot of code duplicated between the ADD and UPDATE_INPLACE cases why 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 ? [FIXED: but kept the call to toSolrDoc() call outside the resolveFullDocument] resolveFullDocument see comments above about modifying this method to call toSolrDocument itself rather then expecting as input, and return early if the entry is an UpdateLog.ADD let's put an assert 0==lastPrevPointer in this else block in case someone improves/breaks ulog.applyPartialUpdates to return -2 in the future... } else { // i.e. lastPrevPointer==0 [FIXED] since ulog.applyPartialUpdates(...) is a No-Op when prevPointer == -1 , can't we remove the redundent calls to mergePartialDocWithFullDocFromIndex & reopenRealtimeSearcherAndGet before and after calling ulog.applyPartialUpdates(...) ... ie: long prevPointer = ( long ) logEntry.get(2); long prevVersion = ( long ) logEntry.get(3); // this is a No-Op if prevPointer is already negative, otherwise... // get the last full document from ulog prevPointer = ulog.applyPartialUpdates(idBytes, prevPointer, prevVersion, partialDoc); if (-1 == prevPointer) { ... } else if (0 < prevPointer) { ... } else { assert 0 == prevPointer; ... } [FIXED] If there is some reason i'm not seeing why it's important to call mergePartialDocWithFullDocFromIndex & reopenRealtimeSearcherAndGet before calling ulog.applyPartialUpdates , then perhaps we should at least refactor the "if mergedDoc == null, return reopenRealtimeSearcherAndGet" logic into mergePartialDocWithFullDocFromIndex since that's the only way it's ever used. mergePartialDocWithFullDocFromIndex since this is a private method, what's the expected usage of docidFromIndex ? ... it's never used, so can we refactor it away? [FIXED, removed] see previous comment about refactoring this method to automatically return reopenRealtimeSearcherAndGet(...) when it would otherwise return null [FIXED] reopenRealtimeSearcherAndGet javadocs, javadocs, javadocs [FIXED] since this method is only used in conjunction with mergePartialDocWithFullDocFromIndex , if the code is refactored so that mergePartialDocWithFullDocFromIndex calls this method directly (see suggestion above), we could make a small micro-optimization by changing the method sig to take in a Term to (re)use rather then passing in idBytes and calling core.getLatestSchema().getUniqueKeyField() twice. [FIXED] re: the INVALID_STATE ... is that really a fatal error, or should this method be accounting for the possibility of a doc that has been completley deleted (or was never in the index) in a diff way? [I think this is fatal, since if the doc was deleted, then there shouldn't have been an attempt to resolve to a previous document by that id. I think this should never be triggered.] getInputDocumentFromTlog lets put an explicit comment here noting that we are intentionally falling through to the Updatelog.ADD case [FIXED] getVersionFromTlog based on it's usage, should this method be changed to return -1 instead of null? ... not clear to me from the given caller usage ... if so should be be declared to return long instead of Long ? [I'm inclined to keep it to Long/null instead of long/-1, since versionInfo.getVersionFromIndex() is also Long/null] DistributedUpdateProcessor Similar question from AddUpdateCommand: do we really need 2 distinct params here, or would it be cleaner / less error-prone to have a single distrib.inplace.prevversion which indicates we're doing an inplace update if it's a positive # ? [FIXED] versionAdd "Something weird has happened" ... perhaps waitForDependentUpdates should return the lastFoundVersion so we can use it in this error msg? ... "Dependent version not found after waiting: ..." [FIXED] "TODO: Should we call waitForDependentUpdates() again?" ... i don't see how that would help? if we get to this point it definitely seems like a "WTF?" fail fast situation. [FIXED, agreed there's no value to trying again. Removed the comment] the way the existing code in this method has been refactored into an "else" block (see comment: // non inplace update, i.e. full document update ) makes sense, but the way the Long lastVersion declaration was refactored out of that block to reuse with the "if isInPlaceUpdate" side of things is a bit confusing and doesn't seem to actaully simplify anything... [FIXED] It's not used after the if/else block, so there's no reason to declare it before the "if" statement by definition it must be null in the "else" case, so if (lastVersion == null) will also be true in that code path it seems simpiler to just let both the "if" and "else" branches declare/define their own "Long lastVersion" and not risk any confusion about why that variable needs to be "shared" waitForDependentUpdates javadocs, javadocs, javadocs [FIXED] fetchFromLeader is always true? why not eliminate arg? [FIXED, this was remnants from a previous patch where I was trying to make this configurable] I'm not a concurrency expert, but is this control flow with the sync/wait actaully safe? ... my understanding was the conditional check you're waiting on should always be a loop inside the sync block? [I consulted Noble on exactly this, and he suggested that this is fine] even if there are no spurious returns from wait, logging at "info" level every 100ms is excessive ... logging that often at trace seems excessive. [FIXED] why don't we log an info once at the start of method (since our of order updates should be rare) and once at debug anytime lastFoundVersion changes? (diff log message if/when lastFoundVersion is == or > prev) the "Waiting attempt" counter "i" doesn't seem like useful info to log given how wait(...) works "...Dropping current update." - that log msg seems missleading, this method doesn't do anything to drop the current update, it just assumes the current update will be droped later [FIXED] i don't really see the point of the boolean foundDependentUpdate variable... why not change the only place where it's set to true to return immediately? [FIXED] fetchMissingUpdateFromLeader can return null, but that possibility isn't handled here. [FIXED] if (uc instanceof AddUpdateCommand) ... what if it's not? [FIXED] currently it's just silently ignored is this a viable scenerio that needs accounted for, or an exceptional scenerio that should have error checking? looks like maybe it's just a confusing way to do a null check? ((System.nanoTime()-startTime)/1000000 ) ... that's a missleading thing to include in the Exception [FIXED] we didn't wait that long, we waited at most 5 seconds – who knows how long we spent calling fetchMissingUpdateFromLeader & executing it. fetchMissingUpdateFromLeader javadocs, javadocs, javadocs [FIXED] should we really be constructing a new HttpSolrClient on the fly like this? [This should be fine, since the underlying HttpClient is created from HttpClientUtil.createClient, and hence security features should all work. However, I've changed this to now use the UpdateShardHandler's httpClient.] is this on the fly SolrClient + GenericSolrRequest going to work if/when the security auth/acl features are in use in the solr cluster? this seems to assume the node that forwarded us the current request (via get(DISTRIB_FROM) is still the leader – but what if there was a leader election? [FIXED: I've now added a check to the cluster state and now fetching from current leader] if the leader failed, and a new one was elected, isn't that a pretty viable/likeley reason why waitForDependentUpdates timed-out and needed to call fetchMissingUpdateFromLeader in the first place? e.printStackTrace(); ... huge red flag that serious error handling is missing here. [FIXED. This one was bad indeed.] This method seems like it expects the possibility that missingUpdates will contain more then one entry, and if it does contain more then one entry it will convert/copy all of them into the updates list – but then it will completley ignore all but the first one. [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.] if we don't expect more then 1, why not assert that? if we expect more then one, and they are all important, why don't we return List<UpdateCommand> ? if we expect more then one, but only care about one in particularly – why loop over all of them? how do we know for sure which one in the list is the one we care about? ditto comments about PeerSync & UpdateLog and creating a public static AddUpdateCommand updateCommandFromTlog(List tlogEntry) somwhere that can be reused here as well [FIXED] switch statement should have some sort of default case ... even if i'ts just to throw an error because anything but an ADD or UPDATE_INPLACE is impossible [FIXED] need to future proof this code against the posibility of other stuff down the road [Not sure what needs to be done more here] And here are some general overall comments / impressions I got while reviewing the code and then edited up once i was all finished... Given that this patch requires some non-trivial changes to the types of records that go in the update log, and requires corisponding changes to PeerSync , it seems like there should definitely be some very explicit testing of log reply and peer sync [FIXED] ie: TestReplay and PeerSyncTest should be updated to include a variety of scenerios involving in-place updates after seeing how complex & hairy some of the new code has to be around the diff handling of "in-place atomic updates", vs existing "atomic updates" (that aren't in place) It seems like we should definitely have more test code that mixes and matches diff types of "updates" [Added such a test to TestInPlaceUpdatesStandalone.] static, non randomized, examples of explicit tests we should definitely have... a doc gets a sequence of atomic updates each containing multiple "in place" inc/set ops on diff fields a doc gets a sequence of atomic updates each containing multiple inc/set ops, where a single update may have a mix of "in place" vs "not in place" eligable ops (due to diff fields having diff docvalue/stored settings) our randomized (cloud and non-cloud) testing of in-place updates should also include updates to the canidate docs that may ocasionally not be viable "in-place" updates (because they involved updates to additional non-dv fields) in all of these tests we should be checking that both the RTG and regualr search results make sense we also need a lot more testing of various deleteById and deleteByQuery commands mixed with in-place atomic updates [FIXED] both deleteByQuerys against the DV fields used in the in-place atomic updates as well as DBQs against other fields in the documents test the results of (uncommited) RTG as well as searches when these deletes are intermixed with in-place atomic updates test the results of peer sync and reply when deletes are mixed with in-place atomic updates. test that we correctly get 409 error codes when trying to do in-place updates w/optimistic concurrency after a delete (and vice versa: DBQ/dID afte in-place update) all logging needs heavily audited [FIXED: I've gone through one round of this, but I'll go through again] there's a lot of logging happening at the info and debug level that should probably be much lower. likewise there may be a few existing info or debug msgs that might be good candidates for warn or error level msgs. uniqueKey and Exception/log msgs [FIXED: I've added the id to many places now. I'll go through another round to see if some places are still missing the id field] there is a lot of log msgs or Exception msgs that cite a version# when reporting a problem, but don't include the uniqueKey of the document involved these messages aren't going to be remotely useful to end users w/o also including the (human readable) uniqueKey of the document involved. it feels like we now have a really large number of methods involved in the merging/combining/converting of documents to apply atomic updates ("in place" or otherwise) ... either for use in RTG, or for use when writing updates to disk, or from reading from the tlog, etc... [TODO: I think these methods are in their suitable places for now, but maybe there could be scope for consolidation.] the ones that jump out at me w/o digging very hard... RealTimeGetComponent.resolveFullDocument RealTimeGetComponent.toSolrDoc RealTimeGetComponent.mergePartialDocWithFullDocFromIndex UpdateLog.applyPartialUpdates UpdateLog.applyOlderUpdate AtomicUpdateDocumentMerger.merge AtomicUpdateDocumentMerger.doInPlaceUpdateMerge i can't help but wonder if there is room for consolidation? in many cases these "merge" methods actually delegate to other "merge" methods, before/after applying additional logic – in which case at a minimum using @link or @see tags in the javadocs to make this (intentional) relationship/similarity more obvious would be helpful. in cases where methods do not delegate to eachother, or have any relationship w/eachother, having @link mentions of eachother in the javadocs to compare/constrast why they are different would be equally helpful. and who knows – perhaps in the process of writing these docs we'll find good oportunities to refactor/consolidate
          Hide
          hossman Hoss Man added a comment -

          I've not had a chance to look at the latest patch, but here's some comment responses...

          Since commitWithin and overwrite was being set here, I thought this is an appropriate place to set the prevVersion to the cmd

          But there's a fundemental difference betwen params like commitWithin and overwrite and the new prevVersion param...

          commitWithin and overwrite are client specified options specific to the xml/javabin update format(s). The fact that they can be specified as request params is an implementation detail of the xml/javabin formats that they happen to have in common, but are not exclusively specifyied as params – for example the XMLLoader only uses the params as defaults, they can be psecified on a per <add/> basis.

          The new prevVersion param however is an implementation detail of DUP ... DUP is the only code that should have to know/care that prevVersion comes from a request param.

          Yes, this was intentional, and I think it doesn't make any difference. If an "id" isn't found in any of these maps, it would mean that the previous update was committed and should be looked up in the index.

          I think we don't need to worry. Upon receiving a prevPointer=-1 by whoever reads this LogPtr, it should be clear why it was -1: if the command's flags|UpdateLog.UPDATE_INPLACE is set, then this command is an in-place update whose previous update is in the index and not in the tlog; if that flag is not set, it is not an in-place update at all, and don't bother about the prevPointer value at all (which is -1 as a dummy value).

          We should have a comment to these affects (literally we could just paste that text directly into a comment) when declaring the prevPointer variable in this method.

          ... This was needed because the lucene document that was originally being returned had copy fields targets of id field, default fields, multiple Field per field (due to FieldType.createFields()) etc., which are not needed for in-place updates.

          Hmm... that makes me wonder – we should make sure we have a test case of doing atomic updates on numeric dv fields which have copyfields to other numeric fields. ie: lets make sure our "is this a candidate for inplace updates" takes into acount that the value being updated might need copied to another field.

          (in theory if both the source & dest of the copy field are single valued dv only then we can still do the in place updated as long as the copyField happens, but even if we don't have that extra bit of logic we need a test that the udpates are happening consistently)

          I wasn't sure how to deal with inc for dates, so left dates out of this for simplicity for now

          Hmmm... is that really the relevant question though?

          I'm not sure how the existing (non-inplace) atomic update code behaves if you try to "inc" a date, but why does it matter for the isSupportedForInPlaceUpdate method?

          • if date "inc" is supported in the existing atomic update code, then whatever that code path looks like (to compute the new value) it should be the same for the new inplace update code.
          • if date "inc" is not supported in the existing atomic update code, then whatever the error is should be the same in the new inplace update code

          Either way, I don't see why isSupportedForInPlaceUpdate should care – or if it is going to care, then it should care about the details (ie: return false for (dv only) date field w/ "inc", but true for (dv only) date field with "set")

          I think this is fatal, since if the doc was deleted, then there shouldn't have been an attempt to resolve to a previous document by that id. I think this should never be triggered.

          let's put those details in a comment where this Exception is thrown ... or better yet, try to incorporate it into the Exception msg?

          I'm inclined to keep it to Long/null instead of long/-1, since versionInfo.getVersionFromIndex() is also Long/null

          Ah, ok ... good point – can we go ahead and add some javadocs to that method as well making that clear?

          ... I've changed this to now use the UpdateShardHandler's httpClient.

          Ok, cool ... Yeah, that probably makes more sense in general.

          Not sure what needs to be done more here

          Yeah, sorry – that was a vague comment that even i don't know what i ment by, was probably ment to be part of the note about the switch statemenet default.

          Show
          hossman Hoss Man added a comment - I've not had a chance to look at the latest patch, but here's some comment responses... Since commitWithin and overwrite was being set here, I thought this is an appropriate place to set the prevVersion to the cmd But there's a fundemental difference betwen params like commitWithin and overwrite and the new prevVersion param... commitWithin and overwrite are client specified options specific to the xml/javabin update format(s). The fact that they can be specified as request params is an implementation detail of the xml/javabin formats that they happen to have in common, but are not exclusively specifyied as params – for example the XMLLoader only uses the params as defaults, they can be psecified on a per <add/> basis. The new prevVersion param however is an implementation detail of DUP ... DUP is the only code that should have to know/care that prevVersion comes from a request param. Yes, this was intentional, and I think it doesn't make any difference. If an "id" isn't found in any of these maps, it would mean that the previous update was committed and should be looked up in the index. I think we don't need to worry. Upon receiving a prevPointer=-1 by whoever reads this LogPtr, it should be clear why it was -1: if the command's flags|UpdateLog.UPDATE_INPLACE is set, then this command is an in-place update whose previous update is in the index and not in the tlog; if that flag is not set, it is not an in-place update at all, and don't bother about the prevPointer value at all (which is -1 as a dummy value). We should have a comment to these affects (literally we could just paste that text directly into a comment) when declaring the prevPointer variable in this method. ... This was needed because the lucene document that was originally being returned had copy fields targets of id field, default fields, multiple Field per field (due to FieldType.createFields()) etc., which are not needed for in-place updates. Hmm... that makes me wonder – we should make sure we have a test case of doing atomic updates on numeric dv fields which have copyfields to other numeric fields. ie: lets make sure our "is this a candidate for inplace updates" takes into acount that the value being updated might need copied to another field. (in theory if both the source & dest of the copy field are single valued dv only then we can still do the in place updated as long as the copyField happens, but even if we don't have that extra bit of logic we need a test that the udpates are happening consistently) I wasn't sure how to deal with inc for dates, so left dates out of this for simplicity for now Hmmm... is that really the relevant question though? I'm not sure how the existing (non-inplace) atomic update code behaves if you try to "inc" a date, but why does it matter for the isSupportedForInPlaceUpdate method? if date "inc" is supported in the existing atomic update code, then whatever that code path looks like (to compute the new value) it should be the same for the new inplace update code. if date "inc" is not supported in the existing atomic update code, then whatever the error is should be the same in the new inplace update code Either way, I don't see why isSupportedForInPlaceUpdate should care – or if it is going to care, then it should care about the details (ie: return false for (dv only) date field w/ "inc", but true for (dv only) date field with "set") I think this is fatal, since if the doc was deleted, then there shouldn't have been an attempt to resolve to a previous document by that id. I think this should never be triggered. let's put those details in a comment where this Exception is thrown ... or better yet, try to incorporate it into the Exception msg? I'm inclined to keep it to Long/null instead of long/-1, since versionInfo.getVersionFromIndex() is also Long/null Ah, ok ... good point – can we go ahead and add some javadocs to that method as well making that clear? ... I've changed this to now use the UpdateShardHandler's httpClient. Ok, cool ... Yeah, that probably makes more sense in general. Not sure what needs to be done more here Yeah, sorry – that was a vague comment that even i don't know what i ment by, was probably ment to be part of the note about the switch statemenet default.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited

          But there's a fundemental difference betwen params like commitWithin and overwrite and the new prevVersion param...

          commitWithin and overwrite are client specified options specific to the xml/javabin update format(s). The fact that they can be specified as request params is an implementation detail of the xml/javabin formats that they happen to have in common, but are not exclusively specifyied as params – for example the XMLLoader only uses the params as defaults, they can be psecified on a per <add/> basis.

          The new prevVersion param however is an implementation detail of DUP ... DUP is the only code that should have to know/care that prevVersion comes from a request param.

          Sure, it makes sense. I'll fix it.

          We should have a comment to these affects (literally we could just paste that text directly into a comment) when declaring the prevPointer variable in this method.

          I had put this comment there:

          @return If cmd is an in-place update, then returns the pointer (in the tlog) of the previous update that the given update depends on. Returns -1 if this is not an in-place update, or if we can't find a previous entry in the tlog.
          

          But now I have updated it to make it even more detailed:

          @return If cmd is an in-place update, then returns the pointer (in the tlog) of the previous update that the given update depends on.
             *        Returns -1 if this is not an in-place update, or if we can't find a previous entry in the tlog. Upon receiving a -1, it 
             *        should be clear why it was -1: if the command's flags|UpdateLog.UPDATE_INPLACE is set, then this
             *        command is an in-place update whose previous update is in the index and not in the tlog; if that flag is not set, it is not an in-place
             *        update at all, and don't bother about the prevPointer value at all (which is -1 as a dummy value).)
          

          Hmm... that makes me wonder – we should make sure we have a test case of doing atomic updates on numeric dv fields which have copyfields to other numeric fields. ie: lets make sure our "is this a candidate for inplace updates" takes into acount that the value being updated might need copied to another field.

          (in theory if both the source & dest of the copy field are single valued dv only then we can still do the in place updated as long as the copyField happens, but even if we don't have that extra bit of logic we need a test that the udpates are happening consistently)

          Sure, I'll add such a test. The latest patch incorporates the behaviour you suggested: if any of the copy field targets is not a in-place updateable field, then the entire operation is not an in-place update (but a traditional atomic update instead). But, if copy field targets of an updated field is also supported for an updateable dv, then it is updated as well.

          Hmmm... is that really the relevant question though?

          I'm not sure how the existing (non-inplace) atomic update code behaves if you try to "inc" a date, but why does it matter for the isSupportedForInPlaceUpdate method?

          if date "inc" is supported in the existing atomic update code, then whatever that code path looks like (to compute the new value) it should be the same for the new inplace update code.
          if date "inc" is not supported in the existing atomic update code, then whatever the error is should be the same in the new inplace update code

          Either way, I don't see why isSupportedForInPlaceUpdate should care – or if it is going to care, then it should care about the details (ie: return false for (dv only) date field w/ "inc", but true for (dv only) date field with "set")

          For now I've removed date field totally out of scope of this patch. If there is a update to date that is needed, it falls back to traditional atomic update. I can try to deal with the trie date field, if you suggest.

          let's put those details in a comment where this Exception is thrown ... or better yet, try to incorporate it into the Exception msg?

          I had put this exception in the patch:
          Unable to resolve the last full doc in tlog fully, and document not found in index even after opening new rt searcher.
          but now I'll change it to:
          Unable to resolve the last full doc in tlog fully, and document not found in index even after opening new rt searcher. If the doc was deleted, then there shouldn't have been an attempt to resolve to a previous document by that id.

          Ah, ok ... good point – can we go ahead and add some javadocs to that method as well making that clear?

          Sure, I'll update the javadocs for that existing method as well.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - - edited But there's a fundemental difference betwen params like commitWithin and overwrite and the new prevVersion param... commitWithin and overwrite are client specified options specific to the xml/javabin update format(s). The fact that they can be specified as request params is an implementation detail of the xml/javabin formats that they happen to have in common, but are not exclusively specifyied as params – for example the XMLLoader only uses the params as defaults, they can be psecified on a per <add/> basis. The new prevVersion param however is an implementation detail of DUP ... DUP is the only code that should have to know/care that prevVersion comes from a request param. Sure, it makes sense. I'll fix it. We should have a comment to these affects (literally we could just paste that text directly into a comment) when declaring the prevPointer variable in this method. I had put this comment there: @ return If cmd is an in-place update, then returns the pointer (in the tlog) of the previous update that the given update depends on. Returns -1 if this is not an in-place update, or if we can't find a previous entry in the tlog. But now I have updated it to make it even more detailed: @ return If cmd is an in-place update, then returns the pointer (in the tlog) of the previous update that the given update depends on. * Returns -1 if this is not an in-place update, or if we can't find a previous entry in the tlog. Upon receiving a -1, it * should be clear why it was -1: if the command's flags|UpdateLog.UPDATE_INPLACE is set, then this * command is an in-place update whose previous update is in the index and not in the tlog; if that flag is not set, it is not an in-place * update at all, and don't bother about the prevPointer value at all (which is -1 as a dummy value).) Hmm... that makes me wonder – we should make sure we have a test case of doing atomic updates on numeric dv fields which have copyfields to other numeric fields. ie: lets make sure our "is this a candidate for inplace updates" takes into acount that the value being updated might need copied to another field. (in theory if both the source & dest of the copy field are single valued dv only then we can still do the in place updated as long as the copyField happens, but even if we don't have that extra bit of logic we need a test that the udpates are happening consistently) Sure, I'll add such a test. The latest patch incorporates the behaviour you suggested: if any of the copy field targets is not a in-place updateable field, then the entire operation is not an in-place update (but a traditional atomic update instead). But, if copy field targets of an updated field is also supported for an updateable dv, then it is updated as well. Hmmm... is that really the relevant question though? I'm not sure how the existing (non-inplace) atomic update code behaves if you try to "inc" a date, but why does it matter for the isSupportedForInPlaceUpdate method? if date "inc" is supported in the existing atomic update code, then whatever that code path looks like (to compute the new value) it should be the same for the new inplace update code. if date "inc" is not supported in the existing atomic update code, then whatever the error is should be the same in the new inplace update code Either way, I don't see why isSupportedForInPlaceUpdate should care – or if it is going to care, then it should care about the details (ie: return false for (dv only) date field w/ "inc", but true for (dv only) date field with "set") For now I've removed date field totally out of scope of this patch. If there is a update to date that is needed, it falls back to traditional atomic update. I can try to deal with the trie date field, if you suggest. let's put those details in a comment where this Exception is thrown ... or better yet, try to incorporate it into the Exception msg? I had put this exception in the patch: Unable to resolve the last full doc in tlog fully, and document not found in index even after opening new rt searcher. but now I'll change it to: Unable to resolve the last full doc in tlog fully, and document not found in index even after opening new rt searcher. If the doc was deleted, then there shouldn't have been an attempt to resolve to a previous document by that id. Ah, ok ... good point – can we go ahead and add some javadocs to that method as well making that clear? Sure, I'll update the javadocs for that existing method as well.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Updated patch:

          1. Refactored the logic to add a previous version to an AddUpdateCommand, moved it to DUP from JavabinLoader/XMLLoader
          2. Updated javadocs to make them more detailed. Added javadocs to some related existing methods in VersionInfo.
          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Updated patch: Refactored the logic to add a previous version to an AddUpdateCommand, moved it to DUP from JavabinLoader/XMLLoader Updated javadocs to make them more detailed. Added javadocs to some related existing methods in VersionInfo.
          Hide
          shalinmangar Shalin Shekhar Mangar added a comment -

          Patch updated to master after resolving conflicts with the SOLR-9285 commit.

          Show
          shalinmangar Shalin Shekhar Mangar added a comment - Patch updated to master after resolving conflicts with the SOLR-9285 commit.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Thanks Shalin. Updated the patch to include a test for the UpdateLog's applyPartialUpdates method. Also, removed a possibly erroneous optimization from DUP that I introduced in my last patch (an if check for lastDependentVersion == -1 inside the synchronization block).

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Thanks Shalin. Updated the patch to include a test for the UpdateLog's applyPartialUpdates method. Also, removed a possibly erroneous optimization from DUP that I introduced in my last patch (an if check for lastDependentVersion == -1 inside the synchronization block).
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Updated the patch:

          1. Fixed a bug with mixing atomic and in-place updates. Problem was that after in-place update, the RTGC.getInputDocument() got only the partial document, and hence further atomic updates on it failed. Changed this to return a "resolved" document for use during atomic update.
          2. Added direct unit tests for AUDM.isInPlaceUpdate() at TestInPlaceUpdatesCopyField.java and UpdateLogTest. applyPartialUpdates() at UpdateLogTest.java.
          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Updated the patch: Fixed a bug with mixing atomic and in-place updates. Problem was that after in-place update, the RTGC.getInputDocument() got only the partial document, and hence further atomic updates on it failed. Changed this to return a "resolved" document for use during atomic update. Added direct unit tests for AUDM.isInPlaceUpdate() at TestInPlaceUpdatesCopyField.java and UpdateLogTest. applyPartialUpdates() at UpdateLogTest.java.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Another TODO item got missed out.

          1. Refactored calls like entry.get(1) etc. (for entries fetched from the ulog/tlog) to entry.get(UpdateLog.VERSION_IDX).
          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Another TODO item got missed out. Refactored calls like entry.get(1) etc. (for entries fetched from the ulog/tlog) to entry.get(UpdateLog.VERSION_IDX).
          Hide
          shalinmangar Shalin Shekhar Mangar added a comment - - edited

          Thanks Ishan. This is great progress since the last time I reviewed this patch.

          I've only skimmed the latest patch but in particular I find a few problems in DistributedUpdateProcessor#waitForDependentUpdates:

          1. This method doesn't look correct. All places which call vinfo.lookupVersion() do it after acquiring a read lock by calling vinfo.lockForUpdate(). Looking at the code for vinfo.lookupVersion() it accesses the map and prevMap in updateLog which can be modified by a writer thread while waitForDependentUpdates is reading their values. So first of all we need to ensure that we acquire and release the read lock. Acquiring this lock and then waiting on a different object (the "bucket") will not cause a deadlock condition because it is a read lock (which can be held by multiple threads).
          2. Secondly, this method can be made more efficient. It currently wakes up every 100ms and reads the new "lastFoundVersion" from the update log or index. This is wasteful. A better way would be to wait for the timeout period directly after calling vinfo.lookupVersion() inside the synchronized block.
          3. Similar to #1 – calling vinfo.lookupVersion() after fetchMissingUpdateFromLeader should be done after acquiring a read lock.
          4. There is no reason to synchronize on bucket when calling the versionAdd method again because it will acquire the monitor anyway.
          5. DistributedUpdateProcessor#waitForDependentUpdates uses wrong javadoc tag '@returns' instead of '@return'
          6. The debug log message should be moved out of the loop instead of introducing a debugMessagePrinted boolean flag
          7. Use the org.apache.solr.util.TimeOut class for timed wait loops
          8. Method can be made private

          I've attempted to write a better wait-loop here (warning: not tested):

          long prev = cmd.prevVersion;
              long lastFoundVersion = 0;
          
          
              TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS);
              vinfo.lockForUpdate();
              try {
                synchronized (bucket) {
                  lastFoundVersion = vinfo.lookupVersion(cmd.getIndexedId());
                  while (lastFoundVersion < prev && !timeOut.hasTimedOut())  {
                    if (log.isDebugEnabled()) {
                      log.debug("Re-ordered inplace update. version=" + (cmd.getVersion() == 0 ? versionOnUpdate : cmd.getVersion()) +
                          ", prevVersion=" + prev + ", lastVersion=" + lastFoundVersion + ", replayOrPeerSync=" + isReplayOrPeersync);
                    }
                    try {
                      bucket.wait(5000);
                    } catch (InterruptedException ie) {
                      throw new RuntimeException(ie);
                    }
                    lastFoundVersion = vinfo.lookupVersion(cmd.getIndexedId());
                  }
                }
              } finally {
                vinfo.unlockForUpdate();
              }
          
          // check lastFoundVersion against prev again and handle all conditions
          

          However I think that since the read lock and bucket monitor has to be acquired by this method anyway, it might be a good idea to just call it from inside versionAdd after acquiring those monitors. Then this method can focus on just waiting for dependent updates and nothing else.

          A random comment on the changes made to DebugFilter: The setDelay mechanism introduced here may be a good candidate for Mark's new TestInjection#injectUpdateRandomPause?

          Show
          shalinmangar Shalin Shekhar Mangar added a comment - - edited Thanks Ishan. This is great progress since the last time I reviewed this patch. I've only skimmed the latest patch but in particular I find a few problems in DistributedUpdateProcessor#waitForDependentUpdates: This method doesn't look correct. All places which call vinfo.lookupVersion() do it after acquiring a read lock by calling vinfo.lockForUpdate(). Looking at the code for vinfo.lookupVersion() it accesses the map and prevMap in updateLog which can be modified by a writer thread while waitForDependentUpdates is reading their values. So first of all we need to ensure that we acquire and release the read lock. Acquiring this lock and then waiting on a different object (the "bucket") will not cause a deadlock condition because it is a read lock (which can be held by multiple threads). Secondly, this method can be made more efficient. It currently wakes up every 100ms and reads the new "lastFoundVersion" from the update log or index. This is wasteful. A better way would be to wait for the timeout period directly after calling vinfo.lookupVersion() inside the synchronized block. Similar to #1 – calling vinfo.lookupVersion() after fetchMissingUpdateFromLeader should be done after acquiring a read lock. There is no reason to synchronize on bucket when calling the versionAdd method again because it will acquire the monitor anyway. DistributedUpdateProcessor#waitForDependentUpdates uses wrong javadoc tag '@returns' instead of '@return' The debug log message should be moved out of the loop instead of introducing a debugMessagePrinted boolean flag Use the org.apache.solr.util.TimeOut class for timed wait loops Method can be made private I've attempted to write a better wait-loop here (warning: not tested): long prev = cmd.prevVersion; long lastFoundVersion = 0; TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS); vinfo.lockForUpdate(); try { synchronized (bucket) { lastFoundVersion = vinfo.lookupVersion(cmd.getIndexedId()); while (lastFoundVersion < prev && !timeOut.hasTimedOut()) { if (log.isDebugEnabled()) { log.debug( "Re-ordered inplace update. version=" + (cmd.getVersion() == 0 ? versionOnUpdate : cmd.getVersion()) + ", prevVersion=" + prev + ", lastVersion=" + lastFoundVersion + ", replayOrPeerSync=" + isReplayOrPeersync); } try { bucket.wait(5000); } catch (InterruptedException ie) { throw new RuntimeException(ie); } lastFoundVersion = vinfo.lookupVersion(cmd.getIndexedId()); } } } finally { vinfo.unlockForUpdate(); } // check lastFoundVersion against prev again and handle all conditions However I think that since the read lock and bucket monitor has to be acquired by this method anyway, it might be a good idea to just call it from inside versionAdd after acquiring those monitors. Then this method can focus on just waiting for dependent updates and nothing else. A random comment on the changes made to DebugFilter: The setDelay mechanism introduced here may be a good candidate for Mark's new TestInjection#injectUpdateRandomPause?
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Thanks Shalin, I've fixed it in my updated patch, mostly on the lines of what you suggested in that snippet. Can you please take a look?

          Btw, I've surrounded only the lookupVersion() calls with the acquiring and releasing of the lock, instead of surrounding the entire wait loop with the acquiring/releasing of the lock: I reasoned that while we are waiting in that wait loop, other threads need to have indexed the update that we're waiting on, and hence I released the lock as soon as it was not needed, only to re-acquire it after 100ms. Does that sound like a valid reason?

          Secondly, this method can be made more efficient. It currently wakes up every 100ms and reads the new "lastFoundVersion" from the update log or index. This is wasteful. A better way would be to wait for the timeout period directly after calling vinfo.lookupVersion() inside the synchronized block.

          Since this method enters the wait loop for every in-place update that has arrived out of order at a replica (an event, that I think is frequent under multithreaded load), I don't want every such update to be waiting for the full timeout period (5s here), but instead check back from time to time. In most of the cases, the dependent update would've been written (by another thread) within the first 100ms, after which we can bail out. Do you think that makes sense?

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Thanks Shalin, I've fixed it in my updated patch, mostly on the lines of what you suggested in that snippet. Can you please take a look? Btw, I've surrounded only the lookupVersion() calls with the acquiring and releasing of the lock, instead of surrounding the entire wait loop with the acquiring/releasing of the lock: I reasoned that while we are waiting in that wait loop, other threads need to have indexed the update that we're waiting on, and hence I released the lock as soon as it was not needed, only to re-acquire it after 100ms. Does that sound like a valid reason? Secondly, this method can be made more efficient. It currently wakes up every 100ms and reads the new "lastFoundVersion" from the update log or index. This is wasteful. A better way would be to wait for the timeout period directly after calling vinfo.lookupVersion() inside the synchronized block. Since this method enters the wait loop for every in-place update that has arrived out of order at a replica (an event, that I think is frequent under multithreaded load), I don't want every such update to be waiting for the full timeout period (5s here), but instead check back from time to time. In most of the cases, the dependent update would've been written (by another thread) within the first 100ms, after which we can bail out. Do you think that makes sense?
          Hide
          shalinmangar Shalin Shekhar Mangar added a comment -

          Btw, I've surrounded only the lookupVersion() calls with the acquiring and releasing of the lock, instead of surrounding the entire wait loop with the acquiring/releasing of the lock: I reasoned that while we are waiting in that wait loop, other threads need to have indexed the update that we're waiting on, and hence I released the lock as soon as it was not needed, only to re-acquire it after 100ms. Does that sound like a valid reason?

          The read lock is for safe publication of fields in update log and it is acquired by indexing threads who only want to read stuff from update log. Also read locks can be held by multiple readers. Therefore, acquiring this lock does not prevent other threads from indexing.

          Also, please be very careful when changing the order of acquiring locks because it can result in deadlocks. It is good practice to do them in the same sequence as everywhere else in the code. So synchronizing on bucket before vinfo.lockForUpdate for a small optimization doesn't seem worthwhile to me.

          Since this method enters the wait loop for every in-place update that has arrived out of order at a replica (an event, that I think is frequent under multithreaded load), I don't want every such update to be waiting for the full timeout period (5s here), but instead check back from time to time. In most of the cases, the dependent update would've been written (by another thread) within the first 100ms, after which we can bail out. Do you think that makes sense?

          You misunderstand. A wait(5000) does not mean that you are waiting the full 5 seconds. Any notifyAll() will wake up the waiting thread and when it does, it will check the lastFoundVersion and proceed accordingly. In practice wait(100) may not be so bad but if an update doesn't arrive for more than 100ms the thread will wake up and lookup the version needlessly with your current patch.

          A few more comments:

          1. In your latest patch, acquiring the read lock to call versionAdd is not necessary – it will do that anyway. You can re-acquire it for reading the version after the method call returns.
          2. I don't think the case of vinfo.lookupVersion returning a negative value (for deletes) is handled here at all.
          3. What happens if the document has been deleted already (due to reordering on the replica) when you enter waitForDependentUpdates? i.e. what if re-ordering leads to new_doc (v1) -> del_doc (v3) -> dv_update (v2) on the replica?
          4. Similarly, how do we handle the case when the doc has been deleted on the leader when you execute fetchMissingUpdateFromLeader. Does RTG return the requested version even if the doc has been deleted already? I suspect it does but be nice to confirm.
          Show
          shalinmangar Shalin Shekhar Mangar added a comment - Btw, I've surrounded only the lookupVersion() calls with the acquiring and releasing of the lock, instead of surrounding the entire wait loop with the acquiring/releasing of the lock: I reasoned that while we are waiting in that wait loop, other threads need to have indexed the update that we're waiting on, and hence I released the lock as soon as it was not needed, only to re-acquire it after 100ms. Does that sound like a valid reason? The read lock is for safe publication of fields in update log and it is acquired by indexing threads who only want to read stuff from update log. Also read locks can be held by multiple readers. Therefore, acquiring this lock does not prevent other threads from indexing. Also, please be very careful when changing the order of acquiring locks because it can result in deadlocks. It is good practice to do them in the same sequence as everywhere else in the code. So synchronizing on bucket before vinfo.lockForUpdate for a small optimization doesn't seem worthwhile to me. Since this method enters the wait loop for every in-place update that has arrived out of order at a replica (an event, that I think is frequent under multithreaded load), I don't want every such update to be waiting for the full timeout period (5s here), but instead check back from time to time. In most of the cases, the dependent update would've been written (by another thread) within the first 100ms, after which we can bail out. Do you think that makes sense? You misunderstand. A wait(5000) does not mean that you are waiting the full 5 seconds. Any notifyAll() will wake up the waiting thread and when it does, it will check the lastFoundVersion and proceed accordingly. In practice wait(100) may not be so bad but if an update doesn't arrive for more than 100ms the thread will wake up and lookup the version needlessly with your current patch. A few more comments: In your latest patch, acquiring the read lock to call versionAdd is not necessary – it will do that anyway. You can re-acquire it for reading the version after the method call returns. I don't think the case of vinfo.lookupVersion returning a negative value (for deletes) is handled here at all. What happens if the document has been deleted already (due to reordering on the replica) when you enter waitForDependentUpdates? i.e. what if re-ordering leads to new_doc (v1) -> del_doc (v3) -> dv_update (v2) on the replica? Similarly, how do we handle the case when the doc has been deleted on the leader when you execute fetchMissingUpdateFromLeader. Does RTG return the requested version even if the doc has been deleted already? I suspect it does but be nice to confirm.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          it is acquired by indexing threads who only want to read stuff from update log

          Ah, I see. That makes it clear for me; I've updated the patch now and have done the wait loop as per your suggestion.

          In your latest patch, acquiring the read lock to call versionAdd is not necessary – it will do that anyway. You can re-acquire it for reading the version after the method call returns.

          Done, removed.

          I don't think the case of vinfo.lookupVersion returning a negative value (for deletes) is handled here at all.

          Indeed, I hadn't handled it! I've now handled it in this new patch.

          What happens if the document has been deleted already (due to reordering on the replica) when you enter waitForDependentUpdates? i.e. what if re-ordering leads to new_doc (v1) -> del_doc (v3) -> dv_update (v2) on the replica?

          I think it should now be fixed: v2 update would be dropped since abs(v3) > v2 (earlier I was just checking v3 > v2, which didn't consider negative versions). I'll add a test for this.

          Similarly, how do we handle the case when the doc has been deleted on the leader when you execute fetchMissingUpdateFromLeader. Does RTG return the requested version even if the doc has been deleted already? I suspect it does but be nice to confirm.

          I think as long as a particular update for the given version is present at the leader's ulog/tlog, it will be returned. I will add a test for this scenario specifically anyway.

          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - it is acquired by indexing threads who only want to read stuff from update log Ah, I see. That makes it clear for me; I've updated the patch now and have done the wait loop as per your suggestion. In your latest patch, acquiring the read lock to call versionAdd is not necessary – it will do that anyway. You can re-acquire it for reading the version after the method call returns. Done, removed. I don't think the case of vinfo.lookupVersion returning a negative value (for deletes) is handled here at all. Indeed, I hadn't handled it! I've now handled it in this new patch. What happens if the document has been deleted already (due to reordering on the replica) when you enter waitForDependentUpdates? i.e. what if re-ordering leads to new_doc (v1) -> del_doc (v3) -> dv_update (v2) on the replica? I think it should now be fixed: v2 update would be dropped since abs(v3) > v2 (earlier I was just checking v3 > v2, which didn't consider negative versions). I'll add a test for this. Similarly, how do we handle the case when the doc has been deleted on the leader when you execute fetchMissingUpdateFromLeader. Does RTG return the requested version even if the doc has been deleted already? I suspect it does but be nice to confirm. I think as long as a particular update for the given version is present at the leader's ulog/tlog, it will be returned. I will add a test for this scenario specifically anyway.
          Hide
          ichattopadhyaya Ishan Chattopadhyaya added a comment -

          Added tests for the two scenarios described in the previous comment.

          1. TestInPlaceUpdatesDistrib.outOfOrderDeleteUpdatesIndividualReplicaTest() for the scenario where in-place update arrives after a delete.
          2. Added another scenario to TestInPlaceUpdatesDistrib.delayedReorderingFetchesMissingUpdateFromLeaderTest() where a missing update needs to be fetched from leader, even though the document itself has been deleted at the leader.
          Show
          ichattopadhyaya Ishan Chattopadhyaya added a comment - Added tests for the two scenarios described in the previous comment. TestInPlaceUpdatesDistrib.outOfOrderDeleteUpdatesIndividualReplicaTest() for the scenario where in-place update arrives after a delete. Added another scenario to TestInPlaceUpdatesDistrib.delayedReorderingFetchesMissingUpdateFromLeaderTest() where a missing update needs to be fetched from leader, even though the document itself has been deleted at the leader.
          Hide
          hossman Hoss Man added a comment -

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

          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)
              • 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);)
          • fetchMissingUpdateFromLeader
            • In response to a previous question you said...

              [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.]

              ... 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 ?)
          AtomicUpdateDocumentMerger
          • isSupportedFieldForInPlaceUpdate
            • javadocs
          • getFieldNamesFromIndex
            • javadocs
            • method name seems VERY missleading considering what it does
          • 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?
            • 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) ?
            • if (indexFields.contains(fieldName) == false && schema.isDynamicField(fieldName))
              • why does it matter one way or the other if it's a dynamicField?
            • the special DELETED sentinal value still isn't being checked against the return value of getInputDocumentFromTlog
            • 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).
              • My suggested rewrite would be something like...
                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;
                
          • 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(...)
              • either that, or make isInPlaceUpdate include VERSION_FIELD, and have this method assert that it's there.
            • if (docid >= 0)
              • 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?
              • 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
              • 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:

              [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.]

              ... 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.
          RealTimeGetComponent
          • SearcherInfo
            • still has no javadocs
          • resolveFullDocument
            • I made this suggestion regarding simplifying the callers of this method:

              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 ?

              . your response was...

              [FIXED: but kept the call to toSolrDoc() call outside the resolveFullDocument]

              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?
          • reopenRealtimeSearcherAndGet
            • 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?
              • the 2 arg method should almost certainly delgate to getInputDocument(core,idByes,false,null,true ... see comments below regarding DocBasedVersionConstraintsProcessorFactory
          DocBasedVersionConstraintsProcessorFactory
          • replacing getInputDocument(core,id with getInputDocument(core,id,false smells wrong here
            • 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)
          DocumentBuilder
          • method javadocs should explain how the forInPlaceUpdate param(s) will be used (ie: why does it matter?)
          • 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?)
            • 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)
            • 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?)
            • in general it seems like this method should just be something much simpler that like...
              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)
              }
              
          • 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
            • 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.
          AddUpdateCommand
          • prevVersion
            • why can't this comment be a javadoc comment?
            • comment/javadoc should explain what special values (like -1) mean
          • getLuceneDocument(boolean)
            • javadocs
          • getLuceneDocument()
            • javadocs should be updated to indicate (default) behavior compared to one arg version
          SolrDocumentBase
          • why does containsKey need declared here? it's already part of Map interface
          • should be able to drop this file from patch
          XMLLoader
          • whitespace only changes ... drop from patch.
          DirectUpdateHandler2
          • WTF: printStackTrace
          • 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?
            • 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?
          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
          • 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?
          • getEntryFromTLog
            • in repsonse to a question i had about the catch (Exception | Error ex) block, you said...

              ..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.

              ...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?
          • 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?
            • 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)
          TransactionLog
          • jdocs for the two write methods seem backwards
            • 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)))
          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.
          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)
          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...
              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
              ...
              
          • 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...
                SolrException ex = expectThrows(SolrException.class, () -> {
                  returnVal = ulog.applyPartialUpdates(cmd.getIndexedId(), prevPointer, prevVersion, partialDoc);
                });
                // TODO: assert stuff about ex.code, ex.getMessage()
                
            • 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.
          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?)
          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?)
          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...
                • 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...
                ... 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...
                long version1 = addAndGetVersion(sdoc("id", "1", "title_s", "first"), null);
                ...
                // Check docValues were "set"
                version1 = addAndAssertVersion(0, "id", "1", "ratings", map("set", 200));
                
            • 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...
                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+"']"
                        );
                
              • 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...
              version1 = addAndAssertVersion(0, "id", "1", "ratings", map("inc", 66), "price", map("inc", 44));
              
          • 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...
                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"
                
          • 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.
          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...
            • ...(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
          • this block reads very awkwardly...
            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);
            
            • wouldn't something like this be functionally equivilent and easier to make sense of?
              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);
              
          • this looks like dead code ... or is something actaully still missing and needs done? ...
            } else if (oper < commitPercent + deletePercent + deleteByQueryPercent) {
              // TODO
            } else {
            
          • 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) ...
            •    [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)
              
            •    [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)
              
          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...
            •    [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)
              
            •    [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)
              
          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...
            • 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...
                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
                  }
                }
                
              • 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...
                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());
                
            • 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
          • 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:
                  • 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
                • 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...
              assertEquals("RTG tlog price: " + sdoc, (int) 100, sdoc.get("price"));
              
            • 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...
              for (SolrClient client: new SolrClient\[\] \{LEADER, NONLEADERS.get(0), 
                  NONLEADERS.get(1)}) { // nonleader 0 re-ordered replica, nonleader 1 well-ordered replica
              
          • 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?
          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
          General Questions / Concerns
          • precommit was failing for me due to javadoc warnings
          • LUCENE-7344
            • previous comment from Ishan...

              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.

            • 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?)
          Show
          hossman Hoss Man added a comment - Ok – it took a while, but here's my notes after reviewing the latest patch.... 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) 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); ) fetchMissingUpdateFromLeader In response to a previous question you said... [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.] ... 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 ?) AtomicUpdateDocumentMerger isSupportedFieldForInPlaceUpdate javadocs getFieldNamesFromIndex javadocs method name seems VERY missleading consid