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/07/14 20:32:34 UTC

[GitHub] [accumulo] DomGarguilo opened a new pull request, #2811: WIP: First pass at refactoring hot methods

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

   In #1099 there are some lists of methods that have been reported as "hot method too big".
   
   This PR refactors some of those methods to make them smaller. After these changes, the following methods are no longer reported as "hot method too big":
   * `org.apache.accumulo.core.file.rfile.RelativeKey::<init>`
   * `org.apache.accumulo.core.file.rfile.RelativeKey::readFields`
   * `org.apache.accumulo.server.fs.FileManager$ScanFileManager::openFiles`
   
   The following have been refactored to reduce size, but are still being reported:
   * `org.apache.accumulo.core.iteratorsImpl.system.LocalityGroupIterator::_seek`
   * `org.apache.accumulo.tserver.scan.LookupTask::run`


-- 
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] keith-turner commented on a diff in pull request #2811: Refactor hot methods

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2811:
URL: https://github.com/apache/accumulo/pull/2811#discussion_r982437354


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/scan/LookupTask.java:
##########
@@ -95,9 +95,14 @@ public void run() {
       // check the time so that the read ahead thread is not monopolized
       while (iter.hasNext() && bytesAdded < maxResultsSize
           && (System.currentTimeMillis() - startTime) < maxScanTime) {
-        Entry<KeyExtent,List<Range>> entry = iter.next();
-        KeyExtent extent = entry.getKey();
-        List<Range> ranges = entry.getValue();
+        final KeyExtent extent;
+        final List<Range> ranges;
+        {
+          final Entry<KeyExtent,List<Range>> entry = iter.next();
+          extent = entry.getKey();
+          ranges = entry.getValue();
+        }

Review Comment:
   > Yes. Maybe its not worth doing as it might cause confusion? What do you think?
   
   When I first saw it I had no idea why it was there.  I looked at it for a bit and guessed why it might be there. I think its an interesting technique and it reminds me of how in golang a function can return zero or more things.  Where in java a functions can only return zero or one things.   I really like how golang works.   To bad we can't write the following in java
   
   ```java
   (entry,ranges) = iter.next();
   ```
   
   Anyway, I am not completely sure if it should stay or go, but I am leaning towards removing it.   One thing that helped me think through this is asking would this be a good practice throughout the code?   It seems like this technique could add a lot of noise that may make the code harder to read if used frequently, so that's a reason not to use it elsewhere and here.  The tight scoping could help avoid some bugs in larger methods, but maybe if we have a large method we just need to make it smaller like you have done for methods in this PR.



-- 
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 #2811: Refactor hot methods

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


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java:
##########
@@ -1202,33 +1202,25 @@ public void flush(TInfo tinfo, TCredentials credentials, String lock, String tab
       throw new RuntimeException(e);
     }
 
-    ArrayList<Tablet> tabletsToFlush = new ArrayList<>();
-
     KeyExtent ke = new KeyExtent(TableId.of(tableId), ByteBufferUtil.toText(endRow),
         ByteBufferUtil.toText(startRow));
 
-    for (Tablet tablet : server.getOnlineTablets().values()) {
-      if (ke.overlaps(tablet.getExtent())) {
-        tabletsToFlush.add(tablet);
-      }
-    }
-
-    Long flushID = null;
+    final Long[] flushID = {null};

Review Comment:
   Good catch. Should be fixed by [09986b2](https://github.com/apache/accumulo/pull/2811/commits/09986b2f730c2e5cd0667de5bb01a0f04615955b).



-- 
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] keith-turner commented on a diff in pull request #2811: Refactor hot methods

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2811:
URL: https://github.com/apache/accumulo/pull/2811#discussion_r982269825


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java:
##########
@@ -396,80 +396,89 @@ Set<StoredTabletFile> getCandidates(Set<StoredTabletFile> currFiles, CompactionK
           if (isCompactionStratConfigured)
             return Set.of();
 
-          switch (selectStatus) {
-            case NOT_ACTIVE:
-            case CANCELED: {
-              Set<StoredTabletFile> candidates = new HashSet<>(currFiles);
-              candidates.removeAll(allCompactingFiles);
-              return Collections.unmodifiableSet(candidates);
-            }
-            case NEW:
-            case SELECTING:
-              return Set.of();
-            case SELECTED: {
-              Set<StoredTabletFile> candidates = new HashSet<>(currFiles);
-              candidates.removeAll(allCompactingFiles);
-              if (getNanoTime() - selectedTimeNanos < selectedExpirationDuration.toNanos()) {
-                candidates.removeAll(selectedFiles);
-              }
-              return Collections.unmodifiableSet(candidates);
-            }
-            case RESERVED: {
-              Set<StoredTabletFile> candidates = new HashSet<>(currFiles);
-              candidates.removeAll(allCompactingFiles);
-              candidates.removeAll(selectedFiles);
-              return Collections.unmodifiableSet(candidates);
-            }
-            default:
-              throw new AssertionError();
-          }
+          return handleSystemCompaction(currFiles);
         }
         case SELECTOR:
           // intentional fall through
         case USER:
-          switch (selectStatus) {
-            case NOT_ACTIVE:
-            case NEW:
-            case SELECTING:
-            case CANCELED:
-              return Set.of();
-            case SELECTED:
-            case RESERVED: {
-              if (selectKind == kind) {
-                Set<StoredTabletFile> candidates = new HashSet<>(selectedFiles);
-                candidates.removeAll(allCompactingFiles);
-                candidates = Collections.unmodifiableSet(candidates);
-                Preconditions.checkState(currFiles.containsAll(candidates),
-                    "selected files not in all files %s %s", candidates, currFiles);
-                return candidates;
-              } else {
-                return Set.of();
-              }
-            }
-            default:
-              throw new AssertionError();
-          }
+          return handleUserSelectorCompaction(currFiles, kind);
         case CHOP: {
-          switch (chopStatus) {
-            case NOT_ACTIVE:
-            case SELECTING:
-            case MARKING:
-              return Set.of();
-            case SELECTED: {
-              if (selectStatus == FileSelectionStatus.NEW
-                  || selectStatus == FileSelectionStatus.SELECTING)
-                return Set.of();
-
-              var filesToChop = getFilesToChop(currFiles);
-              filesToChop.removeAll(allCompactingFiles);
-              if (selectStatus == FileSelectionStatus.SELECTED
-                  || selectStatus == FileSelectionStatus.RESERVED)
-                filesToChop.removeAll(selectedFiles);
-              return Collections.unmodifiableSet(filesToChop);
-            }
-            default:
-              throw new AssertionError();
+          return handleChopCompaction(currFiles);
+        }
+        default:
+          throw new AssertionError();
+      }
+    }
+
+    private Set<StoredTabletFile> handleChopCompaction(Set<StoredTabletFile> currFiles) {
+      switch (chopStatus) {
+        case NOT_ACTIVE:
+        case SELECTING:
+        case MARKING:
+          return Set.of();
+        case SELECTED: {
+          if (selectStatus == FileSelectionStatus.NEW
+              || selectStatus == FileSelectionStatus.SELECTING)
+            return Set.of();
+
+          var filesToChop = getFilesToChop(currFiles);
+          filesToChop.removeAll(allCompactingFiles);
+          if (selectStatus == FileSelectionStatus.SELECTED
+              || selectStatus == FileSelectionStatus.RESERVED)
+            filesToChop.removeAll(selectedFiles);
+          return Collections.unmodifiableSet(filesToChop);
+        }
+        default:
+          throw new AssertionError();
+      }
+    }
+
+    private Set<StoredTabletFile> handleUserSelectorCompaction(Set<StoredTabletFile> currFiles,
+        CompactionKind kind) {
+      switch (selectStatus) {
+        case NOT_ACTIVE:
+        case NEW:
+        case SELECTING:
+        case CANCELED:
+          return Set.of();
+        case SELECTED:
+        case RESERVED: {
+          if (selectKind == kind) {
+            Set<StoredTabletFile> candidates = Sets.difference(selectedFiles, allCompactingFiles);
+            Preconditions.checkState(currFiles.containsAll(candidates),
+                "selected files not in all files %s %s", candidates, currFiles);
+            return Collections.unmodifiableSet(candidates);

Review Comment:
   This code is called in a sync block in which `selectedFiles` and `allCompactingFiles` will not be changing.  The functions `Sets.difference()` and `Collections.unmodifiableSet` may keep references to the passed in sets but will not copy.  So this function may return a set that internally references   `selectedFiles` and `allCompactingFiles` which may be changing after the method returns.  Its therefore important to copy as the original code did.
   
   The following change will copy.  I have no idea if its less or more efficient than the original code, but I think `Sets.difference()` plus `Set.copyOf()` is more readable than the original code so I would go with it. 
   
   ```suggestion
               //must create a copy because the sets passed to Sets.difference could change after this method returns
               return Set.copyOf(candidates);
   ```
   
   I looked around at the other changes and I think all of those copy, but it would be good if you could double check for any other cases like this in the changes in this file.



-- 
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 pull request #2811: WIP: First pass at refactoring hot methods

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

   I have marked this as WIP since there is still work to be done here:
   * Look into refactoring more methods appearing in the list of methods in #1099 
   * Rename new methods. I am not sure what to name the new methods so suggestions are very welcome.
     *  Namely, the new methods in `RelativeKey`: `extracted()` and `getBytes()`. Both of which I have left as thier IDE generated names


-- 
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] keith-turner commented on a diff in pull request #2811: Refactor hot methods

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2811:
URL: https://github.com/apache/accumulo/pull/2811#discussion_r978075208


##########
core/src/main/java/org/apache/accumulo/core/data/Key.java:
##########
@@ -975,30 +981,26 @@ public void write(DataOutput out) throws IOException {
    * @return true if specified parts of keys match, false otherwise
    */
   public boolean equals(Key other, PartialKey part) {
-    switch (part) {
-      case ROW:
-        return isEqual(row, other.row);
-      case ROW_COLFAM:
-        return isEqual(row, other.row) && isEqual(colFamily, other.colFamily);
-      case ROW_COLFAM_COLQUAL:
-        return isEqual(row, other.row) && isEqual(colFamily, other.colFamily)
-            && isEqual(colQualifier, other.colQualifier);
-      case ROW_COLFAM_COLQUAL_COLVIS:
-        return isEqual(row, other.row) && isEqual(colFamily, other.colFamily)
-            && isEqual(colQualifier, other.colQualifier)
-            && isEqual(colVisibility, other.colVisibility);
-      case ROW_COLFAM_COLQUAL_COLVIS_TIME:
-        return isEqual(row, other.row) && isEqual(colFamily, other.colFamily)
-            && isEqual(colQualifier, other.colQualifier)
-            && isEqual(colVisibility, other.colVisibility) && timestamp == other.timestamp;
-      case ROW_COLFAM_COLQUAL_COLVIS_TIME_DEL:
-        return isEqual(row, other.row) && isEqual(colFamily, other.colFamily)
-            && isEqual(colQualifier, other.colQualifier)
-            && isEqual(colVisibility, other.colVisibility) && timestamp == other.timestamp
-            && deleted == other.deleted;
-      default:
-        throw new IllegalArgumentException("Unrecognized partial key specification " + part);
-    }
+    boolean result = isEqual(row, other.row);
+    if (part == ROW)
+      return result;
+    result &= isEqual(colFamily, other.colFamily);
+    if (part == ROW_COLFAM)
+      return result;
+    result &= isEqual(colQualifier, other.colQualifier);
+    if (part == ROW_COLFAM_COLQUAL)
+      return result;
+    result &= isEqual(colVisibility, other.colVisibility);
+    if (part == ROW_COLFAM_COLQUAL_COLVIS)
+      return result;
+    result &= (timestamp == other.timestamp);
+    if (part == ROW_COLFAM_COLQUAL_COLVIS_TIME)
+      return result;
+    result &= (deleted == other.deleted);
+    if (part == ROW_COLFAM_COLQUAL_COLVIS_TIME_DEL)

Review Comment:
   The way the code was written before it would short circuit.  Like in the old code if the rows were not equal it would not do any comparisons after that.  In the new code when the rows are not equal it still compares the rest of the fields. Seems like this could really make a difference for some data.  Not sure we should manually short circuit because I don't know how that would compare to what the compiler does.



-- 
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 #2811: Refactor hot methods

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


##########
core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java:
##########
@@ -135,6 +118,17 @@ else if (cvCommonPrefixLen > 1)
       fieldsSame |= DELETED;
   }
 
+  private int getCommonPrefixLen(ByteSequence prevKeyScratch, ByteSequence keyScratch, byte rowSame,
+      byte commonPrefix) {
+    int commonPrefixLen = getCommonPrefix(prevKeyScratch, keyScratch);
+    if (commonPrefixLen == -1) {
+      fieldsSame |= rowSame;

Review Comment:
   Addressed in c86dac5



##########
core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/LocalityGroupIterator.java:
##########
@@ -207,35 +208,33 @@ static final Collection<LocalityGroup> _seek(HeapIterator hiter, LocalityGroupCo
        * other
        */
       if (!inclusive) {
-        for (Entry<ByteSequence,LocalityGroup> entry : lgContext.groupByCf.entrySet()) {
-          if (!cfSet.contains(entry.getKey())) {
-            groups.add(entry.getValue());
-          }
-        }
+        lgContext.groupByCf.entrySet().stream().filter(entry -> !cfSet.contains(entry.getKey()))
+            .map(Entry::getValue).forEach(groups::add);
       } else if (lgContext.groupByCf.size() <= cfSet.size()) {
-        for (Entry<ByteSequence,LocalityGroup> entry : lgContext.groupByCf.entrySet()) {
-          if (cfSet.contains(entry.getKey())) {
-            groups.add(entry.getValue());
-          }
-        }
+        lgContext.groupByCf.entrySet().stream().filter(entry -> cfSet.contains(entry.getKey()))
+            .map(Entry::getValue).forEach(groups::add);
       } else {
-        for (ByteSequence cf : cfSet) {
-          LocalityGroup group = lgContext.groupByCf.get(cf);
-          if (group != null) {
-            groups.add(group);
-          }
-        }
+        cfSet.stream().map(lgContext.groupByCf::get).filter(Objects::nonNull).forEach(groups::add);
       }
     }
 
-    for (LocalityGroup lgr : groups) {
-      lgr.getIterator().seek(range, EMPTY_CF_SET, false);
-      hiter.addSource(lgr.getIterator());
-    }
-
     return groups;
   }
 
+  private static Set<ByteSequence> getCfSet(Collection<ByteSequence> columnFamilies) {
+    final Set<ByteSequence> cfSet;
+    if (columnFamilies.isEmpty()) {
+      cfSet = Collections.emptySet();
+    } else {
+      if (columnFamilies instanceof Set<?>) {
+        cfSet = (Set<ByteSequence>) columnFamilies;
+      } else {
+        cfSet = new HashSet<>(columnFamilies);

Review Comment:
   Addressed in c86dac5



-- 
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 #2811: Refactor hot methods

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


##########
core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java:
##########
@@ -174,40 +168,13 @@ public void readFields(DataInput in) throws IOException {
       fieldsPrefixed = 0;
     }
 
-    byte[] row, cf, cq, cv;
-    long ts;
-
-    if ((fieldsSame & ROW_SAME) == ROW_SAME) {
-      row = prevKey.getRowData().toArray();
-    } else if ((fieldsPrefixed & ROW_COMMON_PREFIX) == ROW_COMMON_PREFIX) {
-      row = readPrefix(in, prevKey.getRowData());
-    } else {
-      row = read(in);
-    }
+    final byte[] row, cf, cq, cv;
+    final long ts;
 
-    if ((fieldsSame & CF_SAME) == CF_SAME) {
-      cf = prevKey.getColumnFamilyData().toArray();
-    } else if ((fieldsPrefixed & CF_COMMON_PREFIX) == CF_COMMON_PREFIX) {
-      cf = readPrefix(in, prevKey.getColumnFamilyData());
-    } else {
-      cf = read(in);
-    }
-
-    if ((fieldsSame & CQ_SAME) == CQ_SAME) {
-      cq = prevKey.getColumnQualifierData().toArray();
-    } else if ((fieldsPrefixed & CQ_COMMON_PREFIX) == CQ_COMMON_PREFIX) {
-      cq = readPrefix(in, prevKey.getColumnQualifierData());
-    } else {
-      cq = read(in);
-    }
-
-    if ((fieldsSame & CV_SAME) == CV_SAME) {
-      cv = prevKey.getColumnVisibilityData().toArray();
-    } else if ((fieldsPrefixed & CV_COMMON_PREFIX) == CV_COMMON_PREFIX) {
-      cv = readPrefix(in, prevKey.getColumnVisibilityData());
-    } else {
-      cv = read(in);
-    }
+    row = getData(in, ROW_SAME, ROW_COMMON_PREFIX, () -> prevKey.getRowData());
+    cf = getData(in, CF_SAME, CF_COMMON_PREFIX, () -> prevKey.getColumnFamilyData());
+    cq = getData(in, CQ_SAME, CQ_COMMON_PREFIX, () -> prevKey.getColumnQualifierData());
+    cv = getData(in, CV_SAME, CV_COMMON_PREFIX, () -> prevKey.getColumnVisibilityData());

Review Comment:
   Yea looks like it. Nice catch. Included in 59e8357



-- 
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] milleruntime commented on a diff in pull request #2811: Refactor hot methods

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


##########
core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/LocalityGroupIterator.java:
##########
@@ -163,26 +164,14 @@ static final Collection<LocalityGroup> _seek(HeapIterator hiter, LocalityGroupCo
       Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
     hiter.clear();
 
-    Set<ByteSequence> cfSet;
-    if (columnFamilies.isEmpty()) {
-      cfSet = Collections.emptySet();
-    } else {
-      if (columnFamilies instanceof Set<?>) {
-        cfSet = (Set<ByteSequence>) columnFamilies;
-      } else {
-        cfSet = new HashSet<>();
-        cfSet.addAll(columnFamilies);
-      }
-    }
+    final Set<ByteSequence> cfSet = getCfSet(columnFamilies);
 
     // determine the set of groups to use
-    Collection<LocalityGroup> groups = Collections.emptyList();
+    final Collection<LocalityGroup> groups;
 
     // if no column families specified, then include all groups unless !inclusive
     if (cfSet.isEmpty()) {
-      if (!inclusive) {
-        groups = lgContext.groups;
-      }
+      groups = inclusive ? List.of() : lgContext.groups;
     } else {
       groups = new HashSet<>();

Review Comment:
   Or create a method for all the logic for "determining the set of groups to use".



-- 
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 pull request #2811: Refactor hot methods

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

   I have refactored `Key.equals()` in a way that makes it not show up in the hot methods list any more. Here is the method after refactoring:
   ```java
     public boolean equals(Key other, PartialKey part) {
       boolean result = isEqual(row, other.row);
       if (part == ROW)
         return result;
       result &= isEqual(colFamily, other.colFamily);
       if (part == ROW_COLFAM)
         return result;
       result &= isEqual(colQualifier, other.colQualifier);
       if (part == ROW_COLFAM_COLQUAL)
         return result;
       result &= isEqual(colVisibility, other.colVisibility);
       if (part == ROW_COLFAM_COLQUAL_COLVIS)
         return result;
       result &= (timestamp == other.timestamp);
       if (part == ROW_COLFAM_COLQUAL_COLVIS_TIME)
         return result;
       result &= (deleted == other.deleted);
       if (part == ROW_COLFAM_COLQUAL_COLVIS_TIME_DEL)
         return result;
   
       throw new IllegalArgumentException("Unrecognized partial key specification " + part);
     }
   ```
   It will fall through and compare each `PartialKey` until it reaches `part` and returns. Here is the current code to compare:
   https://github.com/apache/accumulo/blob/388d3ffe11b3eaffe5a0022907d10e18bef73253/core/src/main/java/org/apache/accumulo/core/data/Key.java#L977-L1002
   
   While this refactored method is smaller and no longer "hot", it should probably be determined if this has any significant performance hits which is what is being invstigated in #2812 


-- 
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 #2811: Refactor hot methods

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


##########
core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/LocalityGroupIterator.java:
##########
@@ -163,26 +164,14 @@ static final Collection<LocalityGroup> _seek(HeapIterator hiter, LocalityGroupCo
       Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
     hiter.clear();
 
-    Set<ByteSequence> cfSet;
-    if (columnFamilies.isEmpty()) {
-      cfSet = Collections.emptySet();
-    } else {
-      if (columnFamilies instanceof Set<?>) {
-        cfSet = (Set<ByteSequence>) columnFamilies;
-      } else {
-        cfSet = new HashSet<>();
-        cfSet.addAll(columnFamilies);
-      }
-    }
+    final Set<ByteSequence> cfSet = getCfSet(columnFamilies);
 
     // determine the set of groups to use
-    Collection<LocalityGroup> groups = Collections.emptyList();
+    final Collection<LocalityGroup> groups;
 
     // if no column families specified, then include all groups unless !inclusive
     if (cfSet.isEmpty()) {
-      if (!inclusive) {
-        groups = lgContext.groups;
-      }
+      groups = inclusive ? List.of() : lgContext.groups;
     } else {
       groups = new HashSet<>();

Review Comment:
   Extracted all of the logic into its own method in [4b51e91](https://github.com/apache/accumulo/pull/2811/commits/4b51e91e20510be1c5913e68e0d8fbba2be0d361)



-- 
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 #2811: Refactor hot methods

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


-- 
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 #2811: Refactor hot methods

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


##########
core/src/main/java/org/apache/accumulo/core/data/Key.java:
##########
@@ -975,30 +981,26 @@ public void write(DataOutput out) throws IOException {
    * @return true if specified parts of keys match, false otherwise
    */
   public boolean equals(Key other, PartialKey part) {
-    switch (part) {
-      case ROW:
-        return isEqual(row, other.row);
-      case ROW_COLFAM:
-        return isEqual(row, other.row) && isEqual(colFamily, other.colFamily);
-      case ROW_COLFAM_COLQUAL:
-        return isEqual(row, other.row) && isEqual(colFamily, other.colFamily)
-            && isEqual(colQualifier, other.colQualifier);
-      case ROW_COLFAM_COLQUAL_COLVIS:
-        return isEqual(row, other.row) && isEqual(colFamily, other.colFamily)
-            && isEqual(colQualifier, other.colQualifier)
-            && isEqual(colVisibility, other.colVisibility);
-      case ROW_COLFAM_COLQUAL_COLVIS_TIME:
-        return isEqual(row, other.row) && isEqual(colFamily, other.colFamily)
-            && isEqual(colQualifier, other.colQualifier)
-            && isEqual(colVisibility, other.colVisibility) && timestamp == other.timestamp;
-      case ROW_COLFAM_COLQUAL_COLVIS_TIME_DEL:
-        return isEqual(row, other.row) && isEqual(colFamily, other.colFamily)
-            && isEqual(colQualifier, other.colQualifier)
-            && isEqual(colVisibility, other.colVisibility) && timestamp == other.timestamp
-            && deleted == other.deleted;
-      default:
-        throw new IllegalArgumentException("Unrecognized partial key specification " + part);
-    }
+    boolean result = isEqual(row, other.row);
+    if (part == ROW)
+      return result;
+    result &= isEqual(colFamily, other.colFamily);
+    if (part == ROW_COLFAM)
+      return result;
+    result &= isEqual(colQualifier, other.colQualifier);
+    if (part == ROW_COLFAM_COLQUAL)
+      return result;
+    result &= isEqual(colVisibility, other.colVisibility);
+    if (part == ROW_COLFAM_COLQUAL_COLVIS)
+      return result;
+    result &= (timestamp == other.timestamp);
+    if (part == ROW_COLFAM_COLQUAL_COLVIS_TIME)
+      return result;
+    result &= (deleted == other.deleted);
+    if (part == ROW_COLFAM_COLQUAL_COLVIS_TIME_DEL)

Review Comment:
   Good catch. I'll revert this to how it was.



-- 
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 #2811: Refactor hot methods

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


##########
core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java:
##########
@@ -174,40 +168,13 @@ public void readFields(DataInput in) throws IOException {
       fieldsPrefixed = 0;
     }
 
-    byte[] row, cf, cq, cv;
-    long ts;
-
-    if ((fieldsSame & ROW_SAME) == ROW_SAME) {
-      row = prevKey.getRowData().toArray();
-    } else if ((fieldsPrefixed & ROW_COMMON_PREFIX) == ROW_COMMON_PREFIX) {
-      row = readPrefix(in, prevKey.getRowData());
-    } else {
-      row = read(in);
-    }
+    final byte[] row, cf, cq, cv;
+    final long ts;
 
-    if ((fieldsSame & CF_SAME) == CF_SAME) {
-      cf = prevKey.getColumnFamilyData().toArray();
-    } else if ((fieldsPrefixed & CF_COMMON_PREFIX) == CF_COMMON_PREFIX) {
-      cf = readPrefix(in, prevKey.getColumnFamilyData());
-    } else {
-      cf = read(in);
-    }
-
-    if ((fieldsSame & CQ_SAME) == CQ_SAME) {
-      cq = prevKey.getColumnQualifierData().toArray();
-    } else if ((fieldsPrefixed & CQ_COMMON_PREFIX) == CQ_COMMON_PREFIX) {
-      cq = readPrefix(in, prevKey.getColumnQualifierData());
-    } else {
-      cq = read(in);
-    }
-
-    if ((fieldsSame & CV_SAME) == CV_SAME) {
-      cv = prevKey.getColumnVisibilityData().toArray();
-    } else if ((fieldsPrefixed & CV_COMMON_PREFIX) == CV_COMMON_PREFIX) {
-      cv = readPrefix(in, prevKey.getColumnVisibilityData());
-    } else {
-      cv = read(in);
-    }
+    row = getData(in, ROW_SAME, ROW_COMMON_PREFIX, () -> prevKey.getRowData());
+    cf = getData(in, CF_SAME, CF_COMMON_PREFIX, () -> prevKey.getColumnFamilyData());
+    cq = getData(in, CQ_SAME, CQ_COMMON_PREFIX, () -> prevKey.getColumnQualifierData());
+    cv = getData(in, CV_SAME, CV_COMMON_PREFIX, () -> prevKey.getColumnVisibilityData());

Review Comment:
   I love the use of lambdas, but I think these can just use the form like `prevKey::getColumnFamilyData`, right?



##########
core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java:
##########
@@ -297,10 +275,7 @@ List<TabletMigration> move(ServerCounts tooMuch, ServerCounts tooLittle, int cou
        * is only one tabletserver that holds all of the tablets. Here we check to see if in fact
        * that is the case and if so set the value to 0.
        */
-      tooLittleCount = tooLittleMap.get(table);
-      if (tooLittleCount == null) {
-        tooLittleCount = 0;
-      }
+      Integer tooLittleCount = tooLittleMap.getOrDefault(table, 0);

Review Comment:
   I briefly looked, and didn't see an Integer.ZERO. Such a thing would be useful here, if it existed, so we don't have to keep boxing `0`. But, I'm sure this is fine. The JRE can optimize stuff like this if it wanted to.



-- 
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 #2811: Refactor hot methods

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


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java:
##########
@@ -396,80 +396,89 @@ Set<StoredTabletFile> getCandidates(Set<StoredTabletFile> currFiles, CompactionK
           if (isCompactionStratConfigured)
             return Set.of();
 
-          switch (selectStatus) {
-            case NOT_ACTIVE:
-            case CANCELED: {
-              Set<StoredTabletFile> candidates = new HashSet<>(currFiles);
-              candidates.removeAll(allCompactingFiles);
-              return Collections.unmodifiableSet(candidates);
-            }
-            case NEW:
-            case SELECTING:
-              return Set.of();
-            case SELECTED: {
-              Set<StoredTabletFile> candidates = new HashSet<>(currFiles);
-              candidates.removeAll(allCompactingFiles);
-              if (getNanoTime() - selectedTimeNanos < selectedExpirationDuration.toNanos()) {
-                candidates.removeAll(selectedFiles);
-              }
-              return Collections.unmodifiableSet(candidates);
-            }
-            case RESERVED: {
-              Set<StoredTabletFile> candidates = new HashSet<>(currFiles);
-              candidates.removeAll(allCompactingFiles);
-              candidates.removeAll(selectedFiles);
-              return Collections.unmodifiableSet(candidates);
-            }
-            default:
-              throw new AssertionError();
-          }
+          return handleSystemCompaction(currFiles);
         }
         case SELECTOR:
           // intentional fall through
         case USER:
-          switch (selectStatus) {
-            case NOT_ACTIVE:
-            case NEW:
-            case SELECTING:
-            case CANCELED:
-              return Set.of();
-            case SELECTED:
-            case RESERVED: {
-              if (selectKind == kind) {
-                Set<StoredTabletFile> candidates = new HashSet<>(selectedFiles);
-                candidates.removeAll(allCompactingFiles);
-                candidates = Collections.unmodifiableSet(candidates);
-                Preconditions.checkState(currFiles.containsAll(candidates),
-                    "selected files not in all files %s %s", candidates, currFiles);
-                return candidates;
-              } else {
-                return Set.of();
-              }
-            }
-            default:
-              throw new AssertionError();
-          }
+          return handleUserSelectorCompaction(currFiles, kind);
         case CHOP: {
-          switch (chopStatus) {
-            case NOT_ACTIVE:
-            case SELECTING:
-            case MARKING:
-              return Set.of();
-            case SELECTED: {
-              if (selectStatus == FileSelectionStatus.NEW
-                  || selectStatus == FileSelectionStatus.SELECTING)
-                return Set.of();
-
-              var filesToChop = getFilesToChop(currFiles);
-              filesToChop.removeAll(allCompactingFiles);
-              if (selectStatus == FileSelectionStatus.SELECTED
-                  || selectStatus == FileSelectionStatus.RESERVED)
-                filesToChop.removeAll(selectedFiles);
-              return Collections.unmodifiableSet(filesToChop);
-            }
-            default:
-              throw new AssertionError();
+          return handleChopCompaction(currFiles);
+        }
+        default:
+          throw new AssertionError();
+      }
+    }
+
+    private Set<StoredTabletFile> handleChopCompaction(Set<StoredTabletFile> currFiles) {
+      switch (chopStatus) {
+        case NOT_ACTIVE:
+        case SELECTING:
+        case MARKING:
+          return Set.of();
+        case SELECTED: {
+          if (selectStatus == FileSelectionStatus.NEW
+              || selectStatus == FileSelectionStatus.SELECTING)
+            return Set.of();
+
+          var filesToChop = getFilesToChop(currFiles);
+          filesToChop.removeAll(allCompactingFiles);
+          if (selectStatus == FileSelectionStatus.SELECTED
+              || selectStatus == FileSelectionStatus.RESERVED)
+            filesToChop.removeAll(selectedFiles);
+          return Collections.unmodifiableSet(filesToChop);
+        }
+        default:
+          throw new AssertionError();
+      }
+    }
+
+    private Set<StoredTabletFile> handleUserSelectorCompaction(Set<StoredTabletFile> currFiles,
+        CompactionKind kind) {
+      switch (selectStatus) {
+        case NOT_ACTIVE:
+        case NEW:
+        case SELECTING:
+        case CANCELED:
+          return Set.of();
+        case SELECTED:
+        case RESERVED: {
+          if (selectKind == kind) {
+            Set<StoredTabletFile> candidates = Sets.difference(selectedFiles, allCompactingFiles);
+            Preconditions.checkState(currFiles.containsAll(candidates),
+                "selected files not in all files %s %s", candidates, currFiles);
+            return Collections.unmodifiableSet(candidates);

Review Comment:
   Addressed in 7b5a7d9. I added `Set.copyOf()` to one other spot so take a look and see what you think.



-- 
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 #2811: Refactor hot methods

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


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/scan/LookupTask.java:
##########
@@ -95,9 +95,14 @@ public void run() {
       // check the time so that the read ahead thread is not monopolized
       while (iter.hasNext() && bytesAdded < maxResultsSize
           && (System.currentTimeMillis() - startTime) < maxScanTime) {
-        Entry<KeyExtent,List<Range>> entry = iter.next();
-        KeyExtent extent = entry.getKey();
-        List<Range> ranges = entry.getValue();
+        final KeyExtent extent;
+        final List<Range> ranges;
+        {
+          final Entry<KeyExtent,List<Range>> entry = iter.next();
+          extent = entry.getKey();
+          ranges = entry.getValue();
+        }

Review Comment:
   > Was this little code block created just to narrowly scope `entry`?
   
   Yes. Maybe its not worth doing as it might cause confusion? What do you think?



-- 
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] milleruntime commented on pull request #2811: Refactor hot methods

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

   Thanks for taking a look at these methods. I will take a look at this today.


-- 
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] milleruntime commented on pull request #2811: Refactor hot methods

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

   @DomGarguilo is this PR ready to merge? If so, I can take another close look at 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] DomGarguilo commented on a diff in pull request #2811: Refactor hot methods

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


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/scan/LookupTask.java:
##########
@@ -95,9 +95,14 @@ public void run() {
       // check the time so that the read ahead thread is not monopolized
       while (iter.hasNext() && bytesAdded < maxResultsSize
           && (System.currentTimeMillis() - startTime) < maxScanTime) {
-        Entry<KeyExtent,List<Range>> entry = iter.next();
-        KeyExtent extent = entry.getKey();
-        List<Range> ranges = entry.getValue();
+        final KeyExtent extent;
+        final List<Range> ranges;
+        {
+          final Entry<KeyExtent,List<Range>> entry = iter.next();
+          extent = entry.getKey();
+          ranges = entry.getValue();
+        }

Review Comment:
   Addressed in c86dac5



-- 
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 #2811: Refactor hot methods

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


##########
core/src/main/java/org/apache/accumulo/core/data/Key.java:
##########
@@ -975,30 +981,26 @@ public void write(DataOutput out) throws IOException {
    * @return true if specified parts of keys match, false otherwise
    */
   public boolean equals(Key other, PartialKey part) {
-    switch (part) {
-      case ROW:
-        return isEqual(row, other.row);
-      case ROW_COLFAM:
-        return isEqual(row, other.row) && isEqual(colFamily, other.colFamily);
-      case ROW_COLFAM_COLQUAL:
-        return isEqual(row, other.row) && isEqual(colFamily, other.colFamily)
-            && isEqual(colQualifier, other.colQualifier);
-      case ROW_COLFAM_COLQUAL_COLVIS:
-        return isEqual(row, other.row) && isEqual(colFamily, other.colFamily)
-            && isEqual(colQualifier, other.colQualifier)
-            && isEqual(colVisibility, other.colVisibility);
-      case ROW_COLFAM_COLQUAL_COLVIS_TIME:
-        return isEqual(row, other.row) && isEqual(colFamily, other.colFamily)
-            && isEqual(colQualifier, other.colQualifier)
-            && isEqual(colVisibility, other.colVisibility) && timestamp == other.timestamp;
-      case ROW_COLFAM_COLQUAL_COLVIS_TIME_DEL:
-        return isEqual(row, other.row) && isEqual(colFamily, other.colFamily)
-            && isEqual(colQualifier, other.colQualifier)
-            && isEqual(colVisibility, other.colVisibility) && timestamp == other.timestamp
-            && deleted == other.deleted;
-      default:
-        throw new IllegalArgumentException("Unrecognized partial key specification " + part);
-    }
+    boolean result = isEqual(row, other.row);
+    if (part == ROW)
+      return result;
+    result &= isEqual(colFamily, other.colFamily);
+    if (part == ROW_COLFAM)
+      return result;
+    result &= isEqual(colQualifier, other.colQualifier);
+    if (part == ROW_COLFAM_COLQUAL)
+      return result;
+    result &= isEqual(colVisibility, other.colVisibility);
+    if (part == ROW_COLFAM_COLQUAL_COLVIS)
+      return result;
+    result &= (timestamp == other.timestamp);
+    if (part == ROW_COLFAM_COLQUAL_COLVIS_TIME)
+      return result;
+    result &= (deleted == other.deleted);
+    if (part == ROW_COLFAM_COLQUAL_COLVIS_TIME_DEL)

Review Comment:
   @keith-turner this has been reverted in 5238f81



-- 
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 #2811: Refactor hot methods

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


##########
core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java:
##########
@@ -174,40 +168,13 @@ public void readFields(DataInput in) throws IOException {
       fieldsPrefixed = 0;
     }
 
-    byte[] row, cf, cq, cv;
-    long ts;
-
-    if ((fieldsSame & ROW_SAME) == ROW_SAME) {
-      row = prevKey.getRowData().toArray();
-    } else if ((fieldsPrefixed & ROW_COMMON_PREFIX) == ROW_COMMON_PREFIX) {
-      row = readPrefix(in, prevKey.getRowData());
-    } else {
-      row = read(in);
-    }
+    final byte[] row, cf, cq, cv;
+    final long ts;
 
-    if ((fieldsSame & CF_SAME) == CF_SAME) {
-      cf = prevKey.getColumnFamilyData().toArray();
-    } else if ((fieldsPrefixed & CF_COMMON_PREFIX) == CF_COMMON_PREFIX) {
-      cf = readPrefix(in, prevKey.getColumnFamilyData());
-    } else {
-      cf = read(in);
-    }
-
-    if ((fieldsSame & CQ_SAME) == CQ_SAME) {
-      cq = prevKey.getColumnQualifierData().toArray();
-    } else if ((fieldsPrefixed & CQ_COMMON_PREFIX) == CQ_COMMON_PREFIX) {
-      cq = readPrefix(in, prevKey.getColumnQualifierData());
-    } else {
-      cq = read(in);
-    }
-
-    if ((fieldsSame & CV_SAME) == CV_SAME) {
-      cv = prevKey.getColumnVisibilityData().toArray();
-    } else if ((fieldsPrefixed & CV_COMMON_PREFIX) == CV_COMMON_PREFIX) {
-      cv = readPrefix(in, prevKey.getColumnVisibilityData());
-    } else {
-      cv = read(in);
-    }
+    row = getData(in, ROW_SAME, ROW_COMMON_PREFIX, () -> prevKey.getRowData());
+    cf = getData(in, CF_SAME, CF_COMMON_PREFIX, () -> prevKey.getColumnFamilyData());
+    cq = getData(in, CQ_SAME, CQ_COMMON_PREFIX, () -> prevKey.getColumnQualifierData());
+    cv = getData(in, CV_SAME, CV_COMMON_PREFIX, () -> prevKey.getColumnVisibilityData());

Review Comment:
   The test errors can be seen [here](https://github.com/apache/accumulo/actions/runs/3152941923/jobs/5128865689#step:6:2393)
   
   here are the failing tests:
   ```
   [ERROR] Errors: 
   [ERROR]   RFileClientTest.testCache:523->lambda$testCache$0:525 » NullPointer
   [ERROR]   BlockIndexTest.test1:91 » NullPointer
   [ERROR]   BlockIndexTest.testSame:163 » NullPointer
   [ERROR]   RFileTest.test11:1213 » NullPointer
   [ERROR]   RFileTest.test17:1409 » NullPointer
   [ERROR]   RFileTest.test18:1542->t18Verify:1494 » NullPointer
   [ERROR]   RFileTest.test3:472 » NullPointer
   [ERROR]   RFileTest.test8:758 » NullPointer
   [ERROR]   RFileTest.testEncRFile11:1842->test11:1213 » NullPointer
   [ERROR]   RFileTest.testEncRFile17:1876->test17:1409 » NullPointer
   [ERROR]   RFileTest.testEncRFile18:1882->test18:1542->t18Verify:1494 » NullPointer
   [ERROR]   RFileTest.testEncRFile3:1786->test3:472 » NullPointer
   [ERROR]   RFileTest.testEncRFile8:1821->test8:758 » NullPointer
   [ERROR]   RFileTest.testEncSample:2195->testSampleLG:2174->checkSample:2006 » NullPointer
   [ERROR]   RFileTest.testEncryptedRFiles:1898->test3:472 » NullPointer
   [ERROR]   RFileTest.testOldVersions:1684->runVersionTest:1727 » NullPointer
   [ERROR]   RFileTest.testOldVersionsWithCrypto:1693->runVersionTest:1727 » NullPointer
   [ERROR]   RFileTest.testReseekUnconsumed:1634 » NullPointer
   [ERROR]   RFileTest.testSampleLG:2174->checkSample:2006 » NullPointe
   ```
   Locally i kept stepping back through commits to determine that commit was causing these failures. I also think its weird that this causes failures but it seems like it 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.

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 pull request #2811: Refactor hot methods

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

   > @DomGarguilo is this PR ready to merge? If so, I can take another close look at it.
   
   Yes, I think I am done making changes here and was just waiting for someone to verify these changes.


-- 
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 pull request #2811: Refactor hot methods

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

   A few more methods were found ([details](https://github.com/apache/accumulo/issues/1099#issuecomment-1198342829)). I'll look into those now, before this is merged.


-- 
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 #2811: Refactor hot methods

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


##########
core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java:
##########
@@ -174,40 +168,13 @@ public void readFields(DataInput in) throws IOException {
       fieldsPrefixed = 0;
     }
 
-    byte[] row, cf, cq, cv;
-    long ts;
-
-    if ((fieldsSame & ROW_SAME) == ROW_SAME) {
-      row = prevKey.getRowData().toArray();
-    } else if ((fieldsPrefixed & ROW_COMMON_PREFIX) == ROW_COMMON_PREFIX) {
-      row = readPrefix(in, prevKey.getRowData());
-    } else {
-      row = read(in);
-    }
+    final byte[] row, cf, cq, cv;
+    final long ts;
 
-    if ((fieldsSame & CF_SAME) == CF_SAME) {
-      cf = prevKey.getColumnFamilyData().toArray();
-    } else if ((fieldsPrefixed & CF_COMMON_PREFIX) == CF_COMMON_PREFIX) {
-      cf = readPrefix(in, prevKey.getColumnFamilyData());
-    } else {
-      cf = read(in);
-    }
-
-    if ((fieldsSame & CQ_SAME) == CQ_SAME) {
-      cq = prevKey.getColumnQualifierData().toArray();
-    } else if ((fieldsPrefixed & CQ_COMMON_PREFIX) == CQ_COMMON_PREFIX) {
-      cq = readPrefix(in, prevKey.getColumnQualifierData());
-    } else {
-      cq = read(in);
-    }
-
-    if ((fieldsSame & CV_SAME) == CV_SAME) {
-      cv = prevKey.getColumnVisibilityData().toArray();
-    } else if ((fieldsPrefixed & CV_COMMON_PREFIX) == CV_COMMON_PREFIX) {
-      cv = readPrefix(in, prevKey.getColumnVisibilityData());
-    } else {
-      cv = read(in);
-    }
+    row = getData(in, ROW_SAME, ROW_COMMON_PREFIX, () -> prevKey.getRowData());
+    cf = getData(in, CF_SAME, CF_COMMON_PREFIX, () -> prevKey.getColumnFamilyData());
+    cq = getData(in, CQ_SAME, CQ_COMMON_PREFIX, () -> prevKey.getColumnQualifierData());
+    cv = getData(in, CV_SAME, CV_COMMON_PREFIX, () -> prevKey.getColumnVisibilityData());

Review Comment:
   For some reason this is causing some tests to fail so I am going to revert the changes in 59e8357



-- 
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] keith-turner commented on a diff in pull request #2811: Refactor hot methods

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2811:
URL: https://github.com/apache/accumulo/pull/2811#discussion_r982208133


##########
core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java:
##########
@@ -221,6 +188,17 @@ public void readFields(DataInput in) throws IOException {
     this.prevKey = this.key;
   }
 
+  private byte[] getData(DataInput in, byte same, byte commonPrefix, Supplier<ByteSequence> data)

Review Comment:
   I suggested using `fieldBit` in a previous comment instead of `rowSame`.  What ever you end using in the other method it would be good to make the variable name here `same` consistent with the name in the other method.  Could change `rowSame` to `same` in the other method to make it consistent with this method.
   
   Below are suggested renamings for overall consistency.
   
   ```suggestion
     private byte[] getData(DataInput in, byte fieldBit, byte commonPrefix, Supplier<ByteSequence> fieldReader)
   ```



##########
core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java:
##########
@@ -135,6 +118,17 @@ else if (cvCommonPrefixLen > 1)
       fieldsSame |= DELETED;
   }
 
+  private int getCommonPrefixLen(ByteSequence prevKeyScratch, ByteSequence keyScratch, byte rowSame,
+      byte commonPrefix) {
+    int commonPrefixLen = getCommonPrefix(prevKeyScratch, keyScratch);
+    if (commonPrefixLen == -1) {
+      fieldsSame |= rowSame;

Review Comment:
   Need some other variable name for `rowSame` (because it not always a row field thats being dealt with), tried using `fieldBit` instead.
   
   ```suggestion
     private int getCommonPrefixLen(ByteSequence prevKeyScratch, ByteSequence keyScratch, byte fieldBit,
         byte commonPrefix) {
       int commonPrefixLen = getCommonPrefix(prevKeyScratch, keyScratch);
       if (commonPrefixLen == -1) {
         fieldsSame |= fieldBit;
   ```



##########
core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java:
##########
@@ -174,40 +168,13 @@ public void readFields(DataInput in) throws IOException {
       fieldsPrefixed = 0;
     }
 
-    byte[] row, cf, cq, cv;
-    long ts;
-
-    if ((fieldsSame & ROW_SAME) == ROW_SAME) {
-      row = prevKey.getRowData().toArray();
-    } else if ((fieldsPrefixed & ROW_COMMON_PREFIX) == ROW_COMMON_PREFIX) {
-      row = readPrefix(in, prevKey.getRowData());
-    } else {
-      row = read(in);
-    }
-
-    if ((fieldsSame & CF_SAME) == CF_SAME) {
-      cf = prevKey.getColumnFamilyData().toArray();
-    } else if ((fieldsPrefixed & CF_COMMON_PREFIX) == CF_COMMON_PREFIX) {
-      cf = readPrefix(in, prevKey.getColumnFamilyData());
-    } else {
-      cf = read(in);
-    }
-
-    if ((fieldsSame & CQ_SAME) == CQ_SAME) {
-      cq = prevKey.getColumnQualifierData().toArray();
-    } else if ((fieldsPrefixed & CQ_COMMON_PREFIX) == CQ_COMMON_PREFIX) {
-      cq = readPrefix(in, prevKey.getColumnQualifierData());
-    } else {
-      cq = read(in);
-    }
+    final byte[] row, cf, cq, cv;
+    final long ts;
 
-    if ((fieldsSame & CV_SAME) == CV_SAME) {
-      cv = prevKey.getColumnVisibilityData().toArray();
-    } else if ((fieldsPrefixed & CV_COMMON_PREFIX) == CV_COMMON_PREFIX) {
-      cv = readPrefix(in, prevKey.getColumnVisibilityData());
-    } else {
-      cv = read(in);
-    }
+    row = getData(in, ROW_SAME, ROW_COMMON_PREFIX, () -> prevKey.getRowData());

Review Comment:
   This is a really nice change.



##########
server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java:
##########
@@ -396,80 +396,89 @@ Set<StoredTabletFile> getCandidates(Set<StoredTabletFile> currFiles, CompactionK
           if (isCompactionStratConfigured)
             return Set.of();
 
-          switch (selectStatus) {
-            case NOT_ACTIVE:
-            case CANCELED: {
-              Set<StoredTabletFile> candidates = new HashSet<>(currFiles);
-              candidates.removeAll(allCompactingFiles);
-              return Collections.unmodifiableSet(candidates);
-            }
-            case NEW:
-            case SELECTING:
-              return Set.of();
-            case SELECTED: {
-              Set<StoredTabletFile> candidates = new HashSet<>(currFiles);
-              candidates.removeAll(allCompactingFiles);
-              if (getNanoTime() - selectedTimeNanos < selectedExpirationDuration.toNanos()) {
-                candidates.removeAll(selectedFiles);
-              }
-              return Collections.unmodifiableSet(candidates);
-            }
-            case RESERVED: {
-              Set<StoredTabletFile> candidates = new HashSet<>(currFiles);
-              candidates.removeAll(allCompactingFiles);
-              candidates.removeAll(selectedFiles);
-              return Collections.unmodifiableSet(candidates);
-            }
-            default:
-              throw new AssertionError();
-          }
+          return handleSystemCompaction(currFiles);
         }
         case SELECTOR:
           // intentional fall through
         case USER:
-          switch (selectStatus) {
-            case NOT_ACTIVE:
-            case NEW:
-            case SELECTING:
-            case CANCELED:
-              return Set.of();
-            case SELECTED:
-            case RESERVED: {
-              if (selectKind == kind) {
-                Set<StoredTabletFile> candidates = new HashSet<>(selectedFiles);
-                candidates.removeAll(allCompactingFiles);
-                candidates = Collections.unmodifiableSet(candidates);
-                Preconditions.checkState(currFiles.containsAll(candidates),
-                    "selected files not in all files %s %s", candidates, currFiles);
-                return candidates;
-              } else {
-                return Set.of();
-              }
-            }
-            default:
-              throw new AssertionError();
-          }
+          return handleUserSelectorCompaction(currFiles, kind);
         case CHOP: {
-          switch (chopStatus) {
-            case NOT_ACTIVE:
-            case SELECTING:
-            case MARKING:
-              return Set.of();
-            case SELECTED: {
-              if (selectStatus == FileSelectionStatus.NEW
-                  || selectStatus == FileSelectionStatus.SELECTING)
-                return Set.of();
-
-              var filesToChop = getFilesToChop(currFiles);
-              filesToChop.removeAll(allCompactingFiles);
-              if (selectStatus == FileSelectionStatus.SELECTED
-                  || selectStatus == FileSelectionStatus.RESERVED)
-                filesToChop.removeAll(selectedFiles);
-              return Collections.unmodifiableSet(filesToChop);
-            }
-            default:
-              throw new AssertionError();
+          return handleChopCompaction(currFiles);
+        }
+        default:
+          throw new AssertionError();
+      }
+    }
+
+    private Set<StoredTabletFile> handleChopCompaction(Set<StoredTabletFile> currFiles) {
+      switch (chopStatus) {
+        case NOT_ACTIVE:
+        case SELECTING:
+        case MARKING:
+          return Set.of();
+        case SELECTED: {
+          if (selectStatus == FileSelectionStatus.NEW
+              || selectStatus == FileSelectionStatus.SELECTING)
+            return Set.of();
+
+          var filesToChop = getFilesToChop(currFiles);
+          filesToChop.removeAll(allCompactingFiles);
+          if (selectStatus == FileSelectionStatus.SELECTED
+              || selectStatus == FileSelectionStatus.RESERVED)
+            filesToChop.removeAll(selectedFiles);
+          return Collections.unmodifiableSet(filesToChop);
+        }
+        default:
+          throw new AssertionError();
+      }
+    }
+
+    private Set<StoredTabletFile> handleUserSelectorCompaction(Set<StoredTabletFile> currFiles,
+        CompactionKind kind) {
+      switch (selectStatus) {
+        case NOT_ACTIVE:
+        case NEW:
+        case SELECTING:
+        case CANCELED:
+          return Set.of();
+        case SELECTED:
+        case RESERVED: {
+          if (selectKind == kind) {
+            Set<StoredTabletFile> candidates = Sets.difference(selectedFiles, allCompactingFiles);
+            Preconditions.checkState(currFiles.containsAll(candidates),
+                "selected files not in all files %s %s", candidates, currFiles);
+            return Collections.unmodifiableSet(candidates);

Review Comment:
   This code is called in a sync block in which `selectedFiles` and `allCompactingFiles` will not be changing.  The functions `Sets.difference()` and `Collections.unmodifiableSet` may keep references to the passed in sets but will not copy.  So this function may return a set that internally references   `selectedFiles` and `allCompactingFiles` which may be changing.  Its therefore important to copy as the original code did.
   
   The following change will copy.  I have no idea if its less or more efficient than the original code, but I think `Sets.difference()` plus `Set.copyOf()` is more readable than the original code so I would go with it. 
   
   ```suggestion
               //must create a copy because the sets passed to Sets.difference could change after this method returns
               return Set.copyOf(candidates);
   ```
   
   I looked around at the other changes and I think all of those copy, but it would be good if you could double check for any other cases like this in the changes in this file.



##########
server/tserver/src/main/java/org/apache/accumulo/tserver/scan/LookupTask.java:
##########
@@ -95,9 +95,14 @@ public void run() {
       // check the time so that the read ahead thread is not monopolized
       while (iter.hasNext() && bytesAdded < maxResultsSize
           && (System.currentTimeMillis() - startTime) < maxScanTime) {
-        Entry<KeyExtent,List<Range>> entry = iter.next();
-        KeyExtent extent = entry.getKey();
-        List<Range> ranges = entry.getValue();
+        final KeyExtent extent;
+        final List<Range> ranges;
+        {
+          final Entry<KeyExtent,List<Range>> entry = iter.next();
+          extent = entry.getKey();
+          ranges = entry.getValue();
+        }

Review Comment:
   Was this little code block created just to narrowly scope `entry`?



##########
core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/LocalityGroupIterator.java:
##########
@@ -207,35 +208,33 @@ static final Collection<LocalityGroup> _seek(HeapIterator hiter, LocalityGroupCo
        * other
        */
       if (!inclusive) {
-        for (Entry<ByteSequence,LocalityGroup> entry : lgContext.groupByCf.entrySet()) {
-          if (!cfSet.contains(entry.getKey())) {
-            groups.add(entry.getValue());
-          }
-        }
+        lgContext.groupByCf.entrySet().stream().filter(entry -> !cfSet.contains(entry.getKey()))
+            .map(Entry::getValue).forEach(groups::add);
       } else if (lgContext.groupByCf.size() <= cfSet.size()) {
-        for (Entry<ByteSequence,LocalityGroup> entry : lgContext.groupByCf.entrySet()) {
-          if (cfSet.contains(entry.getKey())) {
-            groups.add(entry.getValue());
-          }
-        }
+        lgContext.groupByCf.entrySet().stream().filter(entry -> cfSet.contains(entry.getKey()))
+            .map(Entry::getValue).forEach(groups::add);
       } else {
-        for (ByteSequence cf : cfSet) {
-          LocalityGroup group = lgContext.groupByCf.get(cf);
-          if (group != null) {
-            groups.add(group);
-          }
-        }
+        cfSet.stream().map(lgContext.groupByCf::get).filter(Objects::nonNull).forEach(groups::add);
       }
     }
 
-    for (LocalityGroup lgr : groups) {
-      lgr.getIterator().seek(range, EMPTY_CF_SET, false);
-      hiter.addSource(lgr.getIterator());
-    }
-
     return groups;
   }
 
+  private static Set<ByteSequence> getCfSet(Collection<ByteSequence> columnFamilies) {
+    final Set<ByteSequence> cfSet;
+    if (columnFamilies.isEmpty()) {
+      cfSet = Collections.emptySet();
+    } else {
+      if (columnFamilies instanceof Set<?>) {
+        cfSet = (Set<ByteSequence>) columnFamilies;
+      } else {
+        cfSet = new HashSet<>(columnFamilies);

Review Comment:
   Could do the following, might be better since its a fixed size set that is not later mutated.
   
   ```suggestion
           cfSet = Set.copyOf(columnFamilies);
   ```



-- 
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 pull request #2811: Refactor hot methods

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

   > Looking at the method in 'Key' the method that performs the equality check on a byte array `private static boolean isEqual(byte[] a1, byte[] a2)` is already optimized for Accumulo row comparisons, checking the last few bytes first and does not seem to need additional optimization.
   
   Is this in relation to #2812? If so, do you think its worth investigating rearranging some of the comparisons made as mentioned in your comments [here](https://github.com/apache/accumulo/issues/2812#issuecomment-1188111321), or do you think #2812 should be closed?


-- 
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] milleruntime commented on a diff in pull request #2811: Refactor hot methods

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


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java:
##########
@@ -1202,33 +1202,25 @@ public void flush(TInfo tinfo, TCredentials credentials, String lock, String tab
       throw new RuntimeException(e);
     }
 
-    ArrayList<Tablet> tabletsToFlush = new ArrayList<>();
-
     KeyExtent ke = new KeyExtent(TableId.of(tableId), ByteBufferUtil.toText(endRow),
         ByteBufferUtil.toText(startRow));
 
-    for (Tablet tablet : server.getOnlineTablets().values()) {
-      if (ke.overlaps(tablet.getExtent())) {
-        tabletsToFlush.add(tablet);
-      }
-    }
-
-    Long flushID = null;
+    final Long[] flushID = {null};

Review Comment:
   I think this workaround to make the object final could give you a ConcurrentModificationException or some type of concurrency problems. They made it a compile time error for a reason, this answer here gives a good explanation https://stackoverflow.com/questions/34865383/variable-used-in-lambda-expression-should-be-final-or-effectively-final
   
   I am not sure how the equivalent logic could be accomplished with a stream. It may be better to use a traditional for loop.



-- 
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 #2811: Refactor hot methods

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


##########
core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java:
##########
@@ -174,40 +168,13 @@ public void readFields(DataInput in) throws IOException {
       fieldsPrefixed = 0;
     }
 
-    byte[] row, cf, cq, cv;
-    long ts;
-
-    if ((fieldsSame & ROW_SAME) == ROW_SAME) {
-      row = prevKey.getRowData().toArray();
-    } else if ((fieldsPrefixed & ROW_COMMON_PREFIX) == ROW_COMMON_PREFIX) {
-      row = readPrefix(in, prevKey.getRowData());
-    } else {
-      row = read(in);
-    }
+    final byte[] row, cf, cq, cv;
+    final long ts;
 
-    if ((fieldsSame & CF_SAME) == CF_SAME) {
-      cf = prevKey.getColumnFamilyData().toArray();
-    } else if ((fieldsPrefixed & CF_COMMON_PREFIX) == CF_COMMON_PREFIX) {
-      cf = readPrefix(in, prevKey.getColumnFamilyData());
-    } else {
-      cf = read(in);
-    }
-
-    if ((fieldsSame & CQ_SAME) == CQ_SAME) {
-      cq = prevKey.getColumnQualifierData().toArray();
-    } else if ((fieldsPrefixed & CQ_COMMON_PREFIX) == CQ_COMMON_PREFIX) {
-      cq = readPrefix(in, prevKey.getColumnQualifierData());
-    } else {
-      cq = read(in);
-    }
-
-    if ((fieldsSame & CV_SAME) == CV_SAME) {
-      cv = prevKey.getColumnVisibilityData().toArray();
-    } else if ((fieldsPrefixed & CV_COMMON_PREFIX) == CV_COMMON_PREFIX) {
-      cv = readPrefix(in, prevKey.getColumnVisibilityData());
-    } else {
-      cv = read(in);
-    }
+    row = getData(in, ROW_SAME, ROW_COMMON_PREFIX, () -> prevKey.getRowData());
+    cf = getData(in, CF_SAME, CF_COMMON_PREFIX, () -> prevKey.getColumnFamilyData());
+    cq = getData(in, CQ_SAME, CQ_COMMON_PREFIX, () -> prevKey.getColumnQualifierData());
+    cv = getData(in, CV_SAME, CV_COMMON_PREFIX, () -> prevKey.getColumnVisibilityData());

Review Comment:
   Reverted in [9a042ab](https://github.com/apache/accumulo/pull/2811/commits/9a042ab29a668c1e8a8d036ebf724f95ceedd8c6)



-- 
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] keith-turner commented on a diff in pull request #2811: Refactor hot methods

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2811:
URL: https://github.com/apache/accumulo/pull/2811#discussion_r982437354


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/scan/LookupTask.java:
##########
@@ -95,9 +95,14 @@ public void run() {
       // check the time so that the read ahead thread is not monopolized
       while (iter.hasNext() && bytesAdded < maxResultsSize
           && (System.currentTimeMillis() - startTime) < maxScanTime) {
-        Entry<KeyExtent,List<Range>> entry = iter.next();
-        KeyExtent extent = entry.getKey();
-        List<Range> ranges = entry.getValue();
+        final KeyExtent extent;
+        final List<Range> ranges;
+        {
+          final Entry<KeyExtent,List<Range>> entry = iter.next();
+          extent = entry.getKey();
+          ranges = entry.getValue();
+        }

Review Comment:
   > Yes. Maybe its not worth doing as it might cause confusion? What do you think?
   
   When I first saw it I had no idea why it was there.  I looked at it for a bit and guessed why it might be there. I think its an interesting technique and it reminds me of how in golang a function can return zero or more things.  Where in java a functions can only return zero or one things.   I really like how golang works.   To bad we can't write the following in java
   
   ```java
   (entry,ranges) = iter.next();
   ```
   
   Anyway, I am not completely sure if it should say or go, but I am leaning towards removing it.   One thing that helped me think through this is asking would this be a good practice throughout the code?   It seems like this technique could add a lot of noise that may make the code harder to read if used frequently, so that's a reason not to use it elsewhere and here.  The tight scoping could help avoid some bugs in larger methods, but maybe if we have a large method we just need to make it smaller like you have done for methods in this PR.



-- 
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] EdColeman commented on pull request #2811: Refactor hot methods

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

   Looking at the method in 'Key' the method that performs the equality check on a byte array  `private static boolean isEqual(byte[] a1, byte[] a2)` is already optimized for Accumulo row comparisons, checking the last few bytes first and does not seem to need additional optimization.


-- 
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 #2811: Refactor hot methods

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


##########
core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java:
##########
@@ -174,40 +168,13 @@ public void readFields(DataInput in) throws IOException {
       fieldsPrefixed = 0;
     }
 
-    byte[] row, cf, cq, cv;
-    long ts;
-
-    if ((fieldsSame & ROW_SAME) == ROW_SAME) {
-      row = prevKey.getRowData().toArray();
-    } else if ((fieldsPrefixed & ROW_COMMON_PREFIX) == ROW_COMMON_PREFIX) {
-      row = readPrefix(in, prevKey.getRowData());
-    } else {
-      row = read(in);
-    }
+    final byte[] row, cf, cq, cv;
+    final long ts;
 
-    if ((fieldsSame & CF_SAME) == CF_SAME) {
-      cf = prevKey.getColumnFamilyData().toArray();
-    } else if ((fieldsPrefixed & CF_COMMON_PREFIX) == CF_COMMON_PREFIX) {
-      cf = readPrefix(in, prevKey.getColumnFamilyData());
-    } else {
-      cf = read(in);
-    }
-
-    if ((fieldsSame & CQ_SAME) == CQ_SAME) {
-      cq = prevKey.getColumnQualifierData().toArray();
-    } else if ((fieldsPrefixed & CQ_COMMON_PREFIX) == CQ_COMMON_PREFIX) {
-      cq = readPrefix(in, prevKey.getColumnQualifierData());
-    } else {
-      cq = read(in);
-    }
-
-    if ((fieldsSame & CV_SAME) == CV_SAME) {
-      cv = prevKey.getColumnVisibilityData().toArray();
-    } else if ((fieldsPrefixed & CV_COMMON_PREFIX) == CV_COMMON_PREFIX) {
-      cv = readPrefix(in, prevKey.getColumnVisibilityData());
-    } else {
-      cv = read(in);
-    }
+    row = getData(in, ROW_SAME, ROW_COMMON_PREFIX, () -> prevKey.getRowData());
+    cf = getData(in, CF_SAME, CF_COMMON_PREFIX, () -> prevKey.getColumnFamilyData());
+    cq = getData(in, CQ_SAME, CQ_COMMON_PREFIX, () -> prevKey.getColumnQualifierData());
+    cv = getData(in, CV_SAME, CV_COMMON_PREFIX, () -> prevKey.getColumnVisibilityData());

Review Comment:
   Oh, nevermind. I think I see the test failures. It looks like you can't use `variable::method` on a null variable, but `() -> variable.method()` works fine, so long as it doesn't actually get executed. That's a bit weird, but I'm okay leaving this as-is if it works around a test failure.



-- 
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 #2811: Refactor hot methods

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


##########
core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java:
##########
@@ -174,40 +168,13 @@ public void readFields(DataInput in) throws IOException {
       fieldsPrefixed = 0;
     }
 
-    byte[] row, cf, cq, cv;
-    long ts;
-
-    if ((fieldsSame & ROW_SAME) == ROW_SAME) {
-      row = prevKey.getRowData().toArray();
-    } else if ((fieldsPrefixed & ROW_COMMON_PREFIX) == ROW_COMMON_PREFIX) {
-      row = readPrefix(in, prevKey.getRowData());
-    } else {
-      row = read(in);
-    }
+    final byte[] row, cf, cq, cv;
+    final long ts;
 
-    if ((fieldsSame & CF_SAME) == CF_SAME) {
-      cf = prevKey.getColumnFamilyData().toArray();
-    } else if ((fieldsPrefixed & CF_COMMON_PREFIX) == CF_COMMON_PREFIX) {
-      cf = readPrefix(in, prevKey.getColumnFamilyData());
-    } else {
-      cf = read(in);
-    }
-
-    if ((fieldsSame & CQ_SAME) == CQ_SAME) {
-      cq = prevKey.getColumnQualifierData().toArray();
-    } else if ((fieldsPrefixed & CQ_COMMON_PREFIX) == CQ_COMMON_PREFIX) {
-      cq = readPrefix(in, prevKey.getColumnQualifierData());
-    } else {
-      cq = read(in);
-    }
-
-    if ((fieldsSame & CV_SAME) == CV_SAME) {
-      cv = prevKey.getColumnVisibilityData().toArray();
-    } else if ((fieldsPrefixed & CV_COMMON_PREFIX) == CV_COMMON_PREFIX) {
-      cv = readPrefix(in, prevKey.getColumnVisibilityData());
-    } else {
-      cv = read(in);
-    }
+    row = getData(in, ROW_SAME, ROW_COMMON_PREFIX, () -> prevKey.getRowData());
+    cf = getData(in, CF_SAME, CF_COMMON_PREFIX, () -> prevKey.getColumnFamilyData());
+    cq = getData(in, CQ_SAME, CQ_COMMON_PREFIX, () -> prevKey.getColumnQualifierData());
+    cv = getData(in, CV_SAME, CV_COMMON_PREFIX, () -> prevKey.getColumnVisibilityData());

Review Comment:
   Can you provide more details on the test failures this causes? It seems odd that it would cause test failures, since this should just be syntactic sugar and not a functional change.



-- 
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] keith-turner commented on a diff in pull request #2811: Refactor hot methods

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2811:
URL: https://github.com/apache/accumulo/pull/2811#discussion_r983761529


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java:
##########
@@ -396,80 +396,89 @@ Set<StoredTabletFile> getCandidates(Set<StoredTabletFile> currFiles, CompactionK
           if (isCompactionStratConfigured)
             return Set.of();
 
-          switch (selectStatus) {
-            case NOT_ACTIVE:
-            case CANCELED: {
-              Set<StoredTabletFile> candidates = new HashSet<>(currFiles);
-              candidates.removeAll(allCompactingFiles);
-              return Collections.unmodifiableSet(candidates);
-            }
-            case NEW:
-            case SELECTING:
-              return Set.of();
-            case SELECTED: {
-              Set<StoredTabletFile> candidates = new HashSet<>(currFiles);
-              candidates.removeAll(allCompactingFiles);
-              if (getNanoTime() - selectedTimeNanos < selectedExpirationDuration.toNanos()) {
-                candidates.removeAll(selectedFiles);
-              }
-              return Collections.unmodifiableSet(candidates);
-            }
-            case RESERVED: {
-              Set<StoredTabletFile> candidates = new HashSet<>(currFiles);
-              candidates.removeAll(allCompactingFiles);
-              candidates.removeAll(selectedFiles);
-              return Collections.unmodifiableSet(candidates);
-            }
-            default:
-              throw new AssertionError();
-          }
+          return handleSystemCompaction(currFiles);
         }
         case SELECTOR:
           // intentional fall through
         case USER:
-          switch (selectStatus) {
-            case NOT_ACTIVE:
-            case NEW:
-            case SELECTING:
-            case CANCELED:
-              return Set.of();
-            case SELECTED:
-            case RESERVED: {
-              if (selectKind == kind) {
-                Set<StoredTabletFile> candidates = new HashSet<>(selectedFiles);
-                candidates.removeAll(allCompactingFiles);
-                candidates = Collections.unmodifiableSet(candidates);
-                Preconditions.checkState(currFiles.containsAll(candidates),
-                    "selected files not in all files %s %s", candidates, currFiles);
-                return candidates;
-              } else {
-                return Set.of();
-              }
-            }
-            default:
-              throw new AssertionError();
-          }
+          return handleUserSelectorCompaction(currFiles, kind);
         case CHOP: {
-          switch (chopStatus) {
-            case NOT_ACTIVE:
-            case SELECTING:
-            case MARKING:
-              return Set.of();
-            case SELECTED: {
-              if (selectStatus == FileSelectionStatus.NEW
-                  || selectStatus == FileSelectionStatus.SELECTING)
-                return Set.of();
-
-              var filesToChop = getFilesToChop(currFiles);
-              filesToChop.removeAll(allCompactingFiles);
-              if (selectStatus == FileSelectionStatus.SELECTED
-                  || selectStatus == FileSelectionStatus.RESERVED)
-                filesToChop.removeAll(selectedFiles);
-              return Collections.unmodifiableSet(filesToChop);
-            }
-            default:
-              throw new AssertionError();
+          return handleChopCompaction(currFiles);
+        }
+        default:
+          throw new AssertionError();
+      }
+    }
+
+    private Set<StoredTabletFile> handleChopCompaction(Set<StoredTabletFile> currFiles) {
+      switch (chopStatus) {
+        case NOT_ACTIVE:
+        case SELECTING:
+        case MARKING:
+          return Set.of();
+        case SELECTED: {
+          if (selectStatus == FileSelectionStatus.NEW
+              || selectStatus == FileSelectionStatus.SELECTING)
+            return Set.of();
+
+          var filesToChop = getFilesToChop(currFiles);
+          filesToChop.removeAll(allCompactingFiles);
+          if (selectStatus == FileSelectionStatus.SELECTED
+              || selectStatus == FileSelectionStatus.RESERVED)
+            filesToChop.removeAll(selectedFiles);
+          return Collections.unmodifiableSet(filesToChop);
+        }
+        default:
+          throw new AssertionError();
+      }
+    }
+
+    private Set<StoredTabletFile> handleUserSelectorCompaction(Set<StoredTabletFile> currFiles,
+        CompactionKind kind) {
+      switch (selectStatus) {
+        case NOT_ACTIVE:
+        case NEW:
+        case SELECTING:
+        case CANCELED:
+          return Set.of();
+        case SELECTED:
+        case RESERVED: {
+          if (selectKind == kind) {
+            Set<StoredTabletFile> candidates = Sets.difference(selectedFiles, allCompactingFiles);
+            Preconditions.checkState(currFiles.containsAll(candidates),
+                "selected files not in all files %s %s", candidates, currFiles);
+            return Collections.unmodifiableSet(candidates);

Review Comment:
   > I added Set.copyOf() to one other spot so take a look and see what you think.
   
   yeah that needed the same change, I missed 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] DomGarguilo commented on a diff in pull request #2811: Refactor hot methods

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


##########
core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java:
##########
@@ -221,6 +188,17 @@ public void readFields(DataInput in) throws IOException {
     this.prevKey = this.key;
   }
 
+  private byte[] getData(DataInput in, byte same, byte commonPrefix, Supplier<ByteSequence> data)

Review Comment:
   addressed in c86dac5



-- 
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 pull request #2811: Refactor hot methods

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

   > A few more methods were found ([details](https://github.com/apache/accumulo/issues/1099#issuecomment-1198342829)). I'll look into those now, before this is merged.
   
   Never mind. Took a look and I don't think those methods will be changed in this PR. This PR is ready for review.


-- 
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] milleruntime commented on a diff in pull request #2811: Refactor hot methods

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


##########
core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/LocalityGroupIterator.java:
##########
@@ -163,26 +164,14 @@ static final Collection<LocalityGroup> _seek(HeapIterator hiter, LocalityGroupCo
       Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
     hiter.clear();
 
-    Set<ByteSequence> cfSet;
-    if (columnFamilies.isEmpty()) {
-      cfSet = Collections.emptySet();
-    } else {
-      if (columnFamilies instanceof Set<?>) {
-        cfSet = (Set<ByteSequence>) columnFamilies;
-      } else {
-        cfSet = new HashSet<>();
-        cfSet.addAll(columnFamilies);
-      }
-    }
+    final Set<ByteSequence> cfSet = getCfSet(columnFamilies);
 
     // determine the set of groups to use
-    Collection<LocalityGroup> groups = Collections.emptyList();
+    final Collection<LocalityGroup> groups;
 
     // if no column families specified, then include all groups unless !inclusive
     if (cfSet.isEmpty()) {
-      if (!inclusive) {
-        groups = lgContext.groups;
-      }
+      groups = inclusive ? List.of() : lgContext.groups;
     } else {
       groups = new HashSet<>();

Review Comment:
   You might be able to create a method for the work done in this else. Something like getGroups().



-- 
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 pull request #2811: Refactor hot methods

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

   In 89a2f32 I added [the equals method described above](https://github.com/apache/accumulo/pull/2811#issuecomment-1198615354). This updated method does not appear on the hot methods list.


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