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

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

commit a8f566ef76772fd82143ca5c8054b4bd2d5d0780
Merge: 59a4937fc9 d811807bf1
Author: Keith Turner <ktur...@apache.org>
AuthorDate: Mon Nov 13 11:07:51 2023 -0500

    Merge branch 'main' into elasticity

 .../java/org/apache/accumulo/core/conf/Property.java |  2 +-
 .../accumulo/core/tabletserver/log/LogEntry.java     | 17 +++++------------
 .../core/metadata/schema/TabletMetadataTest.java     | 18 ++++++++----------
 .../accumulo/core/tabletserver/log/LogEntryTest.java | 20 +++++++-------------
 .../manager/state/AbstractTabletStateStore.java      |  2 +-
 .../server/manager/state/TabletManagementTest.java   |  8 ++++----
 .../org/apache/accumulo/tserver/TabletServer.java    |  4 +---
 .../shell/commands/ListTabletsCommandTest.java       |  4 ++--
 .../java/org/apache/accumulo/test/MetaSplitIT.java   |  3 ++-
 .../test/MissingWalHeaderCompletesRecoveryIT.java    |  4 ++--
 .../test/functional/TabletManagementIteratorIT.java  |  4 ++--
 11 files changed, 35 insertions(+), 51 deletions(-)

diff --cc 
core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java
index a2ebe442bf,a3a3d33a45..b890a54299
--- 
a/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java
+++ 
b/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java
@@@ -286,98 -268,4 +284,98 @@@ public class TabletMetadataTest 
      });
      return rowMap;
    }
 +
 +  @Test
 +  public void testBuilder() {
 +    TServerInstance ser1 = new 
TServerInstance(HostAndPort.fromParts("server1", 8555), "s001");
 +
 +    KeyExtent extent = new KeyExtent(TableId.of("5"), new Text("df"), new 
Text("da"));
 +
 +    StoredTabletFile sf1 =
 +        new ReferencedTabletFile(new 
Path("hdfs://nn1/acc/tables/1/t-0001/sf1.rf")).insert();
 +    DataFileValue dfv1 = new DataFileValue(89, 67);
 +
 +    StoredTabletFile sf2 =
 +        new ReferencedTabletFile(new 
Path("hdfs://nn1/acc/tables/1/t-0001/sf2.rf")).insert();
 +    DataFileValue dfv2 = new DataFileValue(890, 670);
 +
 +    ReferencedTabletFile rf1 =
 +        new ReferencedTabletFile(new 
Path("hdfs://nn1/acc/tables/1/t-0001/imp1.rf"));
 +    ReferencedTabletFile rf2 =
 +        new ReferencedTabletFile(new 
Path("hdfs://nn1/acc/tables/1/t-0001/imp2.rf"));
 +
 +    StoredTabletFile sf3 =
 +        new ReferencedTabletFile(new 
Path("hdfs://nn1/acc/tables/1/t-0001/sf3.rf")).insert();
 +    StoredTabletFile sf4 =
 +        new ReferencedTabletFile(new 
Path("hdfs://nn1/acc/tables/1/t-0001/sf4.rf")).insert();
 +
 +    TabletMetadata tm = 
TabletMetadata.builder(extent).putHostingGoal(TabletHostingGoal.NEVER)
 +        .putLocation(Location.future(ser1)).putFile(sf1, dfv1).putFile(sf2, 
dfv2)
 +        .putCompactionId(23).putBulkFile(rf1, 25).putBulkFile(rf2, 
35).putFlushId(27)
 +        
.putDirName("dir1").putScan(sf3).putScan(sf4).putCompacted(17).putCompacted(23)
 +        .build(ECOMP, HOSTING_REQUESTED);
 +
 +    assertEquals(extent, tm.getExtent());
 +    assertEquals(TabletHostingGoal.NEVER, tm.getHostingGoal());
 +    assertEquals(Location.future(ser1), tm.getLocation());
 +    assertEquals(23L, tm.getCompactId().orElse(-1));
 +    assertEquals(27L, tm.getFlushId().orElse(-1));
 +    assertEquals(Map.of(sf1, dfv1, sf2, dfv2), tm.getFilesMap());
 +    assertEquals(Map.of(rf1.insert(), 25L, rf2.insert(), 35L), 
tm.getLoaded());
 +    assertEquals("dir1", tm.getDirName());
 +    assertEquals(Set.of(sf3, sf4), Set.copyOf(tm.getScans()));
 +    assertEquals(Set.of(), tm.getExternalCompactions().keySet());
 +    assertEquals(Set.of(17L, 23L), tm.getCompacted());
 +    assertFalse(tm.getHostingRequested());
 +    assertThrows(IllegalStateException.class, tm::getOperationId);
 +    assertThrows(IllegalStateException.class, tm::getSuspend);
 +    assertThrows(IllegalStateException.class, tm::getTime);
 +
 +    TabletOperationId opid1 = 
TabletOperationId.from(TabletOperationType.SPLITTING, 55);
 +    TabletMetadata tm2 = 
TabletMetadata.builder(extent).putOperation(opid1).build(LOCATION);
 +
 +    assertEquals(extent, tm2.getExtent());
 +    assertEquals(opid1, tm2.getOperationId());
 +    assertNull(tm2.getLocation());
 +    assertThrows(IllegalStateException.class, tm2::getFiles);
 +    assertThrows(IllegalStateException.class, tm2::getHostingGoal);
 +    assertThrows(IllegalStateException.class, tm2::getCompactId);
 +    assertThrows(IllegalStateException.class, tm2::getFlushId);
 +    assertThrows(IllegalStateException.class, tm2::getFiles);
 +    assertThrows(IllegalStateException.class, tm2::getLogs);
 +    assertThrows(IllegalStateException.class, tm2::getLoaded);
 +    assertThrows(IllegalStateException.class, tm2::getDirName);
 +    assertThrows(IllegalStateException.class, tm2::getScans);
 +    assertThrows(IllegalStateException.class, tm2::getExternalCompactions);
 +    assertThrows(IllegalStateException.class, tm2::getHostingRequested);
 +    assertThrows(IllegalStateException.class, tm2::getSelectedFiles);
 +    assertThrows(IllegalStateException.class, tm2::getCompacted);
 +
 +    var ecid1 = ExternalCompactionId.generate(UUID.randomUUID());
 +    ExternalCompactionMetadata ecm = new 
ExternalCompactionMetadata(Set.of(sf1, sf2), rf1, "cid1",
 +        CompactionKind.USER, (short) 3, 
CompactionExecutorIdImpl.externalId("Q1"), true, 99L);
 +
-     LogEntry le1 = new LogEntry(55, "localhost:8020/" + UUID.randomUUID());
-     LogEntry le2 = new LogEntry(57, "localhost:8020/" + UUID.randomUUID());
++    LogEntry le1 = new LogEntry("localhost:8020/" + UUID.randomUUID());
++    LogEntry le2 = new LogEntry("localhost:8020/" + UUID.randomUUID());
 +
 +    SelectedFiles selFiles = new SelectedFiles(Set.of(sf1, sf4), false, 159L);
 +
 +    TabletMetadata tm3 = 
TabletMetadata.builder(extent).putExternalCompaction(ecid1, ecm)
 +        .putSuspension(ser1, 45L).putTime(new MetadataTime(479, 
TimeType.LOGICAL)).putWal(le1)
 +        .putWal(le2).setHostingRequested().putSelectedFiles(selFiles).build();
 +
 +    assertEquals(Set.of(ecid1), tm3.getExternalCompactions().keySet());
 +    assertEquals(Set.of(sf1, sf2), 
tm3.getExternalCompactions().get(ecid1).getJobFiles());
 +    assertEquals(ser1.getHostAndPort(), tm3.getSuspend().server);
 +    assertEquals(45L, tm3.getSuspend().suspensionTime);
 +    assertEquals(new MetadataTime(479, TimeType.LOGICAL), tm3.getTime());
 +    assertTrue(tm3.getHostingRequested());
 +    assertEquals(Stream.of(le1, le2).map(LogEntry::toString).collect(toSet()),
 +        tm3.getLogs().stream().map(LogEntry::toString).collect(toSet()));
 +    assertEquals(Set.of(sf1, sf4), tm3.getSelectedFiles().getFiles());
 +    assertEquals(159L, tm3.getSelectedFiles().getFateTxId());
 +    assertFalse(tm3.getSelectedFiles().initiallySelectedAll());
 +    assertEquals(selFiles.getMetadataValue(), 
tm3.getSelectedFiles().getMetadataValue());
 +  }
 +
  }
diff --cc 
server/base/src/main/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStore.java
index d58830a371,0000000000..1f7a9b057b
mode 100644,000000..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
@@@ -1,169 -1,0 +1,169 @@@
 +/*
 + * 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.server.manager.state;
 +
 +import java.util.Collection;
 +import java.util.List;
 +import java.util.Map;
 +
 +import org.apache.accumulo.core.clientImpl.ClientContext;
 +import org.apache.accumulo.core.dataImpl.KeyExtent;
 +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.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();
 +  }
 +
 +  @Override
 +  public void setLocations(Collection<Assignment> assignments) throws 
DistributedStoreException {
 +    try (var tabletsMutator = ample.conditionallyMutateTablets()) {
 +      for (Assignment assignment : assignments) {
 +        var conditionalMutator = 
tabletsMutator.mutateTablet(assignment.tablet)
 +            
.requireLocation(TabletMetadata.Location.future(assignment.server))
 +            .putLocation(TabletMetadata.Location.current(assignment.server))
 +            
.deleteLocation(TabletMetadata.Location.future(assignment.server)).deleteSuspension();
 +
 +        ManagerMetadataUtil.updateLastForAssignmentMode(context, 
conditionalMutator,
 +            assignment.server, assignment.lastLocation);
 +
 +        conditionalMutator.submit(tabletMetadata -> {
 +          
Preconditions.checkArgument(tabletMetadata.getExtent().equals(assignment.tablet));
 +          return tabletMetadata.getLocation() != null && 
tabletMetadata.getLocation()
 +              .equals(TabletMetadata.Location.current(assignment.server));
 +        });
 +      }
 +
 +      if (tabletsMutator.process().values().stream()
 +          .anyMatch(result -> result.getStatus() != Status.ACCEPTED)) {
 +        throw new DistributedStoreException(
 +            "failed to set tablet location, conditional mutation failed");
 +      }
 +    } catch (RuntimeException ex) {
 +      throw new DistributedStoreException(ex);
 +    }
 +  }
 +
 +  @Override
 +  public void setFutureLocations(Collection<Assignment> assignments)
 +      throws DistributedStoreException {
 +    try (var tabletsMutator = ample.conditionallyMutateTablets()) {
 +      for (Assignment assignment : assignments) {
 +        
tabletsMutator.mutateTablet(assignment.tablet).requireAbsentOperation()
 +            .requireAbsentLocation().deleteSuspension()
 +            .putLocation(TabletMetadata.Location.future(assignment.server))
 +            .submit(tabletMetadata -> {
 +              
Preconditions.checkArgument(tabletMetadata.getExtent().equals(assignment.tablet));
 +              return tabletMetadata.getLocation() != null && 
tabletMetadata.getLocation()
 +                  .equals(TabletMetadata.Location.future(assignment.server));
 +            });
 +      }
 +
 +      var results = tabletsMutator.process();
 +
 +      if (results.values().stream().anyMatch(result -> result.getStatus() != 
Status.ACCEPTED)) {
 +        throw new DistributedStoreException(
 +            "failed to set tablet location, conditional mutation failed. ");
 +      }
 +
 +    } catch (RuntimeException ex) {
 +      throw new DistributedStoreException(ex);
 +    }
 +  }
 +
 +  @Override
 +  public void unassign(Collection<TabletMetadata> tablets,
 +      Map<TServerInstance,List<Path>> logsForDeadServers) throws 
DistributedStoreException {
 +    unassign(tablets, logsForDeadServers, -1);
 +  }
 +
 +  @Override
 +  public void suspend(Collection<TabletMetadata> tablets,
 +      Map<TServerInstance,List<Path>> logsForDeadServers, long 
suspensionTimestamp)
 +      throws DistributedStoreException {
 +    unassign(tablets, logsForDeadServers, suspensionTimestamp);
 +  }
 +
 +  protected abstract void processSuspension(Ample.ConditionalTabletMutator 
tabletMutator,
 +      TabletMetadata tm, long suspensionTimestamp);
 +
 +  private void unassign(Collection<TabletMetadata> tablets,
 +      Map<TServerInstance,List<Path>> logsForDeadServers, long 
suspensionTimestamp)
 +      throws DistributedStoreException {
 +    try (var tabletsMutator = ample.conditionallyMutateTablets()) {
 +      for (TabletMetadata tm : tablets) {
 +        if (tm.getLocation() == null) {
 +          continue;
 +        }
 +
 +        var tabletMutator =
 +            
tabletsMutator.mutateTablet(tm.getExtent()).requireLocation(tm.getLocation());
 +
 +        if (tm.hasCurrent()) {
 +
 +          ManagerMetadataUtil.updateLastForAssignmentMode(context, 
tabletMutator,
 +              tm.getLocation().getServerInstance(), tm.getLast());
 +          tabletMutator.deleteLocation(tm.getLocation());
 +          if (logsForDeadServers != null) {
 +            List<Path> logs = 
logsForDeadServers.get(tm.getLocation().getServerInstance());
 +            if (logs != null) {
 +              for (Path log : logs) {
-                 LogEntry entry = new LogEntry(0, log.toString());
++                LogEntry entry = new LogEntry(log.toString());
 +                tabletMutator.putWal(entry);
 +              }
 +            }
 +          }
 +        }
 +
 +        if (tm.getLocation() != null && tm.getLocation().getType() != null
 +            && tm.getLocation().getType().equals(LocationType.FUTURE)) {
 +          tabletMutator.deleteLocation(tm.getLocation());
 +        }
 +
 +        processSuspension(tabletMutator, tm, suspensionTimestamp);
 +
 +        tabletMutator.submit(tabletMetadata -> tabletMetadata.getLocation() 
== null);
 +      }
 +
 +      Map<KeyExtent,Ample.ConditionalResult> results = 
tabletsMutator.process();
 +
 +      if (results.values().stream()
 +          .anyMatch(conditionalResult -> conditionalResult.getStatus() != 
Status.ACCEPTED)) {
 +        throw new DistributedStoreException("Some unassignments did not 
satisfy conditions.");
 +      }
 +
 +    } catch (RuntimeException ex) {
 +      throw new DistributedStoreException(ex);
 +    }
 +  }
 +}
diff --cc 
server/base/src/test/java/org/apache/accumulo/server/manager/state/TabletManagementTest.java
index a71308dc3e,0000000000..2fd6df2978
mode 100644,000000..100644
--- 
a/server/base/src/test/java/org/apache/accumulo/server/manager/state/TabletManagementTest.java
+++ 
b/server/base/src/test/java/org/apache/accumulo/server/manager/state/TabletManagementTest.java
@@@ -1,193 -1,0 +1,193 @@@
 +/*
 + * 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.server.manager.state;
 +
 +import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.COMPACT_COLUMN;
 +import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN;
 +import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.FLUSH_COLUMN;
 +import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN;
 +import static org.junit.jupiter.api.Assertions.assertEquals;
 +import static org.junit.jupiter.api.Assertions.assertTrue;
 +
 +import java.util.ArrayList;
 +import java.util.Set;
 +import java.util.SortedMap;
 +import java.util.TreeMap;
 +import java.util.UUID;
 +
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.TableId;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.dataImpl.KeyExtent;
 +import org.apache.accumulo.core.fate.FateTxId;
 +import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 +import org.apache.accumulo.core.manager.state.TabletManagement;
 +import 
org.apache.accumulo.core.manager.state.TabletManagement.ManagementAction;
 +import org.apache.accumulo.core.metadata.ReferencedTabletFile;
 +import org.apache.accumulo.core.metadata.StoredTabletFile;
 +import org.apache.accumulo.core.metadata.schema.DataFileValue;
 +import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.BulkFileColumnFamily;
 +import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ClonedColumnFamily;
 +import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
 +import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 +import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LastLocationColumnFamily;
 +import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 +import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
 +import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
 +import org.apache.accumulo.core.tabletserver.log.LogEntry;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.io.Text;
 +import org.junit.jupiter.api.Test;
 +
 +public class TabletManagementTest {
 +
 +  private SortedMap<Key,Value> toRowMap(Mutation mutation) {
 +    SortedMap<Key,Value> rowMap = new TreeMap<>();
 +    mutation.getUpdates().forEach(cu -> {
 +      Key k = new Key(mutation.getRow(), cu.getColumnFamily(), 
cu.getColumnQualifier(),
 +          cu.getTimestamp());
 +      Value v = new Value(cu.getValue());
 +      rowMap.put(k, v);
 +    });
 +    return rowMap;
 +  }
 +
 +  private SortedMap<Key,Value> createMetadataEntryKV(KeyExtent extent) {
 +
 +    Mutation mutation = TabletColumnFamily.createPrevRowMutation(extent);
 +
 +    COMPACT_COLUMN.put(mutation, new Value("5"));
 +    DIRECTORY_COLUMN.put(mutation, new Value("t-0001757"));
 +    FLUSH_COLUMN.put(mutation, new Value("6"));
 +    TIME_COLUMN.put(mutation, new Value("M123456789"));
 +
 +    StoredTabletFile bf1 =
 +        new ReferencedTabletFile(new 
Path("hdfs://nn1/acc/tables/1/t-0001/bf1")).insert();
 +    StoredTabletFile bf2 =
 +        new ReferencedTabletFile(new 
Path("hdfs://nn1/acc/tables/1/t-0001/bf2")).insert();
 +    
mutation.at().family(BulkFileColumnFamily.NAME).qualifier(bf1.getMetadata())
 +        .put(FateTxId.formatTid(56));
 +    
mutation.at().family(BulkFileColumnFamily.NAME).qualifier(bf2.getMetadata())
 +        .put(FateTxId.formatTid(59));
 +
 +    mutation.at().family(ClonedColumnFamily.NAME).qualifier("").put("OK");
 +
 +    DataFileValue dfv1 = new DataFileValue(555, 23);
 +    StoredTabletFile tf1 =
 +        new ReferencedTabletFile(new 
Path("hdfs://nn1/acc/tables/1/t-0001/df1.rf")).insert();
 +    StoredTabletFile tf2 =
 +        new ReferencedTabletFile(new 
Path("hdfs://nn1/acc/tables/1/t-0001/df2.rf")).insert();
 +    
mutation.at().family(DataFileColumnFamily.NAME).qualifier(tf1.getMetadata()).put(dfv1.encode());
 +    DataFileValue dfv2 = new DataFileValue(234, 13);
 +    
mutation.at().family(DataFileColumnFamily.NAME).qualifier(tf2.getMetadata()).put(dfv2.encode());
 +
 +    
mutation.at().family(CurrentLocationColumnFamily.NAME).qualifier("s001").put("server1:8555");
 +
 +    
mutation.at().family(LastLocationColumnFamily.NAME).qualifier("s000").put("server2:8555");
 +
-     LogEntry le1 = new LogEntry(55, "localhost:8020/" + UUID.randomUUID());
++    LogEntry le1 = new LogEntry("localhost:8020/" + UUID.randomUUID());
 +    
mutation.at().family(LogColumnFamily.NAME).qualifier(le1.getColumnQualifier())
-         .timestamp(le1.getTimestamp()).put(le1.getValue());
-     LogEntry le2 = new LogEntry(57, "localhost:8020/" + UUID.randomUUID());
++        .put(le1.getValue());
++    LogEntry le2 = new LogEntry("localhost:8020/" + UUID.randomUUID());
 +    
mutation.at().family(LogColumnFamily.NAME).qualifier(le2.getColumnQualifier())
-         .timestamp(le2.getTimestamp()).put(le2.getValue());
++        .put(le2.getValue());
 +
 +    StoredTabletFile sf1 =
 +        new ReferencedTabletFile(new 
Path("hdfs://nn1/acc/tables/1/t-0001/sf1.rf")).insert();
 +    StoredTabletFile sf2 =
 +        new ReferencedTabletFile(new 
Path("hdfs://nn1/acc/tables/1/t-0001/sf2.rf")).insert();
 +    
mutation.at().family(ScanFileColumnFamily.NAME).qualifier(sf1.getMetadata()).put("");
 +    
mutation.at().family(ScanFileColumnFamily.NAME).qualifier(sf2.getMetadata()).put("");
 +
 +    return toRowMap(mutation);
 +
 +  }
 +
 +  @Test
 +  public void testEncodeDecodeWithReasons() throws Exception {
 +    KeyExtent extent = new KeyExtent(TableId.of("5"), new Text("df"), new 
Text("da"));
 +
 +    final Set<ManagementAction> actions =
 +        Set.of(ManagementAction.NEEDS_LOCATION_UPDATE, 
ManagementAction.NEEDS_SPLITTING);
 +
 +    final SortedMap<Key,Value> entries = createMetadataEntryKV(extent);
 +
 +    TabletManagement.addActions(entries, actions);
 +    Key key = entries.firstKey();
 +    Value val = WholeRowIterator.encodeRow(new ArrayList<>(entries.keySet()),
 +        new ArrayList<>(entries.values()));
 +
 +    // Remove the REASONS column from the entries map for the comparison check
 +    // below
 +    entries.remove(new Key(key.getRow().toString(), "REASONS", ""));
 +
 +    TabletManagement tmi = new TabletManagement(key, val, true);
 +    assertEquals(entries, tmi.getTabletMetadata().getKeyValues());
 +    assertEquals(actions, tmi.getActions());
 +  }
 +
 +  @Test
 +  public void testEncodeDecodeWithErrors() throws Exception {
 +    KeyExtent extent = new KeyExtent(TableId.of("5"), new Text("df"), new 
Text("da"));
 +
 +    final SortedMap<Key,Value> entries = createMetadataEntryKV(extent);
 +
 +    TabletManagement.addError(entries, new UnsupportedOperationException("Not 
supported."));
 +    Key key = entries.firstKey();
 +    Value val = WholeRowIterator.encodeRow(new ArrayList<>(entries.keySet()),
 +        new ArrayList<>(entries.values()));
 +
 +    // Remove the ERROR column from the entries map for the comparison check
 +    // below
 +    entries.remove(new Key(key.getRow().toString(), "ERROR", ""));
 +
 +    TabletManagement tmi = new TabletManagement(key, val, true);
 +    assertEquals(entries, tmi.getTabletMetadata().getKeyValues());
 +    assertEquals("Not supported.", tmi.getErrorMessage());
 +  }
 +
 +  @Test
 +  public void testBinary() throws Exception {
 +    // test end row with non ascii data
 +    Text endRow = new Text(new byte[] {'m', (byte) 0xff});
 +    KeyExtent extent = new KeyExtent(TableId.of("5"), endRow, new Text("da"));
 +
 +    final Set<ManagementAction> actions =
 +        Set.of(ManagementAction.NEEDS_LOCATION_UPDATE, 
ManagementAction.NEEDS_SPLITTING);
 +
 +    final SortedMap<Key,Value> entries = createMetadataEntryKV(extent);
 +
 +    TabletManagement.addActions(entries, actions);
 +    Key key = entries.firstKey();
 +    Value val = WholeRowIterator.encodeRow(new ArrayList<>(entries.keySet()),
 +        new ArrayList<>(entries.values()));
 +
 +    assertTrue(entries.keySet().stream().allMatch(k -> 
k.getRow().equals(extent.toMetaRow())));
 +
 +    // Remove the REASONS column from the entries map for the comparison check
 +    // below
 +    entries.remove(new Key(key.getRow(), new Text("REASONS"), new Text("")));
 +
 +    TabletManagement tmi = new TabletManagement(key, val, true);
 +    assertEquals(entries, tmi.getTabletMetadata().getKeyValues());
 +    assertEquals(actions, tmi.getActions());
 +
 +  }
 +}
diff --cc 
server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index eb158c1d8e,4b610d72ca..f8208a3a04
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@@ -935,12 -1098,10 +935,10 @@@ public class TabletServer extends Abstr
      logger.minorCompactionStarted(tablet, lastUpdateSequence, 
newDataFileLocation, durability);
    }
  
 -  public void recover(VolumeManager fs, KeyExtent extent, List<LogEntry> 
logEntries,
 +  public void recover(VolumeManager fs, KeyExtent extent, 
Collection<LogEntry> logEntries,
        Set<String> tabletFiles, MutationReceiver mutationReceiver) throws 
IOException {
      List<Path> recoveryDirs = new ArrayList<>();
-     List<LogEntry> sorted = new ArrayList<>(logEntries);
-     sorted.sort((e1, e2) -> (int) (e1.getTimestamp() - e2.getTimestamp()));
-     for (LogEntry entry : sorted) {
+     for (LogEntry entry : logEntries) {
        Path recovery = null;
        Path finished = RecoveryPath.getRecoveryPath(new 
Path(entry.getFilePath()));
        finished = SortedLogState.getFinishedMarkerPath(finished);
diff --cc 
shell/src/test/java/org/apache/accumulo/shell/commands/ListTabletsCommandTest.java
index eb4d7e8410,583819b96b..14856bb9e7
--- 
a/shell/src/test/java/org/apache/accumulo/shell/commands/ListTabletsCommandTest.java
+++ 
b/shell/src/test/java/org/apache/accumulo/shell/commands/ListTabletsCommandTest.java
@@@ -128,50 -113,6 +128,50 @@@ public class ListTabletsCommandTest 
    public void mockTest() throws Exception {
      ListTabletsCommand cmd = new TestListTabletsCommand();
  
 +    TableId tableId = TableId.of("123");
 +
 +    TServerInstance ser1 = new 
TServerInstance(HostAndPort.fromParts("server1", 8555), "s001");
 +    TServerInstance ser2 = new 
TServerInstance(HostAndPort.fromParts("server2", 2354), "s002");
 +
 +    StoredTabletFile sf11 =
 +        new ReferencedTabletFile(new 
Path("hdfs://nn1/acc/tables/1/t-dir1/sf11.rf")).insert();
 +    DataFileValue dfv11 = new DataFileValue(5643, 89);
 +
 +    StoredTabletFile sf12 =
 +        new ReferencedTabletFile(new 
Path("hdfs://nn1/acc/tables/1/t-dir1/sf12.rf")).insert();
 +    DataFileValue dfv12 = new DataFileValue(379963, 1027);
 +
 +    StoredTabletFile sf21 =
 +        new ReferencedTabletFile(new 
Path("hdfs://nn1/acc/tables/1/t-dir2/sf21.rf")).insert();
 +    DataFileValue dfv21 = new DataFileValue(5323, 142);
 +
 +    StoredTabletFile sf31 =
 +        new ReferencedTabletFile(new 
Path("hdfs://nn1/acc/tables/1/t-dir3/sf31.rf")).insert();
 +    DataFileValue dfv31 = new DataFileValue(95832L, 231);
 +
 +    KeyExtent extent = new KeyExtent(tableId, new Text("d"), null);
 +
-     LogEntry le1 = new LogEntry(55, "localhost:8020/" + UUID.randomUUID());
-     LogEntry le2 = new LogEntry(57, "localhost:8020/" + UUID.randomUUID());
++    LogEntry le1 = new LogEntry("localhost:8020/" + UUID.randomUUID());
++    LogEntry le2 = new LogEntry("localhost:8020/" + UUID.randomUUID());
 +
 +    TabletMetadata tm1 = 
TabletMetadata.builder(extent).putHostingGoal(TabletHostingGoal.ONDEMAND)
 +        .putLocation(TabletMetadata.Location.current(ser1)).putFile(sf11, 
dfv11)
 +        .putFile(sf12, dfv12).putWal(le1).putDirName("t-dir1").build();
 +
 +    extent = new KeyExtent(tableId, new Text("k"), new Text("e"));
 +    TabletMetadata tm2 = 
TabletMetadata.builder(extent).putHostingGoal(TabletHostingGoal.ALWAYS)
 +        .putLocation(TabletMetadata.Location.current(ser2)).putFile(sf21, 
dfv21)
 +        .putDirName("t-dir2").build(LOGS);
 +
 +    extent = new KeyExtent(tableId, null, new Text("l"));
 +    TabletMetadata tm3 = 
TabletMetadata.builder(extent).putHostingGoal(TabletHostingGoal.NEVER)
 +        .putFile(sf31, 
dfv31).putWal(le1).putWal(le2).putDirName("t-dir3").build(LOCATION);
 +
 +    TabletInformationImpl[] tabletInformation = new TabletInformationImpl[3];
 +    tabletInformation[0] = new TabletInformationImpl(tm1, "HOSTED");
 +    tabletInformation[1] = new TabletInformationImpl(tm2, "HOSTED");
 +    tabletInformation[2] = new TabletInformationImpl(tm3, "UNASSIGNED");
 +
      AccumuloClient client = EasyMock.createMock(AccumuloClient.class);
      ClientContext context = EasyMock.createMock(ClientContext.class);
      TableOperations tableOps = EasyMock.createMock(TableOperations.class);
diff --cc 
test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java
index d6b18a7a8c,25c013fa53..f47d46b45c
--- 
a/test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java
+++ 
b/test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java
@@@ -396,46 -289,66 +396,46 @@@ public class TabletManagementIteratorI
      }
    }
  
 -  private static class State implements CurrentState {
 -
 -    final ClientContext context;
 -
 -    State(AccumuloClient client) {
 -      this.context = (ClientContext) client;
 +  // Creates a log entry on the "some split" extent, this could be modified 
easily to support
 +  // other extents
 +  private void createLogEntry(AccumuloClient client, String table, String 
tableNameToModify)
 +      throws MutationsRejectedException, TableNotFoundException {
 +    TableId tableIdToModify =
 +        
TableId.of(client.tableOperations().tableIdMap().get(tableNameToModify));
 +    KeyExtent extent = new KeyExtent(tableIdToModify, new Text("some split"), 
null);
 +    Mutation m = new Mutation(extent.toMetaRow());
 +    LogEntry logEntry =
-         new LogEntry(55, Path.of(validHost.toString(), 
UUID.randomUUID().toString()).toString());
++        new LogEntry(Path.of(validHost.toString(), 
UUID.randomUUID().toString()).toString());
 +    
m.at().family(LogColumnFamily.NAME).qualifier(logEntry.getColumnQualifier())
-         .timestamp(logEntry.getTimestamp()).put(logEntry.getValue());
++        .put(logEntry.getValue());
 +    try (BatchWriter bw = client.createBatchWriter(table)) {
 +      bw.addMutation(m);
      }
 +  }
  
 -    private Set<TServerInstance> tservers;
 -    private Set<TableId> onlineTables;
 -
 -    @Override
 -    public Set<TServerInstance> onlineTabletServers() {
 -      HashSet<TServerInstance> tservers = new HashSet<>();
 -      for (String tserver : context.instanceOperations().getTabletServers()) {
 -        try {
 -          var zPath = 
ServiceLock.path(ZooUtil.getRoot(context.instanceOperations().getInstanceId())
 -              + Constants.ZTSERVERS + "/" + tserver);
 -          long sessionId = ServiceLock.getSessionId(context.getZooCache(), 
zPath);
 -          tservers.add(new TServerInstance(tserver, sessionId));
 -        } catch (Exception e) {
 -          throw new RuntimeException(e);
 -        }
 +  private static TabletManagementParameters createParameters(AccumuloClient 
client) {
 +    var context = (ClientContext) client;
 +    Set<TableId> onlineTables = 
Sets.filter(context.getTableIdToNameMap().keySet(),
 +        tableId -> context.getTableState(tableId) == TableState.ONLINE);
 +
 +    HashSet<TServerInstance> tservers = new HashSet<>();
 +    for (String tserver : context.instanceOperations().getTabletServers()) {
 +      try {
 +        var zPath = 
ServiceLock.path(ZooUtil.getRoot(context.instanceOperations().getInstanceId())
 +            + Constants.ZTSERVERS + "/" + tserver);
 +        long sessionId = ServiceLock.getSessionId(context.getZooCache(), 
zPath);
 +        tservers.add(new TServerInstance(tserver, sessionId));
 +      } catch (Exception e) {
 +        throw new RuntimeException(e);
        }
 -      this.tservers = Collections.unmodifiableSet(tservers);
 -      return tservers;
 -    }
 -
 -    @Override
 -    public Set<TableId> onlineTables() {
 -      Set<TableId> onlineTables = context.getTableIdToNameMap().keySet();
 -      this.onlineTables =
 -          Sets.filter(onlineTables, tableId -> context.getTableState(tableId) 
== TableState.ONLINE);
 -      return this.onlineTables;
 -    }
 -
 -    @Override
 -    public Collection<MergeInfo> merges() {
 -      return Collections.emptySet();
      }
  
 -    @Override
 -    public Set<KeyExtent> migrationsSnapshot() {
 -      return Collections.emptySet();
 -    }
 -
 -    @Override
 -    public Set<TServerInstance> shutdownServers() {
 -      return Collections.emptySet();
 -    }
 -
 -    @Override
 -    public ManagerState getManagerState() {
 -      return ManagerState.NORMAL;
 -    }
 -
 -    @Override
 -    public String toString() {
 -      return "tservers: " + tservers + " onlineTables: " + onlineTables;
 -    }
 +    return new TabletManagementParameters(ManagerState.NORMAL,
 +        Map.of(
 +            Ample.DataLevel.ROOT, true, Ample.DataLevel.USER, true, 
Ample.DataLevel.METADATA, true),
 +        onlineTables,
 +        new LiveTServerSet.LiveTServersSnapshot(tservers,
 +            Map.of(Constants.DEFAULT_RESOURCE_GROUP_NAME, tservers)),
 +        Set.of(), Map.of(), Ample.DataLevel.USER, Map.of(), true);
    }
 -
  }

Reply via email to