You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by "jmark99 (via GitHub)" <gi...@apache.org> on 2023/11/21 20:18:00 UTC

[PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

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

   This update replaces uses of hosting goal related nomenclature with tablet availability related nomenclature instead.
   
   The hosting values ALWAYS, ONDEMAND, and NEVER have been updated to HOSTED, ONDEMAND, and UNHOSTED. References to hosting goal and other goal related uses have been replaced with tablet availability and/or availability equivalents. Attempts have been made to update the corresponding JavaDocs along the way.
   
   Given the wide usage of the hosting goal nomenclature, there may be some references that have been missed. These can be updated either during code review or with an update later as they are found. By the same token, if there have been too many references, they can be reverted.
   
   Several classes have been renamed in the process. These include:
   
   - `HostingGoalForTablet` -> `AvailabilityForTablet`
   - `TabletHostingGoal` -> `TabletAvailability`
   - `THostingGoal` -> `TTabletAvailability`
   - `SetHostingGoal` -> `SetTabletAvailability`
     -  the containing package was updated from `.../tableOps/goal` to `.../tableOps/availability`
   - `GetTabletHostingGoalCommand` -> `GetAvailabilityCommand`
   - `SetTabletHostingGoalCommand` -> `SetAvailabilityCommand`
   
   The metadata schema was modified from `hosting:goal` to use `hosting:availability`.
   
   Will close #3859


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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "jmark99 (via GitHub)" <gi...@apache.org>.
jmark99 merged PR #3969:
URL: https://github.com/apache/accumulo/pull/3969


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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "EdColeman (via GitHub)" <gi...@apache.org>.
EdColeman commented on code in PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#discussion_r1473232040


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -1907,9 +1907,10 @@ public Locations locate(String tableName, Collection<Range> ranges)
 
       if (foundOnDemandTabletInRange.get()) {
         throw new AccumuloException(
-            "TableOperations.locate() only works with tablets that have a hosting goal of "
-                + TabletHostingGoal.ALWAYS + ". Tablets with other hosting goals were seen.  table:"
-                + tableName + " table id:" + tableId);
+            "TableOperations.locate() only works with tablets that have an availability of "
+                + TabletAvailability.HOSTED
+                + ". Tablets with other availability's were seen.  table:" + tableName

Review Comment:
   ```suggestion
                   + ". Tablets with other availabilities were seen.  table:" + tableName
   ```



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCacheImpl.java:
##########
@@ -130,9 +130,10 @@ private class LockCheckerSession {
     private final HashSet<Pair<String,String>> invalidLocks = new HashSet<>();
 
     private CachedTablet checkLock(CachedTablet tl) {
-      // the goal of this class is to minimize calls out to lockChecker under that assumption that
-      // its a resource synchronized among many threads... want to
-      // avoid fine grained synchronization when binning lots of mutations or ranges... remember
+      // the availability of this class is to minimize calls out to lockChecker under that

Review Comment:
   ```suggestion
         // the goal of this class is to minimize calls out to lockChecker under that
   ```



##########
core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java:
##########
@@ -433,8 +433,9 @@ interface TabletMutator extends TabletUpdates<TabletMutator> {
   /**
    * A tablet operation is a mutually exclusive action that is running against a tablet. Its very
    * important that every conditional mutation specifies requirements about operations in order to
-   * satisfy the mutual exclusion goal. This interface forces those requirements to specified by
-   * making it the only choice available before specifying other tablet requirements or mutations.
+   * satisfy the mutual exclusion availability. This interface forces those requirements to

Review Comment:
   ```suggestion
      * satisfy the mutual exclusion goal. This interface forces those requirements to
   ```



##########
core/src/main/java/org/apache/accumulo/core/util/Merge.java:
##########
@@ -83,7 +83,7 @@ public Text convert(String value) {
   static class Opts extends ClientOpts {
     @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
     String tableName;
-    @Parameter(names = {"-s", "--size"}, description = "merge goal size",
+    @Parameter(names = {"-s", "--size"}, description = "merge availability size",

Review Comment:
   ```suggestion
       @Parameter(names = {"-s", "--size"}, description = "merge goal size",
   ```



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCacheImpl.java:
##########
@@ -130,9 +130,10 @@ private class LockCheckerSession {
     private final HashSet<Pair<String,String>> invalidLocks = new HashSet<>();
 
     private CachedTablet checkLock(CachedTablet tl) {
-      // the goal of this class is to minimize calls out to lockChecker under that assumption that
-      // its a resource synchronized among many threads... want to
-      // avoid fine grained synchronization when binning lots of mutations or ranges... remember
+      // the availability of this class is to minimize calls out to lockChecker under that
+      // assumption that
+      // it's a resource synchronized among many threads... want to

Review Comment:
   ```suggestion
         // it is a resource synchronized among many threads... want to
   ```



##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/PopulateMetadataTable.java:
##########
@@ -173,24 +172,24 @@ public Repo<Manager> call(long tid, Manager manager) throws Exception {
               m = new Mutation(metadataRow);
               ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value(tabletDir));
               currentRow = metadataRow;
-              sawHostingGoal = false;
+              sawTabletAvailability = false;
             }
 
-            if (HostingColumnFamily.GOAL_COLUMN.hasColumns(key)) {
-              sawHostingGoal = true;
+            if (TabletColumnFamily.AVAILABILITY_COLUMN.hasColumns(key)) {
+              sawTabletAvailability = true;
             }
             m.put(key.getColumnFamily(), cq, val);
 
             if (endRow == null && TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key)) {
 
-              if (!sawHostingGoal) {
-                // add a default hosting goal
-                HostingColumnFamily.GOAL_COLUMN.put(m,
-                    TabletHostingGoalUtil.toValue(TabletHostingGoal.ONDEMAND));
+              if (!sawTabletAvailability) {
+                // add a default tablet availability
+                TabletColumnFamily.AVAILABILITY_COLUMN.put(m,
+                    TabletAvailabilityUtil.toValue(TabletAvailability.ONDEMAND));
               }
 
               mbw.addMutation(m);
-              break; // its the last column in the last row
+              break; // it's the last column in the last row

Review Comment:
   ```suggestion
                 break; // it is the last column in the last row
   ```



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -1919,9 +1920,9 @@ public Locations locate(String tableName, Collection<Range> ranges)
 
         if (foundOnDemandTabletInRange.get()) {
           throw new AccumuloException(
-              "TableOperations.locate() only works with tablets that have a hosting goal of "
-                  + TabletHostingGoal.ALWAYS
-                  + ". Tablets with other hosting goals were seen.  table:" + tableName
+              "TableOperations.locate() only works with tablets that have a tablet availability of "
+                  + TabletAvailability.HOSTED
+                  + ". Tablets with other availability's were seen.  table:" + tableName

Review Comment:
   ```suggestion
                     + ". Tablets with other availabilities were seen.  table:" + tableName
   ```



##########
core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java:
##########
@@ -450,7 +450,7 @@ private class DeriverImpl<T> implements Deriver<T> {
     }
 
     /**
-     * This method was written with the goal of avoiding thread contention and minimizing
+     * This method was written with the availability of avoiding thread contention and minimizing

Review Comment:
   ```suggestion
        * This method was written with the goal of avoiding thread contention and minimizing
   ```



##########
core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java:
##########
@@ -50,7 +50,7 @@
 
 /**
  * A simple tablet balancer that attempts to spread tablets out evenly across all available tablet
- * servers. The goal is to achieve the same number of tablets on each tablet server.
+ * servers. The availability is to achieve the same number of tablets on each tablet server.

Review Comment:
   ```suggestion
    * servers. The goal is to achieve the same number of tablets on each tablet server.
   ```



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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#issuecomment-1828641370

   > > My only lingering thought is about the schema. Instead of `hosting:availability`, I'm wondering if this would be a `~tab` CF thing, so something more like `~tab:availability`.
   > 
   > Moving `hosting:availability` to `~tab:availability` seems reasonable to me.
   > 
   > There is also a `hosting:requested` entry, which is currently part of the `hostingColumnFamily` class within MetadataSchema. Do you think the `requested` qualifier should also be moved to the `~tab` family or remain as the sole member of the `hostingColumnFamily` class in MetadataSchema?
   
   I'm not sure if there's value in having a separate CF, but I think `~tab:requestToHost` would suffice.


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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "jmark99 (via GitHub)" <gi...@apache.org>.
jmark99 commented on PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#issuecomment-1921645531

   @dlmarion corrected the two failing ITs. The host:availability check had to be updated to ~tab:availability in both instances.


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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "jmark99 (via GitHub)" <gi...@apache.org>.
jmark99 commented on code in PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#discussion_r1473427165


##########
core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java:
##########
@@ -50,7 +50,7 @@
 
 /**
  * A simple tablet balancer that attempts to spread tablets out evenly across all available tablet
- * servers. The goal is to achieve the same number of tablets on each tablet server.
+ * servers. The availability is to achieve the same number of tablets on each tablet server.

Review Comment:
   Already reverted



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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "jmark99 (via GitHub)" <gi...@apache.org>.
jmark99 commented on code in PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#discussion_r1474436008


##########
core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java:
##########
@@ -729,14 +729,14 @@ void online(String tableName, boolean wait)
       throws AccumuloSecurityException, AccumuloException, TableNotFoundException;
 
   /**
-   * Check if a table is online through its current goal state only. Could run into issues if the
-   * current state of the table is in between states. If you require a specific state, call
+   * Check if a table is online through its current availability state only. Could run into issues
+   * if the current state of the table is in between states. If you require a specific state, call
    * <code>online(tableName, true)</code> or <code>offline(tableName, true)</code>, this will wait
    * until the table reaches the desired state before proceeding.
    *
    * @param tableName the table to check if online
    * @throws AccumuloException when there is a general accumulo error
-   * @return true if table's goal state is online
+   * @return true if table's availability state is online
    *

Review Comment:
   Corrected



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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "jmark99 (via GitHub)" <gi...@apache.org>.
jmark99 commented on code in PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#discussion_r1474714698


##########
core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java:
##########
@@ -447,11 +446,19 @@ public static <E extends Entry<Key,Value>> TabletMetadata convertRow(Iterator<E>
             "Input contains more than one row : " + row + " " + key.getRowData());
       }
 
-      switch (fam.toString()) {
+      switch (fam) {
         case TabletColumnFamily.STR_NAME:
-          if (qual.equals(PREV_ROW_QUAL)) {
-            te.prevEndRow = TabletColumnFamily.decodePrevEndRow(kv.getValue());
-            te.sawPrevEndRow = true;
+          switch (qual) {
+            case PREV_ROW_QUAL:
+              te.prevEndRow = TabletColumnFamily.decodePrevEndRow(kv.getValue());
+              te.sawPrevEndRow = true;
+              break;
+            case AVAILABILITY_QUAL:
+              te.availability = TabletAvailabilityUtil.fromValue(kv.getValue());
+              break;
+            case REQUESTED_QUAL:
+              te.onDemandHostingRequested = true;
+              break;

Review Comment:
   I'm going to create a separate PR for this after this merge.



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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "jmark99 (via GitHub)" <gi...@apache.org>.
jmark99 commented on code in PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#discussion_r1474444419


##########
core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java:
##########
@@ -433,8 +433,9 @@ interface TabletMutator extends TabletUpdates<TabletMutator> {
   /**
    * A tablet operation is a mutually exclusive action that is running against a tablet. Its very
    * important that every conditional mutation specifies requirements about operations in order to
-   * satisfy the mutual exclusion goal. This interface forces those requirements to specified by
-   * making it the only choice available before specifying other tablet requirements or mutations.
+   * satisfy the mutual exclusion availability. This interface forces those requirements to

Review Comment:
   Previously reverted



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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "jmark99 (via GitHub)" <gi...@apache.org>.
jmark99 commented on PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#issuecomment-1921511867

   @dlmarion I'll take a look at those tests.


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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "jmark99 (via GitHub)" <gi...@apache.org>.
jmark99 commented on code in PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#discussion_r1473426468


##########
core/src/main/java/org/apache/accumulo/core/util/Merge.java:
##########
@@ -83,7 +83,7 @@ public Text convert(String value) {
   static class Opts extends ClientOpts {
     @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
     String tableName;
-    @Parameter(names = {"-s", "--size"}, description = "merge goal size",
+    @Parameter(names = {"-s", "--size"}, description = "merge availability size",

Review Comment:
   Previously reverted



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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#issuecomment-1919932526

   I kicked off a full IT build for 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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "jmark99 (via GitHub)" <gi...@apache.org>.
jmark99 commented on code in PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#discussion_r1474443212


##########
core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java:
##########
@@ -101,15 +101,15 @@ public static void split(KeyExtent parent, SortedSet<Text> splits) {
   }
 
   /**
-   * Called when a tablet's current assignment state does not match the goal state.
+   * Called when a tablet's current assignment state does not match the availability state.

Review Comment:
   Done



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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "ddanielr (via GitHub)" <gi...@apache.org>.
ddanielr commented on PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#issuecomment-1823099229

   > @ctubbsii I resolved the two conflicts, but I'm still getting a thrift error during the workflow checks. I'm not very familiar with thrift or the thrift workflow. Can you suggest what is needed to resolve the thrift error that is being thrown?
   
   @jmark99 The build report shows that two files in `thrift-gen-java` changed. 
   ```
   Thrift build changed files in worktree:
    M core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/TTabletAvailability.java
    M core/src/main/thrift-gen-java/org/apache/accumulo/core/tablet/thrift/TabletManagementClientService.java
    ```
    
    I think you need to generate the new thrift class code locally and commit those files, then this build step should pass.


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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "jmark99 (via GitHub)" <gi...@apache.org>.
jmark99 commented on PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#issuecomment-1825714029

   > My only lingering thought is about the schema. Instead of `hosting:availability`, I'm wondering if this would be a `~tab` CF thing, so something more like `~tab:availability`.
   
   Moving `hosting:availability` to `~tab:availability` seems reasonable to me. 
   
   There is also a `hosting:requested` entry, which is currently part of the `hostingColumnFamily` class within MetadataSchema. Do you think the `requested` qualifier should also be moved to the `~tab` family or remain as the sole member of the `hostingColumnFamily` class in MetadataSchema?


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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#issuecomment-1821797904

   The changes described seem reasonable to me. I have not reviewed the actual code changes, but there seems to be a conflict.
   
   My only lingering thought is about the schema. Instead of `hosting:availability`, I'm wondering if this would be a `~tab` CF thing, so something more like `~tab:availability`.


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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "jmark99 (via GitHub)" <gi...@apache.org>.
jmark99 commented on PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#issuecomment-1846002234

   @ctubbsii updated the columns to remove the HostingColumnFamily and place the availability and requested qualifiers to ~tab:availability and ~tab:requestToHost. Should be ready for review now.


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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "jmark99 (via GitHub)" <gi...@apache.org>.
jmark99 commented on code in PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#discussion_r1474440186


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCacheImpl.java:
##########
@@ -130,9 +130,10 @@ private class LockCheckerSession {
     private final HashSet<Pair<String,String>> invalidLocks = new HashSet<>();
 
     private CachedTablet checkLock(CachedTablet tl) {
-      // the goal of this class is to minimize calls out to lockChecker under that assumption that
-      // its a resource synchronized among many threads... want to
-      // avoid fine grained synchronization when binning lots of mutations or ranges... remember
+      // the availability of this class is to minimize calls out to lockChecker under that

Review Comment:
   Previously reverted



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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on code in PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#discussion_r1473219695


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCache.java:
##########
@@ -76,9 +76,10 @@ public enum LocationNeed {
 
   /**
    * This method allows linear scans to host tablet ahead of time that they may read in the future.
-   * The goal of this method is to allow tablets to request hosting of tablet for a scan before the
-   * scan actually needs it. Below is an example of how this method could work with a scan when
-   * {@code minimumHostAhead=4} is passed and avoid the scan having to wait on tablet hosting.
+   * The availability of this method is to allow tablets to request hosting of tablet for a scan

Review Comment:
   ```suggestion
      * The goal of this method is to allow tablets to request hosting of tablet for a scan
   ```



##########
core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java:
##########
@@ -101,15 +101,15 @@ public static void split(KeyExtent parent, SortedSet<Text> splits) {
   }
 
   /**
-   * Called when a tablet's current assignment state does not match the goal state.
+   * Called when a tablet's current assignment state does not match the availability state.

Review Comment:
   you can revert all changes in this file. These are not associated with tablet availability.



##########
core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java:
##########
@@ -450,7 +450,7 @@ private class DeriverImpl<T> implements Deriver<T> {
     }
 
     /**
-     * This method was written with the goal of avoiding thread contention and minimizing
+     * This method was written with the availability of avoiding thread contention and minimizing

Review Comment:
   ```suggestion
        * This method was written with the goal of avoiding thread contention and minimizing
   ```



##########
core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java:
##########
@@ -50,7 +50,7 @@
 
 /**
  * A simple tablet balancer that attempts to spread tablets out evenly across all available tablet
- * servers. The goal is to achieve the same number of tablets on each tablet server.
+ * servers. The availability is to achieve the same number of tablets on each tablet server.

Review Comment:
   I think you can revert this change.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCacheImpl.java:
##########
@@ -130,9 +130,10 @@ private class LockCheckerSession {
     private final HashSet<Pair<String,String>> invalidLocks = new HashSet<>();
 
     private CachedTablet checkLock(CachedTablet tl) {
-      // the goal of this class is to minimize calls out to lockChecker under that assumption that
-      // its a resource synchronized among many threads... want to
-      // avoid fine grained synchronization when binning lots of mutations or ranges... remember
+      // the availability of this class is to minimize calls out to lockChecker under that

Review Comment:
   ```suggestion
         // the goal of this class is to minimize calls out to lockChecker under that
   ```



##########
core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java:
##########
@@ -729,14 +729,14 @@ void online(String tableName, boolean wait)
       throws AccumuloSecurityException, AccumuloException, TableNotFoundException;
 
   /**
-   * Check if a table is online through its current goal state only. Could run into issues if the
-   * current state of the table is in between states. If you require a specific state, call
+   * Check if a table is online through its current availability state only. Could run into issues
+   * if the current state of the table is in between states. If you require a specific state, call
    * <code>online(tableName, true)</code> or <code>offline(tableName, true)</code>, this will wait
    * until the table reaches the desired state before proceeding.
    *
    * @param tableName the table to check if online
    * @throws AccumuloException when there is a general accumulo error
-   * @return true if table's goal state is online
+   * @return true if table's availability state is online
    *

Review Comment:
   I think `goal` here is referring to the online/offline goal, not the availability. I think these changes can be reverted.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCache.java:
##########
@@ -92,8 +93,8 @@ public enum LocationNeed {
    * read the 9th tablet will cause a request to host the 13th,14th,15th, and 16th tablets.</li>
    * </ol>
    *
-   * In the situation above, the goal is that while we are reading 4 hosted tablets the 4 following
-   * tablets are in the process of being hosted.
+   * In the situation above, the availability is that while we are reading 4 hosted tablets the 4

Review Comment:
   ```suggestion
      * In the situation above, the goal is that while we are reading 4 hosted tablets the 4
   ```



##########
core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java:
##########
@@ -447,11 +446,19 @@ public static <E extends Entry<Key,Value>> TabletMetadata convertRow(Iterator<E>
             "Input contains more than one row : " + row + " " + key.getRowData());
       }
 
-      switch (fam.toString()) {
+      switch (fam) {
         case TabletColumnFamily.STR_NAME:
-          if (qual.equals(PREV_ROW_QUAL)) {
-            te.prevEndRow = TabletColumnFamily.decodePrevEndRow(kv.getValue());
-            te.sawPrevEndRow = true;
+          switch (qual) {
+            case PREV_ROW_QUAL:
+              te.prevEndRow = TabletColumnFamily.decodePrevEndRow(kv.getValue());
+              te.sawPrevEndRow = true;
+              break;
+            case AVAILABILITY_QUAL:
+              te.availability = TabletAvailabilityUtil.fromValue(kv.getValue());
+              break;
+            case REQUESTED_QUAL:
+              te.onDemandHostingRequested = true;
+              break;

Review Comment:
   do you think we should have a default case here that throws an IllegalStateException? This would catch cases where additional columns are added in the future.



##########
minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java:
##########
@@ -142,12 +142,13 @@ public void adminStopAll() throws IOException {
   /**
    * Wrapper around SetGoalState
    *
-   * @param goalState The goal state to set
+   * @param goalState The availability state to set

Review Comment:
   I don't think this is related to tablet availabllity, but the Manager goal state (e.g. STOP)



##########
core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java:
##########
@@ -433,8 +433,9 @@ interface TabletMutator extends TabletUpdates<TabletMutator> {
   /**
    * A tablet operation is a mutually exclusive action that is running against a tablet. Its very
    * important that every conditional mutation specifies requirements about operations in order to
-   * satisfy the mutual exclusion goal. This interface forces those requirements to specified by
-   * making it the only choice available before specifying other tablet requirements or mutations.
+   * satisfy the mutual exclusion availability. This interface forces those requirements to

Review Comment:
   ```suggestion
      * satisfy the mutual exclusion goal. This interface forces those requirements to
   ```



##########
core/src/main/java/org/apache/accumulo/core/util/Merge.java:
##########
@@ -83,7 +83,7 @@ public Text convert(String value) {
   static class Opts extends ClientOpts {
     @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
     String tableName;
-    @Parameter(names = {"-s", "--size"}, description = "merge goal size",
+    @Parameter(names = {"-s", "--size"}, description = "merge availability size",

Review Comment:
   I think you can revert this 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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "jmark99 (via GitHub)" <gi...@apache.org>.
jmark99 commented on code in PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#discussion_r1473425872


##########
minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java:
##########
@@ -142,12 +142,13 @@ public void adminStopAll() throws IOException {
   /**
    * Wrapper around SetGoalState
    *
-   * @param goalState The goal state to set
+   * @param goalState The availability state to set

Review Comment:
   Corrected



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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "jmark99 (via GitHub)" <gi...@apache.org>.
jmark99 commented on PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#issuecomment-1823044062

   @ctubbsii  I resolved the two conflicts, but I'm still getting a thrift error during the workflow checks. I'm not very familiar with thrift or the thrift workflow. Can you suggest what is needed to resolve the thrift error that is being thrown?


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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#issuecomment-1823324078

   @jmark99 If you have the correct version of thrift available (0.17.0 for this branch) on your path, then you just need to run `mvn package -DskipTests -Pthrift`, and check in the changed thrift-generated files. I've gone ahead and done this for you.


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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "jmark99 (via GitHub)" <gi...@apache.org>.
jmark99 commented on code in PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#discussion_r1474441323


##########
core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java:
##########
@@ -450,7 +450,7 @@ private class DeriverImpl<T> implements Deriver<T> {
     }
 
     /**
-     * This method was written with the goal of avoiding thread contention and minimizing
+     * This method was written with the availability of avoiding thread contention and minimizing

Review Comment:
   Previously reverted



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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#issuecomment-1921319477

   In the IT run that I kicked off last night, the following two tests failed:
   
   ```
   org.opentest4j.AssertionFailedError: expected: <1> but was: <0>
   	at org.junit.jupiter.api.AssertionFailureBuilder.build(AssertionFailureBuilder.java:151)
   	at org.junit.jupiter.api.AssertionFailureBuilder.buildAndThrow(AssertionFailureBuilder.java:132)
   	at org.junit.jupiter.api.AssertEquals.failNotEqual(AssertEquals.java:197)
   	at org.junit.jupiter.api.AssertEquals.assertEquals(AssertEquals.java:150)
   	at org.junit.jupiter.api.AssertEquals.assertEquals(AssertEquals.java:145)
   	at org.junit.jupiter.api.Assertions.assertEquals(Assertions.java:531)
   	at org.apache.accumulo.test.shell.ShellCreateTableIT.verifyTableWithTabletAvailability(ShellCreateTableIT.java:684)
   	at org.apache.accumulo.test.shell.ShellCreateTableIT.testCreateTableWithInitialTabletAvailability(ShellCreateTableIT.java:646)
   ```
   
   ```
   org.opentest4j.AssertionFailedError: expected: <true> but was: <false>
   	at org.junit.jupiter.api.AssertionFailureBuilder.build(AssertionFailureBuilder.java:151)
   	at org.junit.jupiter.api.AssertionFailureBuilder.buildAndThrow(AssertionFailureBuilder.java:132)
   	at org.junit.jupiter.api.AssertTrue.failNotTrue(AssertTrue.java:63)
   	at org.junit.jupiter.api.AssertTrue.assertTrue(AssertTrue.java:36)
   	at org.junit.jupiter.api.AssertTrue.assertTrue(AssertTrue.java:31)
   	at org.junit.jupiter.api.Assertions.assertTrue(Assertions.java:183)
   	at org.apache.accumulo.test.shell.ShellServerIT.testGetAvailabilityCommand_DelayedSplits(ShellServerIT.java:1352)
   ```


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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#discussion_r1473657012


##########
core/src/main/java/org/apache/accumulo/core/client/admin/TabletAvailability.java:
##########
@@ -19,19 +19,19 @@
 package org.apache.accumulo.core.client.admin;
 
 /**
- * @see TableOperations#setTabletHostingGoal(String, org.apache.accumulo.core.data.Range,
- *      TabletHostingGoal)
+ * @see TableOperations#setTabletAvailability(String, org.apache.accumulo.core.data.Range,
+ *      TabletAvailability)
  * @since 4.0.0
  */
-public enum TabletHostingGoal {
+public enum TabletAvailability {
 
   // Signifies that a Tablet should always be hosted
-  ALWAYS,
+  HOSTED,
 
   // Signifies that a Tablet should be hosted on client request
   ONDEMAND,
 
   // Signifies that a Tablet should never be hosted

Review Comment:
   ```suggestion
     /**
      * Signifies that a Tablet should never be hosted on a tablet server.
       */
   ```



##########
core/src/main/java/org/apache/accumulo/core/client/admin/TabletAvailability.java:
##########
@@ -19,19 +19,19 @@
 package org.apache.accumulo.core.client.admin;
 
 /**
- * @see TableOperations#setTabletHostingGoal(String, org.apache.accumulo.core.data.Range,
- *      TabletHostingGoal)
+ * @see TableOperations#setTabletAvailability(String, org.apache.accumulo.core.data.Range,
+ *      TabletAvailability)
  * @since 4.0.0
  */
-public enum TabletHostingGoal {
+public enum TabletAvailability {
 
   // Signifies that a Tablet should always be hosted
-  ALWAYS,
+  HOSTED,
 
   // Signifies that a Tablet should be hosted on client request

Review Comment:
   ```suggestion
     /** 
       * Signifies that a Tablet should be hosted on a tablet server when a client needs it.
       */
   ```
   



##########
core/src/main/java/org/apache/accumulo/core/client/admin/TabletAvailability.java:
##########
@@ -19,19 +19,19 @@
 package org.apache.accumulo.core.client.admin;
 
 /**
- * @see TableOperations#setTabletHostingGoal(String, org.apache.accumulo.core.data.Range,
- *      TabletHostingGoal)
+ * @see TableOperations#setTabletAvailability(String, org.apache.accumulo.core.data.Range,
+ *      TabletAvailability)
  * @since 4.0.0
  */
-public enum TabletHostingGoal {
+public enum TabletAvailability {
 
   // Signifies that a Tablet should always be hosted

Review Comment:
   This was an existing problem, but could fix it in this commit.  The comments on the enum are not javadoc format and it would be nice if they were.
   
   ```suggestion
     /** 
      * Signifies that a Tablet should always be hosted on a tablet server.
      */
   ```



##########
core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java:
##########
@@ -447,11 +446,19 @@ public static <E extends Entry<Key,Value>> TabletMetadata convertRow(Iterator<E>
             "Input contains more than one row : " + row + " " + key.getRowData());
       }
 
-      switch (fam.toString()) {
+      switch (fam) {
         case TabletColumnFamily.STR_NAME:
-          if (qual.equals(PREV_ROW_QUAL)) {
-            te.prevEndRow = TabletColumnFamily.decodePrevEndRow(kv.getValue());
-            te.sawPrevEndRow = true;
+          switch (qual) {
+            case PREV_ROW_QUAL:
+              te.prevEndRow = TabletColumnFamily.decodePrevEndRow(kv.getValue());
+              te.sawPrevEndRow = true;
+              break;
+            case AVAILABILITY_QUAL:
+              te.availability = TabletAvailabilityUtil.fromValue(kv.getValue());
+              break;
+            case REQUESTED_QUAL:
+              te.onDemandHostingRequested = true;
+              break;

Review Comment:
   Would be nice to throw an exception here.  If we add it and breaks something we can fix it later.



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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#issuecomment-1911010225

   @jmark99 - I think we should try to get this merged soon instead of letting it linger any longer. My apologies for not giving it the attention it needs. Are you able to get this branch updated from the changes in main?


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


Re: [PR] Use "Tablet Availability" rather than "Hosting Goal" [accumulo]

Posted by "jmark99 (via GitHub)" <gi...@apache.org>.
jmark99 commented on code in PR #3969:
URL: https://github.com/apache/accumulo/pull/3969#discussion_r1475030054


##########
core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java:
##########
@@ -447,11 +446,19 @@ public static <E extends Entry<Key,Value>> TabletMetadata convertRow(Iterator<E>
             "Input contains more than one row : " + row + " " + key.getRowData());
       }
 
-      switch (fam.toString()) {
+      switch (fam) {
         case TabletColumnFamily.STR_NAME:
-          if (qual.equals(PREV_ROW_QUAL)) {
-            te.prevEndRow = TabletColumnFamily.decodePrevEndRow(kv.getValue());
-            te.sawPrevEndRow = true;
+          switch (qual) {
+            case PREV_ROW_QUAL:
+              te.prevEndRow = TabletColumnFamily.decodePrevEndRow(kv.getValue());
+              te.sawPrevEndRow = true;
+              break;
+            case AVAILABILITY_QUAL:
+              te.availability = TabletAvailabilityUtil.fromValue(kv.getValue());
+              break;
+            case REQUESTED_QUAL:
+              te.onDemandHostingRequested = true;
+              break;

Review Comment:
   See #4213 



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