You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by "dlmarion (via GitHub)" <gi...@apache.org> on 2023/03/01 19:09:31 UTC

[GitHub] [accumulo] dlmarion opened a new pull request, #3220: Add ondemand table state

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

   This commit adds a new table state called ondemand, which is currently treated like offline. Follow-on commits will need to request that ondemand tablets be hosted for live ingest / immediate scans, modify the tablet servers to unhost ondemand tablets, implement bulk-import into ondemand tables, and more.
   
   
   Closes #3210 


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

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

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


[GitHub] [accumulo] dlmarion commented on a diff in pull request #3220: Add ondemand table state

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


##########
server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java:
##########
@@ -685,8 +685,8 @@ public boolean hasSystemPermission(TCredentials credentials, String user,
       "action: %s; targetTable: %s:%s";
 
   @Override
-  public boolean canOnlineOfflineTable(TCredentials credentials, TableId tableId, FateOperation op,
-      NamespaceId namespaceId) throws ThriftSecurityException {
+  public boolean canOnlineOfflineOnDemandTable(TCredentials credentials, TableId tableId,

Review Comment:
   I agree, I was thinking something similar.



-- 
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 pull request #3220: Add ondemand table state

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

   I was looking at dlmarion/accumulo#37 and I like the on demand column.  Looking at it I was thinking for an ondemand table that the tablet location cache will need to cache tablets with and without locations.  This is so that eventual scans can work against tablets without a location like the changes in #3143 for offline tables.
   
   If the tablet location cache will eventually hold tablets w/o a location, then we could possibly leverage that.  Below is some pseudo code I wrote to help me explore this concept and think through it at a high level.  
   
   ```java
    
    interface ClientLoadRequestProcessor {
          /**
           * Makes a call to one or more managers to load tablets for an ondemand table, knows how to partition 
           * tablets to managers.  May ingore extents it was recently asked to load.
           */
          void  loadTablets(Set<KeyExtent> extents); 
    }
    
    
    interface TabletLocator {
         /**
          * Maps ranges to tablets, partitioning the tablets into hosted and unhosted sets.  Unhosted means the 
          * tablet does not have a location.
          */
         List<Range> locateTablets(List<Range> ranges, Map<...> hostedTablets, Set<KeyExtent> unhostedTablets);
         
         void invalidateExtents(Set<KeyExtent> extents);
    }
    
    
    
   /* Maybe this code would  be in the tablet locator impl or batch scanner impl, not sure of best way to organize 
    * code ATM. I am slightly leaning twoards putting it in TabletLocator like you did in 37, but pulling the code to 
    * make load request to managers out of tablet locator.  
    * /
    class XYZ {
    
       // The following vrs should hang off the client context, just putting it here to make things shorter
        ClientLoadRequestProcessor tabletLoader;
        TabletLocator locator;
    
        // Maps all given ranges to tablets with locations.  This is would be used by batch scanners doing 
        // immediate scans.
        private Map<...> lookupLocations(List<Range> ranges) {
          while(true){
            var hosted = new Map();
            var unhosted = new Set<KeyExtent>();
            
            locator.locatorTablets(ranges, hosted, unhosted);
        
            if(unhosted.isEmpty()) {
               // all ranges were mapped to tablets with a location
               return hosted;
            }
            
            var tableState = getTableState();
            
           if(tableState == ONDEMAND) {
   
            tabletLoader.loadTablets(unhosted);
   
           } else if(tableState != ONLINE) {
             // if table is offline or deleted then tablets will never come online.
             //TODO invalidate cache if table deleted
              throw new Exception();
           }
   
            // TODO sleep with backoff
   
            // this will force the cache to reread these from the metadata table on the next request 
            locator.invalidateExtents(unhosted); 
          }
        }
    }
   
   ```
   


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

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

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


[GitHub] [accumulo] dlmarion commented on pull request #3220: Add ondemand table state

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

   `verify` stage is failing due to Maven 3.9.0 issue mentioned in Slack. It passes locally.


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

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

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


[GitHub] [accumulo] dlmarion commented on a diff in pull request #3220: Add ondemand table state

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


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -1479,36 +1463,90 @@ public void online(String tableName)
     online(tableName, false);
   }
 
-  @Override
-  public void online(String tableName, boolean wait)
+  private void changeTableState(String tableName, boolean wait, TableState newState)
       throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     EXISTING_TABLE_NAME.validate(tableName);
 
     TableId tableId = context.getTableId(tableName);
-    /**
-     * ACCUMULO-4574 if table is already online return without executing fate operation.
-     */
-    if (isOnline(tableName)) {
-      if (wait) {
-        waitForTableStateTransition(tableId, TableState.ONLINE);
-      }
-      return;
+
+    FateOperation op = null;
+    switch (newState) {
+      case OFFLINE:
+        op = FateOperation.TABLE_OFFLINE;
+        break;
+      case ONDEMAND:
+        op = FateOperation.TABLE_ONDEMAND;
+        if (tableName.equals(MetadataTable.NAME) || tableName.equals(RootTable.NAME)) {

Review Comment:
   Yeah, I was just copying what was there, but I did notice that it was not consistent. Root and Metadata should always be online, right? Enforce client side that OFFLINE and ONDEMAND does *not* work for those tables.
   



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

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

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


[GitHub] [accumulo] dlmarion commented on pull request #3220: Add ondemand table state

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

   I don't know how many more states we could add. We have online and offline, ondemand is a hybrid of those two. I see what you are saying though. That is something that could be done in a 3.x version before we add this.


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

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

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


[GitHub] [accumulo] dlmarion commented on a diff in pull request #3220: Add ondemand table state

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


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -1511,6 +1511,60 @@ public void online(String tableName, boolean wait)
     }
   }
 
+  @Override
+  public boolean isOnDemand(String tableName) throws AccumuloException, TableNotFoundException {
+    EXISTING_TABLE_NAME.validate(tableName);
+
+    if (tableName.equals(MetadataTable.NAME) || tableName.equals(RootTable.NAME)) {
+      return false;
+    }
+
+    TableId tableId = context.getTableId(tableName);
+    TableState expectedState = context.getTableState(tableId, true);
+    return expectedState == TableState.ONDEMAND;
+  }
+
+  @Override
+  public void onDemand(String tableName)
+      throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    onDemand(tableName, false);
+  }
+
+  @Override
+  public void onDemand(String tableName, boolean wait)

Review Comment:
   Good catch, I wasn't thinking about that when writing this, just trying to create something functional. Updated in 836b227.



-- 
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 a diff in pull request #3220: Add ondemand table state

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


##########
core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java:
##########
@@ -1086,4 +1086,40 @@ default TimeType getTimeType(String tableName) throws TableNotFoundException {
     throw new UnsupportedOperationException();
   }
 
+  /**

Review Comment:
   Is there a good place to summarize what ondemands means vs on or offline?  I assume the web site would have a more lengthy description, but having something in the javadocs might - would not want to see it on every ondemand method,  but somewhere? (and can be done 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


[GitHub] [accumulo] dlmarion commented on pull request #3220: Add ondemand table state

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

   Closing this in favor of #3250 


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

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

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


[GitHub] [accumulo] dlmarion commented on pull request #3220: Add ondemand table state

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

     There are now 3 PRs up for the OnDemand tablet feature, and I think it's functional although I'm sure there are more edge cases to test. 
   
     This PR introduces new API methods to put a table in an onDemand state (vs online or offline). The implementation of this creates a Fate operation that ends up running the `ChangeTableState` Fate op. The Manager has been updated such that tablets for a table in the onDemand state will not be hosted, just like an offline table.
   
     [37](https://github.com/dlmarion/accumulo/pull/37) builds on this and contains modifications for onDemand tablets to be brought online. This is done via the `TabletLocator`, when the client is unable to find a Tablet location and it's an onDemand table, then it makes a call to a new Thrift API method called `bringOnDemandTabletOnline`. When this method is called, the implementation (`TabletClientHandler.bringOnDemandTabletsOnline`) inserts an "ondemand" column into the metadata for the tablet. The `TabletStateChangeIterator` has been modified to return the `TabletLocationState` via the `MetaDataTableScanner` so that the `TabletGroupWatcher` running in the Manager will see the tablet as unassigned and will assign it using the normal tablet assignment process.
   
     [38](https://github.com/dlmarion/accumulo/pull/38) builds on 37 and contains modifications for onDemand tablets to be unloaded. This is done by a new Thread in the TabletServer that calls the new method `evaluateOnDemandTabletsForUnload` at some interval (defined by a new property). `evaluateOnDemandTabletsForUnload` uses an implementation of a new SPI `OnDemandTabletUnloader` class (default implementation provided) to determine which online tablets for an onDemand tablet should be unloaded. For each tablet returned, the method removes the "ondemand" column from the metadata table for the tablet. Like in 37, the `TabletStateChangeIterator` returns the information to the `TabletGroupWatcher` running in the Manager and it will unload the tablet using the normal process.


-- 
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] cshannon commented on a diff in pull request #3220: Add ondemand table state

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


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -1511,6 +1511,60 @@ public void online(String tableName, boolean wait)
     }
   }
 
+  @Override
+  public boolean isOnDemand(String tableName) throws AccumuloException, TableNotFoundException {
+    EXISTING_TABLE_NAME.validate(tableName);
+
+    if (tableName.equals(MetadataTable.NAME) || tableName.equals(RootTable.NAME)) {
+      return false;
+    }
+
+    TableId tableId = context.getTableId(tableName);
+    TableState expectedState = context.getTableState(tableId, true);
+    return expectedState == TableState.ONDEMAND;
+  }
+
+  @Override
+  public void onDemand(String tableName)
+      throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    onDemand(tableName, false);
+  }
+
+  @Override
+  public void onDemand(String tableName, boolean wait)

Review Comment:
   This method seems like an (almost) identical copy of the online(tablename, wait) method except for the check for MetadataTable/Root. I am thinking we should be able to re-use the logic here by creating a helper method and just pass in the state that we are transitioning too so we don't need to copy/paste.



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

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

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


[GitHub] [accumulo] dlmarion closed pull request #3220: Add ondemand table state

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion closed pull request #3220: Add ondemand table state
URL: https://github.com/apache/accumulo/pull/3220


-- 
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 #3220: Add ondemand table state

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


##########
server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java:
##########
@@ -418,6 +417,32 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe
             goalMessage);
         break;
       }
+      case TABLE_ONDEMAND: {
+        TableOperation tableOp = TableOperation.ONDEMAND;
+        validateArgumentCount(arguments, tableOp, 1);
+        final var tableId = validateTableIdArgument(arguments.get(0), tableOp,
+            NOT_ROOT_TABLE_ID.and(NOT_METADATA_TABLE_ID));

Review Comment:
   Feel like this check should be consistent with TABLE_OFFLINE which only seems to check if its root.  Seems like TABLE_OFFLINE should also check if its metadata table.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -1479,36 +1463,90 @@ public void online(String tableName)
     online(tableName, false);
   }
 
-  @Override
-  public void online(String tableName, boolean wait)
+  private void changeTableState(String tableName, boolean wait, TableState newState)
       throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     EXISTING_TABLE_NAME.validate(tableName);
 
     TableId tableId = context.getTableId(tableName);
-    /**
-     * ACCUMULO-4574 if table is already online return without executing fate operation.
-     */
-    if (isOnline(tableName)) {
-      if (wait) {
-        waitForTableStateTransition(tableId, TableState.ONLINE);
-      }
-      return;
+
+    FateOperation op = null;
+    switch (newState) {
+      case OFFLINE:
+        op = FateOperation.TABLE_OFFLINE;
+        break;
+      case ONDEMAND:
+        op = FateOperation.TABLE_ONDEMAND;
+        if (tableName.equals(MetadataTable.NAME) || tableName.equals(RootTable.NAME)) {

Review Comment:
   OFFLINE has similar restrictions but does not do these checks client side. For consistency, I think neither or both OFFLINE and ONDEMAND should do the checks client side.



-- 
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 a diff in pull request #3220: Add ondemand table state

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


##########
server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java:
##########
@@ -685,8 +685,8 @@ public boolean hasSystemPermission(TCredentials credentials, String user,
       "action: %s; targetTable: %s:%s";
 
   @Override
-  public boolean canOnlineOfflineTable(TCredentials credentials, TableId tableId, FateOperation op,
-      NamespaceId namespaceId) throws ThriftSecurityException {
+  public boolean canOnlineOfflineOnDemandTable(TCredentials credentials, TableId tableId,

Review Comment:
   Could these methods be combined into something like `canChangeTableState(...)` for the method name?  The permission could be a general allowed to change state and not for a specific state.  



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

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

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


[GitHub] [accumulo] dlmarion commented on pull request #3220: Add ondemand table state

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

   If you want to test this yourself, you can:
   
     1. Build Accumulo from PR [38](https://github.com/dlmarion/accumulo/pull/38)
     2. Set the following properties in accumulo.properties:
     ```
       manager.tablet.watcher.interval=15s
       table.ondemand.tablet.unloader.interval=1m
       table.custom.ondemand.unloader.inactivity.threshold=120000
     ```
     3. Start up local instance
     4. Log into Accumulo shell, and do the following
     ```
     createtable test
     ```
     > **Note**
     > notice test and 1 tablet in Monitor
     > notice "loc" column in tablet metadata
     ```
     ondemand test
     ```
     > **Note**
     > notice 0 tablets in monitor
     > notice "loc" missing from tablet metadata
     ```
     insert a b c d
     ```
     > **Note**
     > notice shell command wait for tablet to be hosted
     > notice 1 tablet in Monitor
     > notice "loc" column in tablet metadata
     > notice "ondemand" column in tablet metadata
     ```
     addsplits m - test
     ```
     > **Note**
     > notice 1 tablet in Monitor
     > notice "loc" in tablet metadata for source tablet only
     > notice "ondemand" column in tablet metadata for source tablet only
    ```
    sleep 240
    ```
     > **Note**
     > table.custom.ondemand.unloader.inactivity.threshold is set to 2m, waiting 4m should unload the tablet
     > notice 0 tablets in monitor
     > notice "loc" column missing in tablet metadata
     > notice "ondemand" column missing in tablet metadata
     ```
     insert a b c d
    ```
     > **Note**
     > notice shell command wait for tablet to be hosted
     > notice 1 tablet in Monitor
     > notice "loc" column in tablet metadata
     > notice "ondemand" column in tablet metadata
   
     5. Stop Accumulo
     6. Start Accumulo
     > **Note**
     > test tablet had "ondemand" column on shutdown, so it gets re-hosted
     > notice 1 tablet in Monitor
     > notice "loc" column in tablet metadata
     > notice "ondemand" column in tablet metadata
     7.  Wait a few minutes for tablet to be unloaded
     > **Note**
     > notice 0 tablets in monitor
     > notice "loc" column missing in tablet metadata
     > notice "ondemand" column missing in tablet metadata


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

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

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


[GitHub] [accumulo] dlmarion commented on a diff in pull request #3220: Add ondemand table state

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


##########
server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java:
##########
@@ -685,8 +685,8 @@ public boolean hasSystemPermission(TCredentials credentials, String user,
       "action: %s; targetTable: %s:%s";
 
   @Override
-  public boolean canOnlineOfflineTable(TCredentials credentials, TableId tableId, FateOperation op,
-      NamespaceId namespaceId) throws ThriftSecurityException {
+  public boolean canOnlineOfflineOnDemandTable(TCredentials credentials, TableId tableId,

Review Comment:
   Renamed to `canChangeTableState`



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

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

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


[GitHub] [accumulo] dlmarion commented on a diff in pull request #3220: Add ondemand table state

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


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -1479,36 +1463,90 @@ public void online(String tableName)
     online(tableName, false);
   }
 
-  @Override
-  public void online(String tableName, boolean wait)
+  private void changeTableState(String tableName, boolean wait, TableState newState)
       throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     EXISTING_TABLE_NAME.validate(tableName);
 
     TableId tableId = context.getTableId(tableName);
-    /**
-     * ACCUMULO-4574 if table is already online return without executing fate operation.
-     */
-    if (isOnline(tableName)) {
-      if (wait) {
-        waitForTableStateTransition(tableId, TableState.ONLINE);
-      }
-      return;
+
+    FateOperation op = null;
+    switch (newState) {
+      case OFFLINE:
+        op = FateOperation.TABLE_OFFLINE;
+        break;
+      case ONDEMAND:
+        op = FateOperation.TABLE_ONDEMAND;
+        if (tableName.equals(MetadataTable.NAME) || tableName.equals(RootTable.NAME)) {

Review Comment:
   Updated in 0751fbf



##########
server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java:
##########
@@ -418,6 +417,32 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe
             goalMessage);
         break;
       }
+      case TABLE_ONDEMAND: {
+        TableOperation tableOp = TableOperation.ONDEMAND;
+        validateArgumentCount(arguments, tableOp, 1);
+        final var tableId = validateTableIdArgument(arguments.get(0), tableOp,
+            NOT_ROOT_TABLE_ID.and(NOT_METADATA_TABLE_ID));

Review Comment:
   Updated in 0751fbf



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