You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2020/08/14 16:32:32 UTC

[GitHub] [lucene-solr] cpoerschke opened a new pull request #1754: SOLR-14700: add StreamingTest.testCloudSolrStreamWithoutStreamContext() test

cpoerschke opened a new pull request #1754:
URL: https://github.com/apache/lucene-solr/pull/1754


   https://issues.apache.org/jira/browse/SOLR-14700


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [lucene-solr] cpoerschke commented on a change in pull request #1754: SOLR-14700: avoid NPEs in TupleStream.getShards() when streamContext … …is null.

Posted by GitBox <gi...@apache.org>.
cpoerschke commented on a change in pull request #1754:
URL: https://github.com/apache/lucene-solr/pull/1754#discussion_r471370237



##########
File path: solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
##########
@@ -141,19 +141,30 @@ public UUID getStreamNodeId(){
       shards = shardsMap.get(collection);
     } else {
       //SolrCloud Sharding
-      CloudSolrClient cloudSolrClient =
-          Optional.ofNullable(streamContext.getSolrClientCache()).orElseGet(SolrClientCache::new).getCloudSolrClient(zkHost);
+      SolrClientCache solrClientCache = (streamContext != null ? streamContext.getSolrClientCache() : null);
+      final SolrClientCache localSolrClientCache;
+      if (solrClientCache == null) {
+        solrClientCache = localSolrClientCache = new SolrClientCache();
+      } else {
+        localSolrClientCache = null;
+      }
+      CloudSolrClient cloudSolrClient = solrClientCache.getCloudSolrClient(zkHost);
       ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
       ClusterState clusterState = zkStateReader.getClusterState();
       Slice[] slices = CloudSolrStream.getSlices(collection, zkStateReader, true);
       Set<String> liveNodes = clusterState.getLiveNodes();
 
 
-      ModifiableSolrParams solrParams = new ModifiableSolrParams(streamContext.getRequestParams());
+      final ModifiableSolrParams solrParams;
+      if (streamContext != null) {
+        solrParams = new ModifiableSolrParams(streamContext.getRequestParams());
+      } else {
+        solrParams = new ModifiableSolrParams();
+      }
       solrParams.add(requestParams);
 
       RequestReplicaListTransformerGenerator requestReplicaListTransformerGenerator =
-          Optional.ofNullable(streamContext.getRequestReplicaListTransformerGenerator()).orElseGet(RequestReplicaListTransformerGenerator::new);
+          Optional.ofNullable(streamContext != null ? streamContext.getRequestReplicaListTransformerGenerator() : null).orElseGet(RequestReplicaListTransformerGenerator::new);

Review comment:
       Good question. I just tried to fold the logic into the `streamContext != null` above but then the unit tests failed and (rightly so) identified that `streamContext.getRequestReplicaListTransformerGenerator()` can return `null`. We could still remove use of the `Optional` of course but it would require an additional `if (requestReplicaListTransformerGenerator == null)` block then. Thought?

##########
File path: solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
##########
@@ -141,19 +141,30 @@ public UUID getStreamNodeId(){
       shards = shardsMap.get(collection);
     } else {
       //SolrCloud Sharding
-      CloudSolrClient cloudSolrClient =
-          Optional.ofNullable(streamContext.getSolrClientCache()).orElseGet(SolrClientCache::new).getCloudSolrClient(zkHost);
+      SolrClientCache solrClientCache = (streamContext != null ? streamContext.getSolrClientCache() : null);

Review comment:
       It's possible for `streamContext.getSolrClientCache()` to return `null` and in that case combining the ternary and the conditional would give a different result. Would it help to have a comment re: that subtlety perhaps?

##########
File path: solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
##########
@@ -141,19 +141,30 @@ public UUID getStreamNodeId(){
       shards = shardsMap.get(collection);
     } else {
       //SolrCloud Sharding
-      CloudSolrClient cloudSolrClient =
-          Optional.ofNullable(streamContext.getSolrClientCache()).orElseGet(SolrClientCache::new).getCloudSolrClient(zkHost);
+      SolrClientCache solrClientCache = (streamContext != null ? streamContext.getSolrClientCache() : null);
+      final SolrClientCache localSolrClientCache;
+      if (solrClientCache == null) {
+        solrClientCache = localSolrClientCache = new SolrClientCache();

Review comment:
       Good idea, will do.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [lucene-solr] madrob commented on a change in pull request #1754: SOLR-14700: avoid NPEs in TupleStream.getShards() when streamContext … …is null.

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #1754:
URL: https://github.com/apache/lucene-solr/pull/1754#discussion_r471546425



##########
File path: solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
##########
@@ -141,19 +141,30 @@ public UUID getStreamNodeId(){
       shards = shardsMap.get(collection);
     } else {
       //SolrCloud Sharding
-      CloudSolrClient cloudSolrClient =
-          Optional.ofNullable(streamContext.getSolrClientCache()).orElseGet(SolrClientCache::new).getCloudSolrClient(zkHost);
+      SolrClientCache solrClientCache = (streamContext != null ? streamContext.getSolrClientCache() : null);
+      final SolrClientCache localSolrClientCache;
+      if (solrClientCache == null) {
+        solrClientCache = localSolrClientCache = new SolrClientCache();
+      } else {
+        localSolrClientCache = null;
+      }
+      CloudSolrClient cloudSolrClient = solrClientCache.getCloudSolrClient(zkHost);
       ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
       ClusterState clusterState = zkStateReader.getClusterState();
       Slice[] slices = CloudSolrStream.getSlices(collection, zkStateReader, true);
       Set<String> liveNodes = clusterState.getLiveNodes();
 
 
-      ModifiableSolrParams solrParams = new ModifiableSolrParams(streamContext.getRequestParams());
+      final ModifiableSolrParams solrParams;
+      if (streamContext != null) {
+        solrParams = new ModifiableSolrParams(streamContext.getRequestParams());
+      } else {
+        solrParams = new ModifiableSolrParams();
+      }
       solrParams.add(requestParams);
 
       RequestReplicaListTransformerGenerator requestReplicaListTransformerGenerator =
-          Optional.ofNullable(streamContext.getRequestReplicaListTransformerGenerator()).orElseGet(RequestReplicaListTransformerGenerator::new);
+          Optional.ofNullable(streamContext != null ? streamContext.getRequestReplicaListTransformerGenerator() : null).orElseGet(RequestReplicaListTransformerGenerator::new);

Review comment:
       Yea, I think an extra null check is going to be more clear than the use of Optional here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [lucene-solr] cpoerschke merged pull request #1754: SOLR-14700: avoid NPEs in TupleStream.getShards() when streamContext … …is null.

Posted by GitBox <gi...@apache.org>.
cpoerschke merged pull request #1754:
URL: https://github.com/apache/lucene-solr/pull/1754


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [lucene-solr] madrob commented on a change in pull request #1754: SOLR-14700: avoid NPEs in TupleStream.getShards() when streamContext … …is null.

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #1754:
URL: https://github.com/apache/lucene-solr/pull/1754#discussion_r471546928



##########
File path: solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
##########
@@ -141,19 +141,30 @@ public UUID getStreamNodeId(){
       shards = shardsMap.get(collection);
     } else {
       //SolrCloud Sharding
-      CloudSolrClient cloudSolrClient =
-          Optional.ofNullable(streamContext.getSolrClientCache()).orElseGet(SolrClientCache::new).getCloudSolrClient(zkHost);
+      SolrClientCache solrClientCache = (streamContext != null ? streamContext.getSolrClientCache() : null);

Review comment:
       Yea, add a comment. I definitely missed that subtlety.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [lucene-solr] madrob commented on a change in pull request #1754: SOLR-14700: avoid NPEs in TupleStream.getShards() when streamContext … …is null.

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #1754:
URL: https://github.com/apache/lucene-solr/pull/1754#discussion_r471054161



##########
File path: solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
##########
@@ -141,19 +141,30 @@ public UUID getStreamNodeId(){
       shards = shardsMap.get(collection);
     } else {
       //SolrCloud Sharding
-      CloudSolrClient cloudSolrClient =
-          Optional.ofNullable(streamContext.getSolrClientCache()).orElseGet(SolrClientCache::new).getCloudSolrClient(zkHost);
+      SolrClientCache solrClientCache = (streamContext != null ? streamContext.getSolrClientCache() : null);
+      final SolrClientCache localSolrClientCache;
+      if (solrClientCache == null) {
+        solrClientCache = localSolrClientCache = new SolrClientCache();
+      } else {
+        localSolrClientCache = null;
+      }
+      CloudSolrClient cloudSolrClient = solrClientCache.getCloudSolrClient(zkHost);
       ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
       ClusterState clusterState = zkStateReader.getClusterState();
       Slice[] slices = CloudSolrStream.getSlices(collection, zkStateReader, true);
       Set<String> liveNodes = clusterState.getLiveNodes();
 
 
-      ModifiableSolrParams solrParams = new ModifiableSolrParams(streamContext.getRequestParams());
+      final ModifiableSolrParams solrParams;
+      if (streamContext != null) {
+        solrParams = new ModifiableSolrParams(streamContext.getRequestParams());
+      } else {
+        solrParams = new ModifiableSolrParams();
+      }
       solrParams.add(requestParams);
 
       RequestReplicaListTransformerGenerator requestReplicaListTransformerGenerator =
-          Optional.ofNullable(streamContext.getRequestReplicaListTransformerGenerator()).orElseGet(RequestReplicaListTransformerGenerator::new);
+          Optional.ofNullable(streamContext != null ? streamContext.getRequestReplicaListTransformerGenerator() : null).orElseGet(RequestReplicaListTransformerGenerator::new);

Review comment:
       There's a lot of complexity in this line, can we make it an if/else without wrapping and unwrapping an Optional? We are already doing a check on `streamContext != null` 6 lines up.

##########
File path: solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
##########
@@ -141,19 +141,30 @@ public UUID getStreamNodeId(){
       shards = shardsMap.get(collection);
     } else {
       //SolrCloud Sharding
-      CloudSolrClient cloudSolrClient =
-          Optional.ofNullable(streamContext.getSolrClientCache()).orElseGet(SolrClientCache::new).getCloudSolrClient(zkHost);
+      SolrClientCache solrClientCache = (streamContext != null ? streamContext.getSolrClientCache() : null);
+      final SolrClientCache localSolrClientCache;
+      if (solrClientCache == null) {
+        solrClientCache = localSolrClientCache = new SolrClientCache();

Review comment:
       This is clever, add a comment that we're saving `localCache` because we need to close it later. :)

##########
File path: solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
##########
@@ -141,19 +141,30 @@ public UUID getStreamNodeId(){
       shards = shardsMap.get(collection);
     } else {
       //SolrCloud Sharding
-      CloudSolrClient cloudSolrClient =
-          Optional.ofNullable(streamContext.getSolrClientCache()).orElseGet(SolrClientCache::new).getCloudSolrClient(zkHost);
+      SolrClientCache solrClientCache = (streamContext != null ? streamContext.getSolrClientCache() : null);

Review comment:
       I think this ternary folds into the conditional that we have:
   
   ```
         SolrClientCache scc, lscc;
         if (streamContext != null) {
           scc = streamContext.getSolrClientCache();
         } else {
           scc = lscc = new SolrClientCache();
         }
   ```

##########
File path: solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamingTest.java
##########
@@ -2716,4 +2716,11 @@ private ParallelStream parallelStream(TupleStream stream, FieldComparator compar
     return pstream;
   }
 
+  public void testCloudSolrStreamWithoutStreamContext() throws Exception {
+    SolrParams sParams = StreamingTest.mapParams("q", "*:*", "fl", "id", "sort", "id asc");
+    try (CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams)) {
+      stream.open();

Review comment:
       I really like how simple this test is.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [lucene-solr] madrob commented on pull request #1754: SOLR-14700: avoid NPEs in TupleStream.getShards() when streamContext … …is null.

Posted by GitBox <gi...@apache.org>.
madrob commented on pull request #1754:
URL: https://github.com/apache/lucene-solr/pull/1754#issuecomment-676656810


   I just learned of `Objects.requireNonNullElse()` that can make some of this more readable. Up to you if you want to switch code to that or leave as is, @cpoerschke I'm happy either way!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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