This is an automated email from the ASF dual-hosted git repository.

dlmarion pushed a commit to branch elasticity
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/elasticity by this push:
     new 232f3074d9 Removed tserver.last.location.mode property (#3926)
232f3074d9 is described below

commit 232f3074d9fd483977e40cfd8b5bd50aef92b375
Author: Dave Marion <dlmar...@apache.org>
AuthorDate: Tue Nov 21 15:34:20 2023 -0500

    Removed tserver.last.location.mode property (#3926)
    
    Removed tserver.last.location.mode property, last location is
    always set to where the tablet was last assigned. This property
    no longer made sense as compactions no longer occur in the
    TabletServer.
    
    Fixes #3596
---
 .../org/apache/accumulo/core/conf/Property.java    |  11 ---
 .../apache/accumulo/core/conf/PropertyType.java    |   3 -
 .../accumulo/core/conf/PropertyTypeTest.java       |   6 --
 .../manager/state/AbstractTabletStateStore.java    |  26 +++--
 .../accumulo/server/util/ManagerMetadataUtil.java  |  65 ------------
 .../state/AbstractTabletStateStoreTest.java}       |  42 ++++----
 .../java/org/apache/accumulo/manager/Manager.java  |   5 +-
 .../accumulo/manager/TabletGroupWatcher.java       |   2 +-
 .../org/apache/accumulo/tserver/tablet/Tablet.java |   4 -
 .../test/functional/CompactLocationModeIT.java     | 110 ---------------------
 ...signLocationModeIT.java => LastLocationIT.java} |  15 +--
 11 files changed, 42 insertions(+), 247 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java 
b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 29710d76a8..a3f734164e 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -717,17 +717,6 @@ public enum Property {
       "The number of threads on each tablet server available to retrieve"
           + " summary data, that is not currently in cache, from RFiles.",
       "2.0.0"),
-  // ELASTICITY_TODO reconsider this prop and its impl now that compactions 
never happen on tservers
-  TSERV_LAST_LOCATION_MODE("tserver.last.location.mode", "compaction",
-      PropertyType.LAST_LOCATION_MODE,
-      "Describes how the system will record the 'last' location for tablets, 
which can be used for"
-          + " assigning them when a cluster restarts. If 'compaction' is the 
mode, then the system"
-          + " will record the location where the tablet's most recent 
compaction occurred. If"
-          + " 'assignment' is the mode, then the most recently assigned 
location will be recorded."
-          + " The manager.startup.tserver properties might also need to be set 
to ensure the"
-          + " tserver is available before tablets are initially assigned if 
the 'last' location is"
-          + " to be used.",
-      "2.1.1"),
   
TSERV_ONDEMAND_UNLOADER_INTERVAL("tserver.ondemand.tablet.unloader.interval", 
"10m",
       PropertyType.TIMEDURATION,
       "The interval at which the TabletServer will check if on-demand tablets 
can be unloaded.",
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java 
b/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
index 8696ae295f..20e12d7a60 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
@@ -131,9 +131,6 @@ public enum PropertyType {
   GC_POST_ACTION("gc_post_action", in(true, null, "none", "flush", "compact"),
       "One of 'none', 'flush', or 'compact'."),
 
-  LAST_LOCATION_MODE("last_location_mode", in(true, null, "assignment", 
"compaction"),
-      "Defines how to update the last location.  One of 'assignment', or 
'compaction'."),
-
   STRING("string", x -> true,
       "An arbitrary string of characters whose format is unspecified and"
           + " interpreted based on the context of the property to which it 
applies."),
diff --git 
a/core/src/test/java/org/apache/accumulo/core/conf/PropertyTypeTest.java 
b/core/src/test/java/org/apache/accumulo/core/conf/PropertyTypeTest.java
index 315543ade9..4ffd1ec8cb 100644
--- a/core/src/test/java/org/apache/accumulo/core/conf/PropertyTypeTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/conf/PropertyTypeTest.java
@@ -145,12 +145,6 @@ public class PropertyTypeTest extends WithTestNames {
     invalid("", "other");
   }
 
-  @Test
-  public void testTypeLAST_LOCATION_MODE() {
-    valid(null, "compaction", "assignment");
-    invalid("", "other");
-  }
-
   @Test
   public void testTypeFRACTION() {
     valid(null, "1", "0", "1.0", "25%", "2.5%", "10.2E-3", "10.2E-3%", ".3");
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStore.java
 
b/server/base/src/main/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStore.java
index 1f7a9b057b..526e636f32 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStore.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStore.java
@@ -28,20 +28,18 @@ import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.Ample.ConditionalResult.Status;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.server.util.ManagerMetadataUtil;
 import org.apache.hadoop.fs.Path;
 
 import com.google.common.base.Preconditions;
 
 public abstract class AbstractTabletStateStore implements TabletStateStore {
 
-  private final ClientContext context;
   private final Ample ample;
 
   protected AbstractTabletStateStore(ClientContext context) {
-    this.context = context;
     this.ample = context.getAmple();
   }
 
@@ -54,8 +52,7 @@ public abstract class AbstractTabletStateStore implements 
TabletStateStore {
             .putLocation(TabletMetadata.Location.current(assignment.server))
             
.deleteLocation(TabletMetadata.Location.future(assignment.server)).deleteSuspension();
 
-        ManagerMetadataUtil.updateLastForAssignmentMode(context, 
conditionalMutator,
-            assignment.server, assignment.lastLocation);
+        updateLastLocation(conditionalMutator, assignment.server, 
assignment.lastLocation);
 
         conditionalMutator.submit(tabletMetadata -> {
           
Preconditions.checkArgument(tabletMetadata.getExtent().equals(assignment.tablet));
@@ -131,8 +128,7 @@ public abstract class AbstractTabletStateStore implements 
TabletStateStore {
 
         if (tm.hasCurrent()) {
 
-          ManagerMetadataUtil.updateLastForAssignmentMode(context, 
tabletMutator,
-              tm.getLocation().getServerInstance(), tm.getLast());
+          updateLastLocation(tabletMutator, 
tm.getLocation().getServerInstance(), tm.getLast());
           tabletMutator.deleteLocation(tm.getLocation());
           if (logsForDeadServers != null) {
             List<Path> logs = 
logsForDeadServers.get(tm.getLocation().getServerInstance());
@@ -166,4 +162,20 @@ public abstract class AbstractTabletStateStore implements 
TabletStateStore {
       throw new DistributedStoreException(ex);
     }
   }
+
+  protected static void updateLastLocation(Ample.TabletUpdates<?> 
tabletMutator,
+      TServerInstance location, Location lastLocation) {
+    Preconditions.checkArgument(
+        lastLocation == null || lastLocation.getType() == 
TabletMetadata.LocationType.LAST);
+    Location newLocation = Location.last(location);
+    if (lastLocation != null) {
+      if (!lastLocation.equals(newLocation)) {
+        tabletMutator.deleteLocation(lastLocation);
+        tabletMutator.putLocation(newLocation);
+      }
+    } else {
+      tabletMutator.putLocation(newLocation);
+    }
+  }
+
 }
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java
 
b/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java
index 69e7da8e6e..e252512843 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java
@@ -29,9 +29,7 @@ import java.util.TreeMap;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.ScannerImpl;
-import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
@@ -43,7 +41,6 @@ import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.ReferencedTabletFile;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.Ample.TabletMutator;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
@@ -57,8 +54,6 @@ import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Preconditions;
-
 public class ManagerMetadataUtil {
 
   private static final Logger log = 
LoggerFactory.getLogger(ManagerMetadataUtil.class);
@@ -177,64 +172,4 @@ public class ManagerMetadataUtil {
     }
   }
 
-  /**
-   * Update the last location if the location mode is "assignment". This will 
delete the previous
-   * last location if needed and set the new last location
-   *
-   * @param context The server context
-   * @param tabletMutator The mutator being built
-   * @param location The new location
-   * @param lastLocation The previous last location, which may be null
-   */
-  public static void updateLastForAssignmentMode(ClientContext context,
-      Ample.TabletUpdates<?> tabletMutator, TServerInstance location, Location 
lastLocation) {
-    Preconditions.checkArgument(
-        lastLocation == null || lastLocation.getType() == 
TabletMetadata.LocationType.LAST);
-
-    // if the location mode is assignment, then preserve the current location 
in the last
-    // location value
-    if 
("assignment".equals(context.getConfiguration().get(Property.TSERV_LAST_LOCATION_MODE)))
 {
-      ManagerMetadataUtil.updateLocation(tabletMutator, lastLocation, 
Location.last(location));
-    }
-  }
-
-  /**
-   * Update the last location if the location mode is "compaction". This will 
delete the previous
-   * last location if needed and set the new last location
-   *
-   * @param context The server context
-   * @param tabletMutator The mutator being built
-   * @param lastLocation The last location
-   * @param tServerInstance The server address
-   */
-  public static void updateLastForCompactionMode(ClientContext context,
-      Ample.ConditionalTabletMutator tabletMutator, Location lastLocation,
-      TServerInstance tServerInstance) {
-    // if the location mode is 'compaction', then preserve the current 
compaction location in the
-    // last location value
-    if 
("compaction".equals(context.getConfiguration().get(Property.TSERV_LAST_LOCATION_MODE)))
 {
-      Location newLocation = Location.last(tServerInstance);
-      updateLocation(tabletMutator, lastLocation, newLocation);
-    }
-  }
-
-  /**
-   * Update the location, deleting the previous location if needed
-   *
-   * @param tabletMutator The mutator being built
-   * @param previousLocation The location (may be null)
-   * @param newLocation The new location
-   */
-  private static void updateLocation(Ample.TabletUpdates<?> tabletMutator,
-      Location previousLocation, Location newLocation) {
-    // ELASTICITY_TODO pending #3301, update this code to use conditional 
mutations
-    if (previousLocation != null) {
-      if (!previousLocation.equals(newLocation)) {
-        tabletMutator.deleteLocation(previousLocation);
-        tabletMutator.putLocation(newLocation);
-      }
-    } else {
-      tabletMutator.putLocation(newLocation);
-    }
-  }
 }
diff --git 
a/server/base/src/test/java/org/apache/accumulo/server/util/ManagerMetadataUtilTest.java
 
b/server/base/src/test/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStoreTest.java
similarity index 66%
rename from 
server/base/src/test/java/org/apache/accumulo/server/util/ManagerMetadataUtilTest.java
rename to 
server/base/src/test/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStoreTest.java
index 748dffaa65..b9d8ddcbdd 100644
--- 
a/server/base/src/test/java/org/apache/accumulo/server/util/ManagerMetadataUtilTest.java
+++ 
b/server/base/src/test/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStoreTest.java
@@ -16,13 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.server.util;
+package org.apache.accumulo.server.manager.state;
 
 import static org.junit.jupiter.api.Assertions.assertThrows;
 
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
@@ -30,10 +27,8 @@ import org.easymock.EasyMock;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
-public class ManagerMetadataUtilTest {
+public class AbstractTabletStateStoreTest {
 
-  private AccumuloConfiguration conf;
-  private ClientContext context;
   private Ample.TabletMutator tabletMutator;
   private final TServerInstance server1 = new 
TServerInstance("127.0.0.1:10000", 0);
   private final Location last1 = Location.last(server1);
@@ -42,61 +37,58 @@ public class ManagerMetadataUtilTest {
 
   @BeforeEach
   public void before() {
-    conf = EasyMock.createMock(AccumuloConfiguration.class);
-    
EasyMock.expect(conf.get(Property.TSERV_LAST_LOCATION_MODE)).andReturn("assignment");
-    context = EasyMock.createMock(ClientContext.class);
-    EasyMock.expect(context.getConfiguration()).andReturn(conf).once();
     tabletMutator = EasyMock.createMock(Ample.TabletMutator.class);
   }
 
   @Test
-  public void testUpdateLastForAssignmentModeNullLastLocation() {
+  public void testUpdateLastNullLastLocation() {
     // Expect a put of last1 as the previous value
     
EasyMock.expect(tabletMutator.putLocation(last1)).andReturn(tabletMutator).once();
-    EasyMock.replay(conf, context, tabletMutator);
+    EasyMock.replay(tabletMutator);
 
     // Pass in a null last location value. There should be a call to
     // tabletMutator.putLocation of last 1 but no deletion as lastLocation is 
null
-    ManagerMetadataUtil.updateLastForAssignmentMode(context, tabletMutator, 
server1, null);
-    EasyMock.verify(conf, context, tabletMutator);
+    AbstractTabletStateStore.updateLastLocation(tabletMutator, server1, null);
+    EasyMock.verify(tabletMutator);
   }
 
   @Test
-  public void testUpdateLastForAssignModeInvalidType() {
+  public void testUpdateLastInvalidType() {
+    EasyMock.replay(tabletMutator);
     assertThrows(IllegalArgumentException.class, () -> {
       // Should throw an IllegalArgumentException as the lastLocation is not 
LocationType.LAST
-      ManagerMetadataUtil.updateLastForAssignmentMode(context, tabletMutator, 
server1,
+      AbstractTabletStateStore.updateLastLocation(tabletMutator, server1,
           Location.current(server1));
     });
   }
 
   @Test
-  public void testUpdateLastForAssignModeLastLocationSame() {
-    EasyMock.replay(conf, context, tabletMutator);
+  public void testUpdateLastLocationSame() {
+    EasyMock.replay(tabletMutator);
 
     // Pass in a last location value that matches the new value of server 1
     // There should be no call to tabletMutator.putLocation or 
tabletMutator.deleteLocation
     // as the locations are equal so no expects() are defined and any method 
calls would
     // throw an error
-    ManagerMetadataUtil.updateLastForAssignmentMode(context, tabletMutator, 
server1, last1);
-    EasyMock.verify(conf, context, tabletMutator);
+    AbstractTabletStateStore.updateLastLocation(tabletMutator, server1, last1);
+    EasyMock.verify(tabletMutator);
   }
 
   @Test
-  public void testUpdateLastForAssignModeLastLocationDifferent() {
+  public void testUpdateLastLocationDifferent() {
     // Expect a delete of last1 as we are providing that as the previous last 
location
     // which is different from server 2 location
     
EasyMock.expect(tabletMutator.deleteLocation(last1)).andReturn(tabletMutator).once();
     
EasyMock.expect(tabletMutator.putLocation(last2)).andReturn(tabletMutator).once();
 
-    EasyMock.replay(conf, context, tabletMutator);
+    EasyMock.replay(tabletMutator);
 
     // Pass in last1 as the last location value.
     // There should be no read from Ample as we provided a value as an argument
     // There should be a call to tabletMutator.putLocation and 
tabletMutator.deleteLocation
     // as the last location is being updated as last1 does not match server 2
-    ManagerMetadataUtil.updateLastForAssignmentMode(context, tabletMutator, 
server2, last1);
-    EasyMock.verify(conf, context, tabletMutator);
+    AbstractTabletStateStore.updateLastLocation(tabletMutator, server2, last1);
+    EasyMock.verify(tabletMutator);
   }
 
 }
diff --git 
a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java 
b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
index 45f17c412c..21da4e3bad 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
@@ -1647,7 +1647,10 @@ public class Manager extends AbstractServer
     AssignmentParamsImpl params =
         AssignmentParamsImpl.fromThrift(currentStatus, currentTServerGroups,
             unassigned.entrySet().stream().collect(HashMap::new,
-                (m, e) -> m.put(e.getKey(), e.getValue().getServerInstance()), 
Map::putAll),
+                (m, e) -> m.put(e.getKey(),
+                    e.getValue().getLastLocation() == null ? null
+                        : e.getValue().getLastLocation().getServerInstance()),
+                Map::putAll),
             assignedOut);
     tabletBalancer.getAssignments(params);
   }
diff --git 
a/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java
 
b/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java
index fd6b4ddb86..c93d2eebc6 100644
--- 
a/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java
+++ 
b/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java
@@ -403,7 +403,7 @@ abstract class TabletGroupWatcher extends 
AccumuloDaemonThread {
       }
 
       final Set<ManagementAction> actions = mti.getActions();
-      if (tm.isFutureAndCurrentLocationSet()) {
+      if (actions.contains(ManagementAction.BAD_STATE) && 
tm.isFutureAndCurrentLocationSet()) {
         throw new BadLocationStateException(
             tm.getExtent() + " is both assigned and hosted, which should never 
happen: " + this,
             tm.getExtent().toMetaRow());
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 11c37ae5ea..8eadceb9b8 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -82,7 +82,6 @@ import org.apache.accumulo.server.problems.ProblemType;
 import 
org.apache.accumulo.server.tablets.ConditionCheckerContext.ConditionChecker;
 import org.apache.accumulo.server.tablets.TabletNameGenerator;
 import org.apache.accumulo.server.tablets.TabletTime;
-import org.apache.accumulo.server.util.ManagerMetadataUtil;
 import org.apache.accumulo.server.util.MetadataTableUtil;
 import org.apache.accumulo.tserver.InMemoryMap;
 import org.apache.accumulo.tserver.MinorCompactionReason;
@@ -1332,9 +1331,6 @@ public class Tablet extends TabletBase {
       if (dfv.getNumEntries() > 0) {
         tablet.putFile(newDatafile, dfv);
         newFile = Optional.of(newDatafile.insert());
-
-        ManagerMetadataUtil.updateLastForCompactionMode(getContext(), tablet, 
lastLocation,
-            tabletServer.getTabletSession());
       }
 
       var newTime = tabletTime.getMetadataTime(maxCommittedTime);
diff --git 
a/test/src/main/java/org/apache/accumulo/test/functional/CompactLocationModeIT.java
 
b/test/src/main/java/org/apache/accumulo/test/functional/CompactLocationModeIT.java
deleted file mode 100644
index eb672574f0..0000000000
--- 
a/test/src/main/java/org/apache/accumulo/test/functional/CompactLocationModeIT.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.test.functional;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.time.Duration;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.admin.NewTableConfiguration;
-import org.apache.accumulo.core.client.admin.TabletHostingGoal;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.metadata.schema.TabletMetadata;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.hadoop.conf.Configuration;
-import org.junit.jupiter.api.Test;
-
-public class CompactLocationModeIT extends ConfigurableMacBase {
-
-  @Override
-  protected Duration defaultTimeout() {
-    return Duration.ofMinutes(2);
-  }
-
-  @Override
-  public void configure(MiniAccumuloConfigImpl cfg, Configuration fsConf) {
-    cfg.setProperty(Property.TSERV_LAST_LOCATION_MODE, "compaction");
-  }
-
-  @Test
-  public void test() throws Exception {
-    try (AccumuloClient c =
-        getCluster().createAccumuloClient("root", new 
PasswordToken(ROOT_PASSWORD))) {
-      String tableName = super.getUniqueNames(1)[0];
-      NewTableConfiguration ntc =
-          new 
NewTableConfiguration().withInitialHostingGoal(TabletHostingGoal.ALWAYS);
-      c.tableOperations().create(tableName, ntc);
-      String tableId = c.tableOperations().tableIdMap().get(tableName);
-      // wait for the table to be online
-      TabletMetadata newTablet;
-      do {
-        UtilWaitThread.sleep(250);
-        newTablet = ManagerAssignmentIT.getTabletMetadata(c, tableId, null);
-      } while (!newTablet.hasCurrent());
-      assertNull(newTablet.getLast());
-      assertNotNull(newTablet.getLocation());
-
-      // put something in it
-      try (BatchWriter bw = c.createBatchWriter(tableName)) {
-        Mutation m = new Mutation("a");
-        m.put("b", "c", "d");
-        bw.addMutation(m);
-      }
-      // assert that the default mode is "compact"
-      assertEquals("compaction", 
c.instanceOperations().getSystemConfiguration()
-          .get(Property.TSERV_LAST_LOCATION_MODE.getKey()));
-
-      // no last location should be set yet
-      TabletMetadata unflushed = ManagerAssignmentIT.getTabletMetadata(c, 
tableId, null);
-      assertEquals(newTablet.getLocation().getHostPort(), 
unflushed.getLocation().getHostPort());
-      assertNull(unflushed.getLast());
-      assertTrue(newTablet.hasCurrent());
-
-      // This should give it a last location if the mode is being used 
correctly
-      c.tableOperations().flush(tableName, null, null, true);
-
-      TabletMetadata flushed = ManagerAssignmentIT.getTabletMetadata(c, 
tableId, null);
-      assertEquals(newTablet.getLocation().getHostPort(), 
flushed.getLocation().getHostPort());
-      assertEquals(flushed.getLocation().getHostPort(), 
flushed.getLast().getHostPort());
-      assertTrue(newTablet.hasCurrent());
-
-      // take the tablet offline
-      c.tableOperations().offline(tableName, true);
-      TabletMetadata offline = ManagerAssignmentIT.getTabletMetadata(c, 
tableId, null);
-      assertNull(offline.getLocation());
-      assertEquals(flushed.getLocation().getHostPort(), 
offline.getLast().getHostPort());
-
-      // put it back online, should have the same last location
-      c.tableOperations().online(tableName, true);
-      TabletMetadata online = ManagerAssignmentIT.getTabletMetadata(c, 
tableId, null);
-      assertTrue(online.hasCurrent());
-      assertNotNull(online.getLocation());
-      assertEquals(offline.getLast().getHostPort(), 
online.getLast().getHostPort());
-    }
-  }
-
-}
diff --git 
a/test/src/main/java/org/apache/accumulo/test/functional/AssignLocationModeIT.java
 b/test/src/main/java/org/apache/accumulo/test/functional/LastLocationIT.java
similarity index 85%
rename from 
test/src/main/java/org/apache/accumulo/test/functional/AssignLocationModeIT.java
rename to 
test/src/main/java/org/apache/accumulo/test/functional/LastLocationIT.java
index 92805e572a..4721b70ebc 100644
--- 
a/test/src/main/java/org/apache/accumulo/test/functional/AssignLocationModeIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/LastLocationIT.java
@@ -31,28 +31,18 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.client.admin.TabletHostingGoal;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.hadoop.conf.Configuration;
-import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 
-@Disabled // ELASTICITY_TODO
-public class AssignLocationModeIT extends ConfigurableMacBase {
+public class LastLocationIT extends ConfigurableMacBase {
 
   @Override
   protected Duration defaultTimeout() {
     return Duration.ofMinutes(2);
   }
 
-  @Override
-  public void configure(MiniAccumuloConfigImpl cfg, Configuration fsConf) {
-    cfg.setProperty(Property.TSERV_LAST_LOCATION_MODE, "assignment");
-  }
-
   @Test
   public void test() throws Exception {
     try (AccumuloClient c =
@@ -77,9 +67,6 @@ public class AssignLocationModeIT extends ConfigurableMacBase 
{
         m.put("b", "c", "d");
         bw.addMutation(m);
       }
-      // assert that the default mode is "assign"
-      assertEquals("assignment", 
c.instanceOperations().getSystemConfiguration()
-          .get(Property.TSERV_LAST_LOCATION_MODE.getKey()));
 
       // last location should not be set yet
       TabletMetadata unflushed = ManagerAssignmentIT.getTabletMetadata(c, 
tableId, null);

Reply via email to