You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2022/04/15 18:24:52 UTC

[GitHub] [accumulo] ctubbsii opened a new pull request, #2636: Update Apache parent POM, modernizer

ctubbsii opened a new pull request, #2636:
URL: https://github.com/apache/accumulo/pull/2636

   * Update modernizer and fix new recommendations, primarily replacing
     Guava Iterables and making more use of Stream
   * Add ConfigurationCopy constructor to accept a Stream
   * Add sequential stream() to ScannerBase to easily stream results and
     replace `new StreamSupport(scanner.spliterator(), false)` with this
   * Fix incorrect imports of dependencies shaded into Hadoop
   * Remove no longer needed m2e ignores in the POM
   * Also bump sortpom and datasketches and add two new enforcer QA rules


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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #2636: Update Apache parent POM, modernizer

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on code in PR #2636:
URL: https://github.com/apache/accumulo/pull/2636#discussion_r851542074


##########
core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java:
##########
@@ -98,7 +98,8 @@ public RFileWriter build() throws IOException {
     userProps.putAll(samplerProps);
 
     if (!userProps.isEmpty()) {
-      acuconf = new ConfigurationCopy(Iterables.concat(acuconf, userProps.entrySet()));
+      acuconf =
+          new ConfigurationCopy(Stream.concat(acuconf.stream(), userProps.entrySet().stream()));

Review Comment:
   If you mean in the constructor, it's just because Stream has an iterator method already, but isn't marked with the "Iterable" interface. Using invoke direct on the stream's iterator method by declaring it as a lambda in the constructor effectively forces it to conform to the Iterable interface, but doesn't actually construct a new Iterable or do any conversion.
   
   This is really only weird because we have so many "Iterable" usages that are leftover from before Java had Streams. As we use Streams more, and Iterables less, we'll run into fewer of these weird situations.



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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #2636: Update Apache parent POM, modernizer

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on code in PR #2636:
URL: https://github.com/apache/accumulo/pull/2636#discussion_r851542494


##########
server/manager/src/test/java/org/apache/accumulo/manager/tableOps/bulkVer2/PrepBulkImportTest.java:
##########
@@ -79,14 +79,11 @@ List<KeyExtent> createExtents(Iterable<String> rowsIter) {
 
   Iterable<List<KeyExtent>> powerSet(KeyExtent... extents) {
     Set<Set<KeyExtent>> powerSet = Sets.powerSet(Set.of(extents));
-
-    return Iterables.transform(powerSet, set -> {
+    return () -> powerSet.stream().map(set -> {
       List<KeyExtent> list = new ArrayList<>(set);
-
       Collections.sort(list);
-
       return list;
-    });
+    }).iterator();

Review Comment:
   I would love to use Iterable less often, and Streams more. We definitely should do that. But that's a much larger change, as those kinds of things cascade. This was a minimal change to modernize this and avoid the Guava Iterables class.



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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2636: Update Apache parent POM, modernizer

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2636:
URL: https://github.com/apache/accumulo/pull/2636#discussion_r852207512


##########
server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java:
##########
@@ -246,7 +244,7 @@ public Stream<ExternalCompactionFinalState> getExternalCompactionFinalStates() {
 
     scanner.setRange(ExternalCompactionSection.getRange());
     int pLen = ExternalCompactionSection.getRowPrefix().length();
-    return StreamSupport.stream(scanner.spliterator(), false)
+    return scanner.stream()
         .map(e -> ExternalCompactionFinalState.fromJson(
             ExternalCompactionId.of(e.getKey().getRowData().toString().substring(pLen)),
             e.getValue().toString()));

Review Comment:
   Nevermind, the stream is not being returned, it's being collected into the Map and the Map is returned.



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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #2636: Update Apache parent POM, modernizer

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on code in PR #2636:
URL: https://github.com/apache/accumulo/pull/2636#discussion_r851543564


##########
core/src/main/java/org/apache/accumulo/core/conf/ConfigurationCopy.java:
##########
@@ -42,16 +43,24 @@ public ConfigurationCopy(Map<String,String> config) {
     this(config.entrySet());
   }
 
+  /**
+   * Creates a new configuration.
+   *
+   * @param config
+   *          configuration property stream to use for copying
+   */
+  public ConfigurationCopy(Stream<Entry<String,String>> config) {
+    this(() -> config.iterator());

Review Comment:
   Good catch.



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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2636: Update Apache parent POM, modernizer

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2636:
URL: https://github.com/apache/accumulo/pull/2636#discussion_r852195468


##########
core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java:
##########
@@ -370,4 +372,15 @@ default void forEach(BiConsumer<? super Key,? super Value> keyValueConsumer) {
       keyValueConsumer.accept(entry.getKey(), entry.getValue());
     }
   }
+
+  /**
+   * Stream the Scanner results sequentially from this scanner's iterator
+   *

Review Comment:
   Should we add a note here that if the resulting stream cannot be used outside of a try-with-resources block that closes the Scanner and that if the stream is returned from a function without the Scanner being closed then `.onClose(scanner::close)` should be called before leaving the method?



##########
core/src/main/java/org/apache/accumulo/core/metadata/schema/AmpleImpl.java:
##########
@@ -44,7 +44,7 @@ public TabletMetadata readTablet(KeyExtent extent, ReadConsistency readConsisten
     builder.readConsistency(readConsistency);
 
     try (TabletsMetadata tablets = builder.build()) {
-      return Iterables.getOnlyElement(tablets);
+      return tablets.stream().collect(onlyElement());

Review Comment:
   do we need `onClose(tablets::close)` here to close the underlying Scanner?



##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader9to10.java:
##########
@@ -549,8 +548,7 @@ private Iterator<String> getOldCandidates(ServerContext context, String tableNam
     Range range = DeletesSection.getRange();
     Scanner scanner = context.createScanner(tableName, Authorizations.EMPTY);
     scanner.setRange(range);
-    return StreamSupport.stream(scanner.spliterator(), false)
-        .filter(entry -> !entry.getValue().equals(UPGRADED))
+    return scanner.stream().filter(entry -> !entry.getValue().equals(UPGRADED))
         .map(entry -> entry.getKey().getRow().toString().substring(OLD_DELETE_PREFIX.length()))

Review Comment:
   Do we need `.onClose(scanner::close)` here?



##########
server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java:
##########
@@ -246,7 +244,7 @@ public Stream<ExternalCompactionFinalState> getExternalCompactionFinalStates() {
 
     scanner.setRange(ExternalCompactionSection.getRange());
     int pLen = ExternalCompactionSection.getRowPrefix().length();
-    return StreamSupport.stream(scanner.spliterator(), false)
+    return scanner.stream()
         .map(e -> ExternalCompactionFinalState.fromJson(
             ExternalCompactionId.of(e.getKey().getRowData().toString().substring(pLen)),
             e.getValue().toString()));

Review Comment:
   do we need `.onClose(scanner::close)` 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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] ctubbsii commented on pull request #2636: Update Apache parent POM, modernizer

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on PR #2636:
URL: https://github.com/apache/accumulo/pull/2636#issuecomment-1100752893

   @DomGarguilo Let me know when you're done with your review. I have verified all the ITs pass now. I had a slight problem with RowHashIT, but fixed it.


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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2636: Update Apache parent POM, modernizer

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2636:
URL: https://github.com/apache/accumulo/pull/2636#discussion_r852208205


##########
server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader9to10.java:
##########
@@ -549,8 +548,7 @@ private Iterator<String> getOldCandidates(ServerContext context, String tableNam
     Range range = DeletesSection.getRange();
     Scanner scanner = context.createScanner(tableName, Authorizations.EMPTY);
     scanner.setRange(range);
-    return StreamSupport.stream(scanner.spliterator(), false)
-        .filter(entry -> !entry.getValue().equals(UPGRADED))
+    return scanner.stream().filter(entry -> !entry.getValue().equals(UPGRADED))
         .map(entry -> entry.getKey().getRow().toString().substring(OLD_DELETE_PREFIX.length()))

Review Comment:
   Nevermind, the stream is not being returned, it's being collected into the Map and the Map is returned.



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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #2636: Update Apache parent POM, modernizer

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on code in PR #2636:
URL: https://github.com/apache/accumulo/pull/2636#discussion_r852259145


##########
core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java:
##########
@@ -370,4 +372,15 @@ default void forEach(BiConsumer<? super Key,? super Value> keyValueConsumer) {
       keyValueConsumer.accept(entry.getKey(), entry.getValue());
     }
   }
+
+  /**
+   * Stream the Scanner results sequentially from this scanner's iterator
+   *

Review Comment:
   I don't think that's necessary. We shouldn't specifically require the use of a try-with-resources block to close the Scanner. That's an optional language construct. The same goes for using `.onClose(scanner::close)`. It's a good idea for the user to close the Scanner object when they are done with it, but we shouldn't be prescriptive about how they manage their resource. They can close it whenever they like. They may want to re-use the Scanner for another stream.
   
   We don't offer recommendations for closing Scanner when iterator() or forEach() is used. I don't see why this would be different. It should suffice that Scanner is Closeable, and therefore expected to be closed when they are finished using it.



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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] ctubbsii merged pull request #2636: Update Apache parent POM, modernizer

Posted by GitBox <gi...@apache.org>.
ctubbsii merged PR #2636:
URL: https://github.com/apache/accumulo/pull/2636


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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] ctubbsii commented on pull request #2636: Update Apache parent POM, modernizer

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on PR #2636:
URL: https://github.com/apache/accumulo/pull/2636#issuecomment-1100478362

   I made some extra changes... I didn't expect the dependency analyzer changes, but I fixed the problems due to that.


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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2636: Update Apache parent POM, modernizer

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2636:
URL: https://github.com/apache/accumulo/pull/2636#discussion_r852207883


##########
core/src/main/java/org/apache/accumulo/core/metadata/schema/AmpleImpl.java:
##########
@@ -44,7 +44,7 @@ public TabletMetadata readTablet(KeyExtent extent, ReadConsistency readConsisten
     builder.readConsistency(readConsistency);
 
     try (TabletsMetadata tablets = builder.build()) {
-      return Iterables.getOnlyElement(tablets);
+      return tablets.stream().collect(onlyElement());

Review Comment:
   Nevermind, the stream is not being returned, it's being collected and the collection is returned.



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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #2636: Update Apache parent POM, modernizer

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on code in PR #2636:
URL: https://github.com/apache/accumulo/pull/2636#discussion_r852260229


##########
core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java:
##########
@@ -370,4 +372,15 @@ default void forEach(BiConsumer<? super Key,? super Value> keyValueConsumer) {
       keyValueConsumer.accept(entry.getKey(), entry.getValue());
     }
   }
+
+  /**
+   * Stream the Scanner results sequentially from this scanner's iterator
+   *

Review Comment:
   Will merge as-is, but feel free to follow up with additional javadoc changes or other docs if you feel it would be helpful.



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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2636: Update Apache parent POM, modernizer

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2636:
URL: https://github.com/apache/accumulo/pull/2636#discussion_r852195468


##########
core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java:
##########
@@ -370,4 +372,15 @@ default void forEach(BiConsumer<? super Key,? super Value> keyValueConsumer) {
       keyValueConsumer.accept(entry.getKey(), entry.getValue());
     }
   }
+
+  /**
+   * Stream the Scanner results sequentially from this scanner's iterator
+   *

Review Comment:
   Should we add a note here that the resulting stream cannot be used outside of a try-with-resources block that closes the Scanner and that if the stream is returned from a function without the Scanner being closed then `.onClose(scanner::close)` should be called before leaving the method?



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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] DomGarguilo commented on a diff in pull request #2636: Update Apache parent POM, modernizer

Posted by GitBox <gi...@apache.org>.
DomGarguilo commented on code in PR #2636:
URL: https://github.com/apache/accumulo/pull/2636#discussion_r851429539


##########
core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java:
##########
@@ -98,7 +98,8 @@ public RFileWriter build() throws IOException {
     userProps.putAll(samplerProps);
 
     if (!userProps.isEmpty()) {
-      acuconf = new ConfigurationCopy(Iterables.concat(acuconf, userProps.entrySet()));
+      acuconf =
+          new ConfigurationCopy(Stream.concat(acuconf.stream(), userProps.entrySet().stream()));

Review Comment:
   It seems weird that we would need to convert from a stream right back to an Iterable instead of keeping it this way. I guess the modernizer will complain if you don't do it this way and keep this as an iterable.



##########
server/manager/src/test/java/org/apache/accumulo/manager/tableOps/bulkVer2/PrepBulkImportTest.java:
##########
@@ -79,14 +79,11 @@ List<KeyExtent> createExtents(Iterable<String> rowsIter) {
 
   Iterable<List<KeyExtent>> powerSet(KeyExtent... extents) {
     Set<Set<KeyExtent>> powerSet = Sets.powerSet(Set.of(extents));
-
-    return Iterables.transform(powerSet, set -> {
+    return () -> powerSet.stream().map(set -> {
       List<KeyExtent> list = new ArrayList<>(set);
-
       Collections.sort(list);
-
       return list;
-    });
+    }).iterator();

Review Comment:
   Not sure if its worth it but this could return a stream instead of an Iterable and then be used with `powerSet().forEach`. 
   
   EDIT: It could potentially also accept a Stream of Extents too if that would be better.



##########
core/src/main/java/org/apache/accumulo/core/conf/ConfigurationCopy.java:
##########
@@ -42,16 +43,24 @@ public ConfigurationCopy(Map<String,String> config) {
     this(config.entrySet());
   }
 
+  /**
+   * Creates a new configuration.
+   *
+   * @param config
+   *          configuration property stream to use for copying
+   */
+  public ConfigurationCopy(Stream<Entry<String,String>> config) {
+    this(() -> config.iterator());

Review Comment:
   ```suggestion
       this(config::iterator);
   ```



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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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