diff --git a/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java b/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java index 16032686bec..7106cd89413 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java +++ b/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java @@ -28,6 +28,7 @@ import java.util.Set; import java.util.stream.Collectors; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.client.security.SecurityErrorCode; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.data.ConstraintViolationSummary; @@ -41,7 +42,7 @@ public class MutationsRejectedException extends AccumuloException { private final ArrayList cvsl = new ArrayList<>(); private final HashMap> af = new HashMap<>(); - private final HashSet es = new HashSet<>(); + private final HashSet es = new HashSet<>(); private final int unknownErrors; /** @@ -56,7 +57,7 @@ public class MutationsRejectedException extends AccumuloException { * @since 2.0.0 */ public MutationsRejectedException(AccumuloClient client, List cvsList, - Map> hashMap, Collection serverSideErrors, + Map> hashMap, Collection serverSideErrors, int unknownErrors, Throwable cause) { super( "constraint violation codes : " @@ -108,7 +109,7 @@ public Map> getSecurityErrorCodes() { * @return A list of servers that had internal errors when mutations were written * */ - public Collection getErrorServers() { + public Collection getErrorServers() { return es; } diff --git a/core/src/main/java/org/apache/accumulo/core/client/TimedOutException.java b/core/src/main/java/org/apache/accumulo/core/client/TimedOutException.java index 4b40aa0c54f..dfc90f3907e 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/TimedOutException.java +++ b/core/src/main/java/org/apache/accumulo/core/client/TimedOutException.java @@ -23,16 +23,18 @@ import java.util.HashSet; import java.util.Set; +import org.apache.accumulo.core.client.admin.servers.ServerId; + /** * @since 1.5.0 */ public class TimedOutException extends RuntimeException { - private final HashSet timedoutServers = new HashSet<>(); + private final HashSet timedoutServers = new HashSet<>(); private static final long serialVersionUID = 1L; - private static String shorten(Set set) { + private static String shorten(Set set) { if (set.size() < 10) { return set.toString(); } @@ -40,7 +42,7 @@ private static String shorten(Set set) { return new ArrayList<>(set).subList(0, 10) + " ... " + (set.size() - 10) + " servers not shown"; } - public TimedOutException(Set timedoutServers) { + public TimedOutException(Set timedoutServers) { super("Servers timed out " + shorten(timedoutServers)); this.timedoutServers.addAll(timedoutServers); @@ -50,7 +52,7 @@ public TimedOutException(String msg) { super(msg); } - public Set getTimedOutSevers() { + public Set getTimedOutSevers() { return Collections.unmodifiableSet(timedoutServers); } } diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/Locations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/Locations.java index 3b80258cba7..ad190663bff 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/admin/Locations.java +++ b/core/src/main/java/org/apache/accumulo/core/client/admin/Locations.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Map; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.TabletId; @@ -50,5 +51,5 @@ public interface Locations { * * @return A tablet server location in the form of {@code :} */ - String getTabletLocation(TabletId tabletId); + ServerId getTabletLocation(TabletId tabletId); } diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/servers/ServerId.java b/core/src/main/java/org/apache/accumulo/core/client/admin/servers/ServerId.java index 19182bc7c92..ef47e0cb6fe 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/admin/servers/ServerId.java +++ b/core/src/main/java/org/apache/accumulo/core/client/admin/servers/ServerId.java @@ -18,19 +18,27 @@ */ package org.apache.accumulo.core.client.admin.servers; +import static org.apache.accumulo.core.util.LazySingletons.GSON; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.Serializable; import java.util.Objects; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; +import org.apache.accumulo.core.clientImpl.ServerIdUtil.ServerIdInfo; import org.apache.accumulo.core.conf.PropertyType.PortRange; import org.apache.accumulo.core.data.ResourceGroupId; import com.google.common.base.Preconditions; +import com.google.common.net.HostAndPort; /** * Object representing the type, resource group, and address of a server process. * * @since 4.0.0 */ -public final class ServerId implements Comparable { +public final class ServerId implements Comparable, Serializable { /** * Server process type names. @@ -38,13 +46,20 @@ public final class ServerId implements Comparable { * @since 4.0.0 */ public enum Type { - MANAGER, MONITOR, GARBAGE_COLLECTOR, COMPACTOR, SCAN_SERVER, TABLET_SERVER; + MANAGER, MINI, MONITOR, GARBAGE_COLLECTOR, COMPACTOR, SCAN_SERVER, TABLET_SERVER; + } + + private static final long serialVersionUID = 1L; + + public static final ServerId deserialize(String json) { + return GSON.get().fromJson(json, ServerIdInfo.class).getServerId(); } private final Type type; private final ResourceGroupId resourceGroup; private final String host; private final int port; + private transient HostAndPort hostPort; public ServerId(Type type, ResourceGroupId resourceGroup, String host, int port) { super(); @@ -54,6 +69,7 @@ public ServerId(Type type, ResourceGroupId resourceGroup, String host, int port) this.resourceGroup = Objects.requireNonNull(resourceGroup); this.host = Objects.requireNonNull(host); this.port = port; + this.hostPort = HostAndPort.fromParts(host, port); } public Type getType() { @@ -72,6 +88,13 @@ public int getPort() { return port; } + private synchronized HostAndPort getHostPort() { + if (hostPort == null) { + hostPort = HostAndPort.fromParts(host, port); + } + return hostPort; + } + @Override public int compareTo(ServerId other) { if (this == other) { @@ -117,6 +140,15 @@ public String toString() { } public String toHostPortString() { - return host + ":" + port; + return getHostPort().toString(); + } + + public String serialize() { + return GSON.get().toJson(ServerIdUtil.toServerIdInfo(this)); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + this.hostPort = HostAndPort.fromParts(host, port); } } diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/AccumuloServerException.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/AccumuloServerException.java index ad568c7f50a..ebaea7bd4e8 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/AccumuloServerException.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/AccumuloServerException.java @@ -19,29 +19,30 @@ package org.apache.accumulo.core.clientImpl; import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.thrift.TApplicationException; /** * This class is intended to encapsulate errors that occurred on the server side. */ public class AccumuloServerException extends AccumuloException { - private static final long serialVersionUID = 1L; - private String server; + private static final long serialVersionUID = 2L; + private ServerId server; AccumuloServerException(final AccumuloServerException cause) { super("Error on server " + cause.getServer(), cause); } - public AccumuloServerException(final String server, final TApplicationException tae) { + public AccumuloServerException(final ServerId server, final TApplicationException tae) { super("Error on server " + server, tae); this.setServer(server); } - private void setServer(final String server) { + private void setServer(final ServerId server) { this.server = server; } - public String getServer() { + public ServerId getServer() { return server; } diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java index 21130c2a98b..9fafe780c39 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java @@ -79,6 +79,7 @@ import org.apache.accumulo.core.client.admin.ResourceGroupOperations; import org.apache.accumulo.core.client.admin.SecurityOperations; import org.apache.accumulo.core.client.admin.TableOperations; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.client.security.tokens.AuthenticationToken; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.ClientProperty; @@ -126,6 +127,7 @@ import com.github.benmanes.caffeine.cache.Cache; import com.google.common.base.Suppliers; +import com.google.common.net.HostAndPort; import io.micrometer.core.instrument.MeterRegistry; @@ -225,13 +227,9 @@ public Supplier> getScanServers() { .getScanServer(ResourceGroupPredicate.ANY, AddressSelector.all(), true).stream() .map(entry -> new ScanServerInfo() { @Override - public String getAddress() { - return entry.getServer(); - } - - @Override - public ResourceGroupId getGroup() { - return entry.getResourceGroup(); + public ServerId getServer() { + HostAndPort hp = HostAndPort.fromString(entry.getServer()); + return ServerIdUtil.sserver(entry.getResourceGroup(), hp.getHost(), hp.getPort()); } }).collect(Collectors.toSet()); } @@ -455,9 +453,9 @@ public ScanServerSelector getScanServerSelector() { /** * @return map of live scan server addresses to lock uuids. */ - public Map> getScanServers() { + public Map> getScanServers() { ensureOpen(); - Map> liveScanServers = new HashMap<>(); + Map> liveScanServers = new HashMap<>(); Set scanServerPaths = getServerPaths().getScanServer(ResourceGroupPredicate.ANY, AddressSelector.all(), true); for (ServiceLockPath path : scanServerPaths) { @@ -466,7 +464,7 @@ public Map> getScanServers() { Optional sld = ServiceLock.getLockData(getZooCache(), path, stat); if (sld.isPresent()) { final ServiceLockData data = sld.orElseThrow(); - final String addr = data.getAddressString(ThriftService.TABLET_SCAN); + final ServerId addr = data.getServer(ThriftService.TABLET_SCAN); final UUID uuid = data.getServerUUID(ThriftService.TABLET_SCAN); final ResourceGroupId group = data.getGroup(ThriftService.TABLET_SCAN); liveScanServers.put(addr, new Pair<>(uuid, group)); diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCache.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCache.java index 0c56f650509..c1b7a81234d 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCache.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCache.java @@ -34,9 +34,11 @@ import org.apache.accumulo.core.client.InvalidTabletHostingRequestException; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.admin.TabletAvailability; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.core.util.Interner; import org.apache.accumulo.core.util.Timer; import org.apache.accumulo.core.util.UtilWaitThread; @@ -128,7 +130,7 @@ public CachedTablet findTabletWithRetry(ClientContext context, Text row, boolean } public abstract void binMutations(ClientContext context, List mutations, - Map> binnedMutations, List failures) + Map> binnedMutations, List failures) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, InvalidTabletHostingRequestException; @@ -165,7 +167,7 @@ public abstract List findTablets(ClientContext context, List range * hosted tablets with a location. */ public List binRanges(ClientContext context, List ranges, - Map>> binnedRanges) throws AccumuloException, + Map>> binnedRanges) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, InvalidTabletHostingRequestException { return findTablets(context, ranges, ((cachedTablet, range) -> ClientTabletCacheImpl .addRange(binnedRanges, cachedTablet, range)), LocationNeed.REQUIRED); @@ -201,30 +203,30 @@ public static class CachedTablet { private static final Interner interner = new Interner<>(); private final KeyExtent tablet_extent; - private final String tserverLocation; + private final ServerId tserverLocation; private final String tserverSession; private final TabletAvailability availability; private final boolean hostingRequested; private final Timer creationTimer = Timer.startNew(); - public CachedTablet(KeyExtent tablet_extent, String tablet_location, String session, + public CachedTablet(KeyExtent tablet_extent, ServerId tablet_location, String session, TabletAvailability availability, boolean hostingRequested) { checkArgument(tablet_extent != null, "tablet_extent is null"); checkArgument(tablet_location != null, "tablet_location is null"); checkArgument(session != null, "session is null"); this.tablet_extent = tablet_extent; - this.tserverLocation = interner.intern(tablet_location); + this.tserverLocation = tablet_location; this.tserverSession = interner.intern(session); this.availability = Objects.requireNonNull(availability); this.hostingRequested = hostingRequested; } - public CachedTablet(KeyExtent tablet_extent, Optional tablet_location, + public CachedTablet(KeyExtent tablet_extent, Optional tablet_location, Optional session, TabletAvailability availability, boolean hostingRequested) { checkArgument(tablet_extent != null, "tablet_extent is null"); this.tablet_extent = tablet_extent; - this.tserverLocation = tablet_location.map(interner::intern).orElse(null); + this.tserverLocation = tablet_location.orElse(null); this.tserverSession = session.map(interner::intern).orElse(null); this.availability = Objects.requireNonNull(availability); this.hostingRequested = hostingRequested; @@ -268,7 +270,7 @@ public KeyExtent getExtent() { return tablet_extent; } - public Optional getTserverLocation() { + public Optional getTserverLocation() { return Optional.ofNullable(tserverLocation); } @@ -300,9 +302,9 @@ public boolean wasHostingRequested() { public static class TabletServerMutations { private final Map> mutations; - private final String tserverSession; + private final TServerInstance tserverSession; - public TabletServerMutations(String tserverSession) { + public TabletServerMutations(TServerInstance tserverSession) { this.tserverSession = tserverSession; this.mutations = new HashMap<>(); } @@ -316,7 +318,7 @@ public Map> getMutations() { return mutations; } - final String getSession() { + public TServerInstance getTServerInstance() { return tserverSession; } } diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCacheImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCacheImpl.java index 90bd2c0ce13..b1369d70d47 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCacheImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCacheImpl.java @@ -44,6 +44,7 @@ import org.apache.accumulo.core.client.InvalidTabletHostingRequestException; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.admin.TabletAvailability; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.PartialKey; @@ -54,6 +55,7 @@ import org.apache.accumulo.core.lock.ServiceLockPaths.ResourceGroupPredicate; import org.apache.accumulo.core.manager.state.tables.TableState; import org.apache.accumulo.core.metadata.SystemTables; +import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.core.util.LockMap; @@ -155,15 +157,15 @@ CachedTablets lookupTablet(ClientContext context, CachedTablet src, Text row, Te } public interface TabletServerLockChecker { - boolean isLockHeld(String tserver, String session); + boolean isLockHeld(ServerId tserver, String session); void invalidateCache(String server); } private class LockCheckerSession { - private final HashSet> okLocks = new HashSet<>(); - private final HashSet> invalidLocks = new HashSet<>(); + private final HashSet> okLocks = new HashSet<>(); + private final HashSet> invalidLocks = new HashSet<>(); private CachedTablet checkLock(CachedTablet tl) { // the goal of this class is to minimize calls out to lockChecker under that @@ -181,7 +183,7 @@ private CachedTablet checkLock(CachedTablet tl) { return tl; } - Pair lock = + Pair lock = new Pair<>(tl.getTserverLocation().orElseThrow(), tl.getTserverSession().orElseThrow()); if (okLocks.contains(lock)) { @@ -222,7 +224,7 @@ public ClientTabletCacheImpl(TableId tableId, ClientTabletCache parent, CachedTa @Override public void binMutations(ClientContext context, List mutations, - Map> binnedMutations, List failures) + Map> binnedMutations, List failures) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, InvalidTabletHostingRequestException { @@ -293,7 +295,7 @@ public void binMutations(ClientContext context, List mut } private boolean addMutation( - Map> binnedMutations, T mutation, CachedTablet tl, + Map> binnedMutations, T mutation, CachedTablet tl, LockCheckerSession lcSession) { if (tl == null || tl.getTserverLocation().isEmpty()) { @@ -306,7 +308,8 @@ private boolean addMutation( // do lock check once per tserver here to make binning faster boolean lockHeld = lcSession.checkLock(tl) != null; if (lockHeld) { - tsm = new TabletServerMutations<>(tl.getTserverSession().orElseThrow()); + tsm = new TabletServerMutations<>(new TServerInstance(tl.getTserverLocation().orElseThrow(), + tl.getTserverSession().orElseThrow())); binnedMutations.put(tl.getTserverLocation().orElseThrow(), tsm); } else { return false; @@ -314,7 +317,7 @@ private boolean addMutation( } // its possible the same tserver could be listed with different sessions - if (tsm.getSession().equals(tl.getTserverSession().orElseThrow())) { + if (tsm.getTServerInstance().getSession().equals(tl.getTserverSession().orElseThrow())) { tsm.addMutation(tl.getExtent(), mutation); return true; } @@ -873,7 +876,7 @@ private CachedTablet lookupTabletLocationAndCheckLock(ClientContext context, Tex return lcSession.checkLock(findTabletInCache(row)); } - static void addRange(Map>> binnedRanges, CachedTablet ct, + static void addRange(Map>> binnedRanges, CachedTablet ct, Range range) { binnedRanges.computeIfAbsent(ct.getTserverLocation().orElseThrow(), k -> new HashMap<>()) .computeIfAbsent(ct.getExtent(), k -> new ArrayList<>()).add(range); diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java index 2103f90392c..e014098b454 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java @@ -51,6 +51,7 @@ import org.apache.accumulo.core.client.ConditionalWriterConfig; import org.apache.accumulo.core.client.Durability; import org.apache.accumulo.core.client.TimedOutException; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ClientTabletCache.TabletServerMutations; import org.apache.accumulo.core.clientImpl.thrift.TInfo; import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; @@ -88,7 +89,6 @@ import org.slf4j.LoggerFactory; import com.google.common.annotations.VisibleForTesting; -import com.google.common.net.HostAndPort; public class ConditionalWriterImpl implements ConditionalWriter { @@ -113,7 +113,7 @@ private static class ServerQueue { boolean taskQueued = false; } - private final Map serverQueues; + private final Map serverQueues; private final DelayQueue failedMutations = new DelayQueue<>(); private final ScheduledThreadPoolExecutor threadPool; private final ScheduledFuture failureTaskFuture; @@ -209,7 +209,7 @@ void queueResult(Result result) { } } - private ServerQueue getServerQueue(String location) { + private ServerQueue getServerQueue(ServerId location) { ServerQueue serverQueue; synchronized (serverQueues) { serverQueue = serverQueues.get(location); @@ -250,7 +250,7 @@ public void run() { } } - private void queueRetry(List mutations, HostAndPort server) { + private void queueRetry(List mutations, ServerId server) { if (timeout < Long.MAX_VALUE) { @@ -263,7 +263,7 @@ private void queueRetry(List mutations, HostAndPort server) { if (time + qcm.getDelay(MILLISECONDS) > qcm.entryTime + timeout) { TimedOutException toe; if (server != null) { - toe = new TimedOutException(Collections.singleton(server.toString())); + toe = new TimedOutException(Collections.singleton(server)); } else { toe = new TimedOutException("Conditional mutation timed out"); } @@ -286,7 +286,7 @@ private void queueRetry(List mutations, HostAndPort server) { private void queue(List mutations) { List failures = new ArrayList<>(); - Map> binnedMutations = new HashMap<>(); + Map> binnedMutations = new HashMap<>(); try { locator.binMutations(context, mutations, binnedMutations, failures); @@ -311,7 +311,7 @@ private void queue(List mutations) { binnedMutations.forEach(this::queue); } - private void queue(String location, TabletServerMutations mutations) { + private void queue(ServerId location, TabletServerMutations mutations) { ServerQueue serverQueue = getServerQueue(location); @@ -345,7 +345,7 @@ private void reschedule(SendTask task) { } - private TabletServerMutations dequeue(String location) { + private TabletServerMutations dequeue(ServerId location) { var queue = getServerQueue(location).queue; var mutations = new ArrayList>(); @@ -438,9 +438,9 @@ public Iterator write(Iterator mutations) { private class SendTask implements Runnable { - final String location; + final ServerId location; - public SendTask(String location) { + public SendTask(ServerId location) { this.location = location; } @@ -450,7 +450,7 @@ public void run() { try { TabletServerMutations mutations = dequeue(location); if (mutations != null) { - sendToServer(HostAndPort.fromString(location), mutations); + sendToServer(location, mutations); } } finally { reschedule(this); @@ -470,7 +470,7 @@ public CMK(KeyExtent ke, QCMutation cm) { } private static class SessionID { - HostAndPort location; + ServerId location; String lockId; long sessionID; boolean reserved; @@ -482,9 +482,9 @@ boolean isActive() { } } - private final HashMap cachedSessionIDs = new HashMap<>(); + private final HashMap cachedSessionIDs = new HashMap<>(); - private SessionID reserveSessionID(HostAndPort location, TabletIngestClientService.Iface client, + private SessionID reserveSessionID(ServerId location, TabletIngestClientService.Iface client, TInfo tinfo) throws ThriftSecurityException, TException { // avoid cost of repeatedly making RPC to create sessions, reuse sessions synchronized (cachedSessionIDs) { @@ -523,14 +523,14 @@ private SessionID reserveSessionID(HostAndPort location, TabletIngestClientServi } - private void invalidateSessionID(HostAndPort location) { + private void invalidateSessionID(ServerId location) { synchronized (cachedSessionIDs) { cachedSessionIDs.remove(location); } } - private void unreserveSessionID(HostAndPort location) { + private void unreserveSessionID(ServerId location) { synchronized (cachedSessionIDs) { SessionID sid = cachedSessionIDs.get(location); if (sid != null) { @@ -553,8 +553,7 @@ List getActiveSessions() { return activeSessions; } - private TabletIngestClientService.Iface getClient(HostAndPort location) - throws TTransportException { + private TabletIngestClientService.Iface getClient(ServerId location) throws TTransportException { TabletIngestClientService.Iface client; if (timeout < context.getClientTimeoutInMillis()) { client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, location, context, timeout); @@ -564,7 +563,7 @@ private TabletIngestClientService.Iface getClient(HostAndPort location) return client; } - private void sendToServer(HostAndPort location, TabletServerMutations mutations) { + private void sendToServer(ServerId location, TabletServerMutations mutations) { TabletIngestClientService.Iface client = null; TInfo tinfo = TraceUtil.traceInfo(); @@ -622,7 +621,7 @@ private void sendToServer(HostAndPort location, TabletServerMutations cmidToCm, HostAndPort location) { + private void queueRetry(Map cmidToCm, ServerId location) { ArrayList ignored = new ArrayList<>(); for (CMK cmk : cmidToCm.values()) { ignored.add(cmk.cm); @@ -644,14 +643,13 @@ private void queueRetry(Map cmidToCm, HostAndPort location) { queueRetry(ignored, location); } - private void queueException(HostAndPort location, Map cmidToCm, Exception e) { + private void queueException(ServerId location, Map cmidToCm, Exception e) { for (CMK cmk : cmidToCm.values()) { cmk.cm.queueResult(new Result(e, cmk.cm, location.toString())); } } - private void invalidateSession(HostAndPort location, Map cmidToCm, - SessionID sessionId) { + private void invalidateSession(ServerId location, Map cmidToCm, SessionID sessionId) { if (sessionId == null) { queueRetry(cmidToCm, location); } else { @@ -675,8 +673,7 @@ private void invalidateSession(HostAndPort location, Map cmidToCm, * If a conditional mutation is taking a long time to process, then this method will wait for it * to finish... unless this exceeds timeout. */ - private void invalidateSession(SessionID sessionId, HostAndPort location) - throws AccumuloException { + private void invalidateSession(SessionID sessionId, ServerId location) throws AccumuloException { long sleepTime = 50; @@ -699,14 +696,14 @@ private void invalidateSession(SessionID sessionId, HostAndPort location) log.trace("Invalidated {} at {}", sessionId.sessionID, location); return; } catch (TApplicationException tae) { - throw new AccumuloServerException(location.toString(), tae); + throw new AccumuloServerException(location, tae); } catch (TException e) { log.trace("Failed to invalidate {} at {} {}", sessionId.sessionID, location, e.getMessage()); } if ((System.currentTimeMillis() - startTime) + sleepTime > timeout) { - throw new TimedOutException(Collections.singleton(location.toString())); + throw new TimedOutException(Collections.singleton(location)); } sleepUninterruptibly(sleepTime, MILLISECONDS); @@ -714,7 +711,7 @@ private void invalidateSession(SessionID sessionId, HostAndPort location) } } - private void invalidateSession(long sessionId, HostAndPort location) throws TException { + private void invalidateSession(long sessionId, ServerId location) throws TException { TabletIngestClientService.Iface client = null; TInfo tinfo = TraceUtil.traceInfo(); diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java index 3008b079f04..3aa03b664dd 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java @@ -68,7 +68,6 @@ import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService; import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService.Client; import org.apache.accumulo.core.trace.TraceUtil; -import org.apache.accumulo.core.util.AddressUtil; import org.apache.accumulo.core.util.LocalityGroupUtil; import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError; import org.apache.accumulo.core.util.Retry; @@ -302,10 +301,9 @@ private List getActiveScans(ServerId server) checkActiveScanServer(server); - final var parsedTserver = HostAndPort.fromParts(server.getHost(), server.getPort()); TabletScanClientService.Client rpcClient = null; try { - rpcClient = getClient(ThriftClientTypes.TABLET_SCAN, parsedTserver, context); + rpcClient = getClient(ThriftClientTypes.TABLET_SCAN, server, context); List as = new ArrayList<>(); for (var activeScan : rpcClient.getActiveScans(TraceUtil.traceInfo(), context.rpcCreds())) { @@ -349,13 +347,12 @@ private List getActiveCompactions(ServerId server) checkActiveCompactionServer(server); - final HostAndPort serverHostAndPort = HostAndPort.fromParts(server.getHost(), server.getPort()); final List as = new ArrayList<>(); try { if (server.getType() == ServerId.Type.TABLET_SERVER) { Client client = null; try { - client = getClient(ThriftClientTypes.TABLET_SERVER, serverHostAndPort, context); + client = getClient(ThriftClientTypes.TABLET_SERVER, server, context); for (var tac : client.getActiveCompactions(TraceUtil.traceInfo(), context.rpcCreds())) { as.add(new ActiveCompactionImpl(context, tac, server)); } @@ -366,7 +363,7 @@ private List getActiveCompactions(ServerId server) } } else { // if not a TabletServer address, maybe it's a Compactor - for (var tac : ExternalCompactionUtil.getActiveCompaction(serverHostAndPort, context)) { + for (var tac : ExternalCompactionUtil.getActiveCompaction(server, context)) { as.add(new ActiveCompactionImpl(context, tac, server)); } } @@ -448,7 +445,19 @@ private List queryServers(Collection servers, ServerQuery s = this.getServers(t, ResourceGroupPredicate.ANY, + (h, p) -> hp.getHost().equals(h) && hp.getPort() == p); + if (s != null && !s.isEmpty()) { + ping(s.iterator().next()); + } + } + } + + @Override + public void ping(ServerId server) throws AccumuloException { + try (TTransport transport = createTransport(server, context)) { ClientService.Client client = createClient(ThriftClientTypes.CLIENT, transport, context.getInstanceID()); client.ping(context.rpcCreds()); @@ -457,11 +466,6 @@ public void ping(String server) throws AccumuloException { } } - @Override - public void ping(ServerId server) throws AccumuloException { - ping(server.toHostPortString()); - } - @Override public void waitForBalance() throws AccumuloException { try { @@ -574,12 +578,11 @@ private Set getServers(ServerId.Type type, ServiceLockPath m = context.getServerPaths().getManager(true); if (m != null) { Optional sld = context.getZooCache().getLockData(m); - String location = null; if (sld.isPresent()) { - location = sld.orElseThrow().getAddressString(ThriftService.MANAGER); - if (location != null && addressSelector.getPredicate().test(location)) { - HostAndPort hp = HostAndPort.fromString(location); - results.add(new ServerId(type, ResourceGroupId.DEFAULT, hp.getHost(), hp.getPort())); + ServerId location = sld.orElseThrow().getServer(ThriftService.MANAGER); + if (location != null + && addressSelector.getPredicate().test(location.toHostPortString())) { + results.add(location); } } } @@ -588,12 +591,11 @@ private Set getServers(ServerId.Type type, ServiceLockPath mon = context.getServerPaths().getMonitor(true); if (mon != null) { Optional sld = context.getZooCache().getLockData(mon); - String location = null; if (sld.isPresent()) { - location = sld.orElseThrow().getAddressString(ThriftService.NONE); - if (location != null && addressSelector.getPredicate().test(location)) { - HostAndPort hp = HostAndPort.fromString(location); - results.add(new ServerId(type, ResourceGroupId.DEFAULT, hp.getHost(), hp.getPort())); + ServerId location = sld.orElseThrow().getServer(ThriftService.NONE); + if (location != null + && addressSelector.getPredicate().test(location.toHostPortString())) { + results.add(location); } } } @@ -602,12 +604,11 @@ private Set getServers(ServerId.Type type, ServiceLockPath gc = context.getServerPaths().getGarbageCollector(true); if (gc != null) { Optional sld = context.getZooCache().getLockData(gc); - String location = null; if (sld.isPresent()) { - location = sld.orElseThrow().getAddressString(ThriftService.GC); - if (location != null && addressSelector.getPredicate().test(location)) { - HostAndPort hp = HostAndPort.fromString(location); - results.add(new ServerId(type, ResourceGroupId.DEFAULT, hp.getHost(), hp.getPort())); + ServerId location = sld.orElseThrow().getServer(ThriftService.GC); + if (location != null + && addressSelector.getPredicate().test(location.toHostPortString())) { + results.add(location); } } } @@ -635,7 +636,7 @@ private ServerId createServerId(ServerId.Type type, ServiceLockPath slp) { HostAndPort hp = HostAndPort.fromString(Objects.requireNonNull(slp.getServer())); String host = hp.getHost(); int port = hp.getPort(); - return new ServerId(type, resourceGroup, host, port); + return ServerIdUtil.dynamic(type, resourceGroup, host, port); } private Optional getServerId(String server, List types) { diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/RootClientTabletCache.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/RootClientTabletCache.java index 4eac34a7bb7..e12c1c97dce 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/RootClientTabletCache.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/RootClientTabletCache.java @@ -30,11 +30,13 @@ import java.util.function.BiConsumer; import org.apache.accumulo.core.client.admin.TabletAvailability; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ClientTabletCacheImpl.TabletServerLockChecker; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.metadata.RootTable; +import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.core.metadata.schema.RootTabletMetadata; import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location; import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType; @@ -67,10 +69,12 @@ public class RootClientTabletCache extends ClientTabletCache { @Override public void binMutations(ClientContext context, List mutations, - Map> binnedMutations, List failures) { + Map> binnedMutations, List failures) { CachedTablet rootCachedTablet = getRootTabletLocation(context); if (rootCachedTablet != null && rootCachedTablet.getTserverLocation().isPresent()) { - var tsm = new TabletServerMutations(rootCachedTablet.getTserverSession().orElseThrow()); + var tsm = new TabletServerMutations( + new TServerInstance(rootCachedTablet.getTserverLocation().orElseThrow(), + rootCachedTablet.getTserverSession().orElseThrow())); for (T mutation : mutations) { tsm.addMutation(RootTable.EXTENT, mutation); } @@ -137,11 +141,11 @@ protected CachedTablet getRootTabletLocation(ClientContext context) { TabletAvailability.HOSTED, false); } - String server = loc.getHostPort(); + ServerId server = loc.getServerInstance().getServer(); - if (lockChecker.isLockHeld(server, loc.getSession())) { - return new CachedTablet(RootTable.EXTENT, server, loc.getSession(), TabletAvailability.HOSTED, - false); + if (lockChecker.isLockHeld(server, loc.getServerInstance().getSession())) { + return new CachedTablet(RootTable.EXTENT, server, loc.getServerInstance().getSession(), + TabletAvailability.HOSTED, false); } else { return new CachedTablet(RootTable.EXTENT, Optional.empty(), Optional.empty(), TabletAvailability.HOSTED, false); diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScanServerAttemptImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScanServerAttemptImpl.java index bda2f26880a..65edde783e3 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScanServerAttemptImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScanServerAttemptImpl.java @@ -20,20 +20,21 @@ import java.util.Objects; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.spi.scan.ScanServerAttempt; class ScanServerAttemptImpl implements ScanServerAttempt { - private final String server; + private final ServerId server; private final Result result; - ScanServerAttemptImpl(Result result, String server) { + ScanServerAttemptImpl(Result result, ServerId server) { this.result = result; this.server = Objects.requireNonNull(server); } @Override - public String getServer() { + public ServerId getServer() { return server; } diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScanServerAttemptsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScanServerAttemptsImpl.java index 48ff72b5326..1186ca21f88 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScanServerAttemptsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScanServerAttemptsImpl.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.Map.Entry; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.data.TabletId; import org.apache.accumulo.core.spi.scan.ScanServerAttempt; import org.slf4j.Logger; @@ -45,7 +46,7 @@ public class ScanServerAttemptsImpl { private final Map> attempts = new HashMap<>(); - ScanServerAttemptReporter createReporter(String server, TabletId tablet) { + ScanServerAttemptReporter createReporter(ServerId server, TabletId tablet) { return result -> { LOG.trace("Received result: {}", result); synchronized (attempts) { diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ServerIdUtil.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ServerIdUtil.java new file mode 100644 index 00000000000..56ca253ee93 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ServerIdUtil.java @@ -0,0 +1,125 @@ +/* + * 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.core.clientImpl; + +import org.apache.accumulo.core.client.admin.servers.ServerId; +import org.apache.accumulo.core.client.admin.servers.ServerId.Type; +import org.apache.accumulo.core.data.ResourceGroupId; +import org.apache.accumulo.core.util.cache.Caches; +import org.apache.accumulo.core.util.cache.Caches.CacheName; + +import com.github.benmanes.caffeine.cache.Cache; +import com.google.common.base.Preconditions; + +public class ServerIdUtil { + + public static record ServerIdInfo(String type, String resourceGroup, String host, int port) { + public ServerId getServerId() { + return new ServerId(ServerId.Type.valueOf(type), ResourceGroupId.of(resourceGroup), host, + port); + } + } + + // cache is for canonicalization/deduplication of created objects, + // to limit the number of ServerId objects in the JVM at any given moment + // WeakReferences are used because we don't need them to stick around any longer than they need to + private static final Cache cache = + Caches.getInstance().createNewBuilder(CacheName.SERVER_ID, false).weakValues().build(); + + public static ServerIdInfo toServerIdInfo(ServerId server) { + return new ServerIdInfo(server.getType().name(), server.getResourceGroup().canonical(), + server.getHost(), server.getPort()); + } + + private static ServerId resolve(ServerIdInfo info) { + return cache.get(info, k -> info.getServerId()); + } + + public static ServerId compactor(String host, int port) { + return resolve( + new ServerIdInfo(Type.COMPACTOR.name(), ResourceGroupId.DEFAULT.canonical(), host, port)); + } + + public static ServerId compactor(ResourceGroupId rgid, String host, int port) { + return resolve(new ServerIdInfo(Type.COMPACTOR.name(), rgid.canonical(), host, port)); + } + + public static ServerId gc(String host, int port) { + return resolve(new ServerIdInfo(Type.GARBAGE_COLLECTOR.name(), + ResourceGroupId.DEFAULT.canonical(), host, port)); + } + + public static ServerId manager(String host, int port) { + return resolve( + new ServerIdInfo(Type.MANAGER.name(), ResourceGroupId.DEFAULT.canonical(), host, port)); + } + + public static ServerId mini(String host, int port) { + return resolve( + new ServerIdInfo(Type.MINI.name(), ResourceGroupId.DEFAULT.canonical(), host, port)); + } + + public static ServerId monitor(String host, int port) { + return resolve( + new ServerIdInfo(Type.MONITOR.name(), ResourceGroupId.DEFAULT.canonical(), host, port)); + } + + public static ServerId sserver(String host, int port) { + return resolve( + new ServerIdInfo(Type.SCAN_SERVER.name(), ResourceGroupId.DEFAULT.canonical(), host, port)); + } + + public static ServerId sserver(ResourceGroupId rgid, String host, int port) { + return resolve(new ServerIdInfo(Type.SCAN_SERVER.name(), rgid.canonical(), host, port)); + } + + public static ServerId tserver(String host, int port) { + return resolve(new ServerIdInfo(Type.TABLET_SERVER.name(), ResourceGroupId.DEFAULT.canonical(), + host, port)); + } + + public static ServerId tserver(ResourceGroupId rgid, String host, int port) { + return resolve(new ServerIdInfo(Type.TABLET_SERVER.name(), rgid.canonical(), host, port)); + } + + public static ServerId dynamic(Type type, ResourceGroupId rgid, String host, int port) { + return resolve(new ServerIdInfo(type.name(), rgid.canonical(), host, port)); + } + + public static ServerId fromWalFileName(String name) { + // Old path was host+port + // New path is group+host+port + // Need to handle both + String parts[] = name.split("\\+"); + Preconditions.checkArgument(parts.length == 2 || parts.length == 3, + "Invalid server id in wal file: " + name); + if (parts.length == 2) { + // return an uncached tserver object + return tserver(parts[0], Integer.parseInt(parts[1])); + } else { + // return an uncached tserver object + return tserver(ResourceGroupId.of(parts[0]), parts[1], Integer.parseInt(parts[2])); + } + } + + public static String toWalFileName(ServerId server) { + return server.getResourceGroup() + "+" + server.getHost() + "+" + server.getPort(); + } + +} diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingClientTabletCache.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingClientTabletCache.java index ffc2c0c5485..5742eb22af6 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingClientTabletCache.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingClientTabletCache.java @@ -28,6 +28,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.InvalidTabletHostingRequestException; import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.TableId; @@ -76,7 +77,7 @@ public CachedTablet findTablet(ClientContext context, Text row, boolean skipRow, @Override public void binMutations(ClientContext context, List mutations, - Map> binnedMutations, List failures) + Map> binnedMutations, List failures) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, InvalidTabletHostingRequestException { syncLocator().binMutations(context, mutations, binnedMutations, failures); @@ -92,7 +93,7 @@ public List findTablets(ClientContext context, List ranges, @Override public List binRanges(ClientContext context, List ranges, - Map>> binnedRanges) throws AccumuloException, + Map>> binnedRanges) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, InvalidTabletHostingRequestException { return syncLocator().binRanges(context, ranges, binnedRanges); } diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java index a0be93ea15e..973df399e1f 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java @@ -110,6 +110,7 @@ import org.apache.accumulo.core.client.admin.TimeType; import org.apache.accumulo.core.client.admin.compaction.CompactionConfigurer; import org.apache.accumulo.core.client.admin.compaction.CompactionSelector; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.client.sample.SamplerConfiguration; import org.apache.accumulo.core.client.summary.SummarizerConfiguration; import org.apache.accumulo.core.client.summary.Summary; @@ -1423,7 +1424,7 @@ private void waitForTableStateTransition(TableId tableId, TableState expectedSta lastRow = tablet.getExtent().toMetaRow(); if (loc != null) { - serverCounts.increment(loc.getHostPortSession(), 1); + serverCounts.increment(loc.getServerInstance().toZooKeeperPathString(), 1); } } @@ -1588,7 +1589,7 @@ public List getDiskUsage(Set tableNames) List diskUsages = null; while (diskUsages == null) { - Pair pair = null; + Pair pair = null; try { // this operation may us a lot of memory... it's likely that connections to tabletservers // hosting metadata tablets will be cached, so do not use cached @@ -1891,15 +1892,15 @@ private static class LocationsImpl implements Locations { private Map> groupedByRanges; private Map> groupedByTablets; - private final Map tabletLocations; + private final Map tabletLocations; - public LocationsImpl(Map>> binnedRanges) { + public LocationsImpl(Map>> binnedRanges) { groupedByTablets = new HashMap<>(); groupedByRanges = null; tabletLocations = new HashMap<>(); - for (Entry>> entry : binnedRanges.entrySet()) { - String location = entry.getKey(); + for (Entry>> entry : binnedRanges.entrySet()) { + ServerId location = entry.getKey(); for (Entry> entry2 : entry.getValue().entrySet()) { TabletIdImpl tabletId = new TabletIdImpl(entry2.getKey()); @@ -1917,7 +1918,7 @@ public LocationsImpl(Map>> binnedRanges) { } @Override - public String getTabletLocation(TabletId tabletId) { + public ServerId getTabletLocation(TabletId tabletId) { return tabletLocations.get(tabletId); } @@ -1972,7 +1973,7 @@ public Locations locate(String tableName, Collection ranges) .logInterval(Duration.ofMinutes(3)).createRetry(); final ArrayList locationLess = new ArrayList<>(); - final Map>> binnedRanges = new HashMap<>(); + final Map>> binnedRanges = new HashMap<>(); final AtomicBoolean foundOnDemandTabletInRange = new AtomicBoolean(false); BiConsumer rangeConsumer = (cachedTablet, range) -> { diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletInformationImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletInformationImpl.java index 10c86c172c2..8a04a1c7068 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletInformationImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletInformationImpl.java @@ -99,8 +99,8 @@ public String getTabletState() { @Override public Optional getLocation() { Location location = tabletMetadata.getLocation(); - return location == null ? Optional.empty() - : Optional.of(location.getType() + ":" + location.getHostPort()); + return location == null ? Optional.empty() : Optional + .of(location.getType() + ":" + location.getServerInstance().getServer().toHostPortString()); } @Override diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java index a7de7f5e76c..b435a8f98a1 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java @@ -52,6 +52,7 @@ import org.apache.accumulo.core.client.TableDeletedException; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.TimedOutException; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ClientTabletCache.LocationNeed; import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; import org.apache.accumulo.core.data.Column; @@ -88,8 +89,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.net.HostAndPort; - public class TabletServerBatchReaderIterator implements Iterator> { private static final Logger log = LoggerFactory.getLogger(TabletServerBatchReaderIterator.class); @@ -112,8 +111,8 @@ public class TabletServerBatchReaderIterator implements Iterator timeoutTrackers; - private final Set timedoutServers; + private final Map timeoutTrackers; + private final Set timedoutServers; private final long retryTimeout; private final ClientTabletCache locator; @@ -243,7 +242,7 @@ private synchronized void lookup(List ranges, ResultReceiver receiver) List columns = new ArrayList<>(options.fetchedColumns); ranges = Range.mergeOverlapping(ranges); - Map>> binnedRanges = new HashMap<>(); + Map>> binnedRanges = new HashMap<>(); var ssd = binRanges(locator, ranges, binnedRanges); @@ -251,7 +250,7 @@ private synchronized void lookup(List ranges, ResultReceiver receiver) } private ScanServerData binRanges(ClientTabletCache clientTabletCache, List ranges, - Map>> binnedRanges) throws AccumuloException, + Map>> binnedRanges) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, InvalidTabletHostingRequestException { int lastFailureSize = Integer.MAX_VALUE; @@ -317,8 +316,8 @@ private ScanServerData binRanges(ClientTabletCache clientTabletCache, List>> binnedRanges2 = new HashMap<>(); - for (Entry>> entry : binnedRanges.entrySet()) { + Map>> binnedRanges2 = new HashMap<>(); + for (Entry>> entry : binnedRanges.entrySet()) { Map> tabletMap = new HashMap<>(); binnedRanges2.put(entry.getKey(), tabletMap); for (Entry> tabletRanges : entry.getValue().entrySet()) { @@ -360,7 +359,7 @@ private void processFailures(Map> failures, ResultReceiver failSleepTime = Math.min(5000, failSleepTime * 2); - Map>> binnedRanges = new HashMap<>(); + Map>> binnedRanges = new HashMap<>(); List allRanges = new ArrayList<>(); for (List ranges : failures.values()) { @@ -382,7 +381,7 @@ private String getTableInfo() { private class QueryTask implements Runnable { - private final String tsLocation; + private final ServerId tsLocation; private final Map> tabletsRanges; private final ResultReceiver receiver; private Semaphore semaphore = null; @@ -393,7 +392,7 @@ private class QueryTask implements Runnable { private final ScanServerAttemptReporter reporter; private final Duration scanServerSelectorDelay; - QueryTask(String tsLocation, Map> tabletsRanges, + QueryTask(ServerId tsLocation, Map> tabletsRanges, Map> failures, ResultReceiver receiver, List columns, long busyTimeout, ScanServerAttemptReporter reporter, Duration scanServerSelectorDelay) { this.tsLocation = tsLocation; @@ -528,7 +527,7 @@ public void run() { } - private void doLookups(Map>> binnedRanges, + private void doLookups(Map>> binnedRanges, final ResultReceiver receiver, List columns, ScanServerData ssd) { int maxTabletsPerRequest = Integer.MAX_VALUE; @@ -539,7 +538,7 @@ private void doLookups(Map>> binnedRanges, // following code determines if this is the case if (numThreads / binnedRanges.size() > 1) { int totalNumberOfTablets = 0; - for (Entry>> entry : binnedRanges.entrySet()) { + for (Entry>> entry : binnedRanges.entrySet()) { totalNumberOfTablets += entry.getValue().size(); } @@ -561,9 +560,9 @@ private void doLookups(Map>> binnedRanges, if (!timedoutServers.isEmpty()) { // go ahead and fail any timed out servers - for (Iterator>>> iterator = + for (Iterator>>> iterator = binnedRanges.entrySet().iterator(); iterator.hasNext();) { - Entry>> entry = iterator.next(); + Entry>> entry = iterator.next(); if (timedoutServers.contains(entry.getKey())) { failures.putAll(entry.getValue()); iterator.remove(); @@ -573,12 +572,12 @@ private void doLookups(Map>> binnedRanges, // randomize tabletserver order... this will help when there are multiple // batch readers and writers running against accumulo - List locations = new ArrayList<>(binnedRanges.keySet()); + List locations = new ArrayList<>(binnedRanges.keySet()); Collections.shuffle(locations); List queryTasks = new ArrayList<>(); - for (final String tsLocation : locations) { + for (final ServerId tsLocation : locations) { final Map> tabletsRanges = binnedRanges.get(tsLocation); if (maxTabletsPerRequest == Integer.MAX_VALUE || tabletsRanges.size() == 1) { @@ -619,7 +618,7 @@ private void doLookups(Map>> binnedRanges, private static class ScanServerData { final List failures; final ScanServerSelections actions; - final Map reporters; + final Map reporters; public ScanServerData(List failures) { this.failures = failures; @@ -628,7 +627,7 @@ public ScanServerData(List failures) { } public ScanServerData(ScanServerSelections actions, - Map reporters) { + Map reporters) { this.actions = actions; this.reporters = reporters; this.failures = List.of(); @@ -650,13 +649,13 @@ public Duration getDelay() { } private ScanServerData binRangesForScanServers(ClientTabletCache clientTabletCache, - List ranges, Map>> binnedRanges, Timer startTime) + List ranges, Map>> binnedRanges, Timer startTime) throws AccumuloException, TableNotFoundException, AccumuloSecurityException, InvalidTabletHostingRequestException { ScanServerSelector ecsm = context.getScanServerSelector(); - Map extentToTserverMap = new HashMap<>(); + Map extentToTserverMap = new HashMap<>(); Map> extentToRangesMap = new HashMap<>(); Set tabletIds = new HashSet<>(); @@ -705,13 +704,13 @@ public Optional waitUntil(Supplier> condition, Duration maxWa var actions = ecsm.selectServers(params); - Map reporters = new HashMap<>(); + Map reporters = new HashMap<>(); failures = new ArrayList<>(); for (TabletIdImpl tabletId : tabletIds) { KeyExtent extent = tabletId.toKeyExtent(); - String serverToUse = actions.getScanServer(tabletId); + ServerId serverToUse = actions.getScanServer(tabletId); if (serverToUse == null) { // no scan server was given so use the tablet server serverToUse = extentToTserverMap.get(extent); @@ -797,13 +796,13 @@ static void trackScanning(Map> failures, private static class TimeoutTracker { - String server; - Set badServers; + ServerId server; + Set badServers; final long timeOut; long activityTime; Long firstErrorTime = null; - TimeoutTracker(String server, Set badServers, long timeOut) { + TimeoutTracker(ServerId server, Set badServers, long timeOut) { this(timeOut); this.server = server; this.badServers = badServers; @@ -843,7 +842,7 @@ public long getTimeOut() { } } - public static void doLookup(ClientContext context, String server, + public static void doLookup(ClientContext context, ServerId server, Map> requested, Map> failures, Map> unscanned, ResultReceiver receiver, List columns, ScannerOptions options, Authorizations authorizations) @@ -852,7 +851,7 @@ public static void doLookup(ClientContext context, String server, authorizations, new TimeoutTracker(Long.MAX_VALUE), 0L); } - static void doLookup(ClientContext context, String server, Map> requested, + static void doLookup(ClientContext context, ServerId server, Map> requested, Map> failures, Map> unscanned, ResultReceiver receiver, List columns, ScannerOptions options, Authorizations authorizations, TimeoutTracker timeoutTracker, long busyTimeout) @@ -873,13 +872,12 @@ static void doLookup(ClientContext context, String server, Map latencyTimerFuture; - private final Map timeoutTrackers = + private final Map timeoutTrackers = Collections.synchronizedMap(new HashMap<>()); // stats @@ -166,7 +166,7 @@ public class TabletServerBatchWriter implements AutoCloseable { // error handling private final Violations violations = new Violations(); private final Map> authorizationFailures = new HashMap<>(); - private final HashSet serverSideErrors = new HashSet<>(); + private final HashSet serverSideErrors = new HashSet<>(); private final FailedMutations failedMutations; private int unknownErrors = 0; private final AtomicBoolean somethingFailed = new AtomicBoolean(false); @@ -174,12 +174,12 @@ public class TabletServerBatchWriter implements AutoCloseable { private static class TimeoutTracker { - final String server; + final ServerId server; final long timeOut; long activityTime; Long firstErrorTime = null; - TimeoutTracker(String server, long timeOut) { + TimeoutTracker(ServerId server, long timeOut) { this.timeOut = timeOut; this.server = server; } @@ -455,7 +455,7 @@ private void updateSendStats(long count, long time) { } public void updateBinningStats(int count, long time, - Map> binnedMutations) { + Map> binnedMutations) { if (log.isTraceEnabled()) { totalBinTime.addAndGet(time); totalBinned.addAndGet(count); @@ -477,7 +477,7 @@ private static void computeMax(AtomicInteger stat, int update) { } } - private void updateBatchStats(Map> binnedMutations) { + private void updateBatchStats(Map> binnedMutations) { tabletServersBatchSum.addAndGet(binnedMutations.size()); computeMin(minTabletServersBatch, binnedMutations.size()); @@ -485,7 +485,7 @@ private void updateBatchStats(Map> binned int numTablets = 0; - for (Entry> entry : binnedMutations.entrySet()) { + for (Entry> entry : binnedMutations.entrySet()) { TabletServerMutations tsm = entry.getValue(); numTablets += tsm.getMutations().size(); } @@ -542,7 +542,7 @@ private void updateAuthorizationFailures(Map author } } - private synchronized void updateServerErrors(String server, Exception e) { + private synchronized void updateServerErrors(ServerId server, Exception e) { somethingFailed.set(true); this.serverSideErrors.add(server); this.notifyAll(); @@ -667,8 +667,8 @@ private class MutationWriter { private static final int MUTATION_BATCH_SIZE = 1 << 17; private final ThreadPoolExecutor sendThreadPool; private final ThreadPoolExecutor binningThreadPool; - private final Map> serversMutations; - private final Set queued; + private final Map> serversMutations; + private final Set queued; private final Map locators; public MutationWriter(int numSendThreads) { @@ -693,7 +693,7 @@ private synchronized ClientTabletCache getLocator(TableId tableId) { } private void binMutations(MutationSet mutationsToProcess, - Map> binnedMutations) { + Map> binnedMutations) { TableId tableId = null; try { Set>> es = mutationsToProcess.getMutations().entrySet(); @@ -752,7 +752,7 @@ void queueMutations(final MutationSet mutationsToSend) { } private void addMutations(MutationSet mutationsToSend) { - Map> binnedMutations = new HashMap<>(); + Map> binnedMutations = new HashMap<>(); Span span = TraceUtil.startSpan(this.getClass(), "binMutations"); try (Scope scope = span.makeCurrent()) { long t1 = System.currentTimeMillis(); @@ -769,13 +769,13 @@ private void addMutations(MutationSet mutationsToSend) { } private synchronized void - addMutations(Map> binnedMutations) { + addMutations(Map> binnedMutations) { int count = 0; // merge mutations into existing mutations for a tablet server - for (Entry> entry : binnedMutations.entrySet()) { - String server = entry.getKey(); + for (Entry> entry : binnedMutations.entrySet()) { + ServerId server = entry.getKey(); TabletServerMutations currentMutations = serversMutations.get(server); @@ -805,10 +805,10 @@ private void addMutations(MutationSet mutationsToSend) { } // randomize order of servers - ArrayList servers = new ArrayList<>(binnedMutations.keySet()); + ArrayList servers = new ArrayList<>(binnedMutations.keySet()); Collections.shuffle(servers); - for (String server : servers) { + for (ServerId server : servers) { if (!queued.contains(server)) { sendThreadPool.execute(new SendTask(server)); queued.add(server); @@ -816,7 +816,7 @@ private void addMutations(MutationSet mutationsToSend) { } } - private synchronized TabletServerMutations getMutationsToSend(String server) { + private synchronized TabletServerMutations getMutationsToSend(ServerId server) { TabletServerMutations tsmuts = serversMutations.remove(server); if (tsmuts == null) { queued.remove(server); @@ -827,9 +827,9 @@ private synchronized TabletServerMutations getMutationsToSend(String s class SendTask implements Runnable { - private final String location; + private final ServerId location; - SendTask(String server) { + SendTask(ServerId server) { this.location = server; } @@ -931,7 +931,7 @@ public void send(TabletServerMutations tsm) } } - private MutationSet sendMutationsToTabletServer(String location, + private MutationSet sendMutationsToTabletServer(ServerId location, Map> tabMuts, TimeoutTracker timeoutTracker, SessionCloser sessionCloser) throws IOException, AccumuloSecurityException, AccumuloServerException { @@ -945,14 +945,13 @@ private MutationSet sendMutationsToTabletServer(String location, // If there is an open session, must close it before the batchwriter closes or writes could // happen after the batch writer closes. See #3721 try { - final HostAndPort parsedServer = HostAndPort.fromString(location); final TabletIngestClientService.Iface client; if (timeoutTracker.getTimeOut() < context.getClientTimeoutInMillis()) { - client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, parsedServer, context, + client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, location, context, timeoutTracker.getTimeOut()); } else { - client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, parsedServer, context); + client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, location, context); } try { @@ -1048,10 +1047,10 @@ private MutationSet sendMutationsToTabletServer(String location, class SessionCloser implements AutoCloseable { - private final String location; + private final ServerId location; private OptionalLong usid; - SessionCloser(String location) { + SessionCloser(ServerId location) { this.location = location; usid = OptionalLong.empty(); } @@ -1082,7 +1081,7 @@ public void close() throws ThriftSecurityException { /** * Checks if there is a lock held by a tserver at a specific host and port. */ - private boolean isALockHeld(String tserver) { + private boolean isALockHeld(ServerId tserver) { return context.getTServerLockChecker().doesTabletServerLockExist(tserver); } @@ -1092,8 +1091,6 @@ private void cancelSession() throws InterruptedException, ThriftSecurityExceptio .incrementBy(Duration.ofMillis(100)).maxWait(Duration.ofMinutes(1)).backOffFactor(1.5) .logInterval(Duration.ofMinutes(3)).createRetry(); - final HostAndPort parsedServer = HostAndPort.fromString(location); - long startTime = System.nanoTime(); // If somethingFailed is true then the batch writer will throw an exception on close or @@ -1116,10 +1113,10 @@ private void cancelSession() throws InterruptedException, ThriftSecurityExceptio try { if (timeout < context.getClientTimeoutInMillis()) { - client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, parsedServer, context, - timeout); + client = + ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, location, context, timeout); } else { - client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, parsedServer, context); + client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, location, context); } if (client.cancelUpdate(TraceUtil.traceInfo(), usid.getAsLong())) { retry.logCompletion(log, "Canceled failed write session " + location + " " + usid); diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java index 4d6c792c0d5..c3fbcf49186 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java @@ -46,6 +46,7 @@ import org.apache.accumulo.core.client.SampleNotPresentException; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.TimedOutException; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.client.sample.SamplerConfiguration; import org.apache.accumulo.core.clientImpl.ClientTabletCache.CachedTablet; import org.apache.accumulo.core.clientImpl.thrift.TInfo; @@ -88,7 +89,6 @@ import org.slf4j.LoggerFactory; import com.google.common.base.Preconditions; -import com.google.common.net.HostAndPort; import io.opentelemetry.api.trace.Span; import io.opentelemetry.context.Scope; @@ -107,7 +107,7 @@ public class ThriftScanner { // the restarted client may not see the write unless we wait here. // this behavior is very important when the client is reading the // metadata - public static final Map> serversWaitedForWrites = + public static final Map> serversWaitedForWrites = new EnumMap<>(TabletType.class); static { @@ -117,7 +117,7 @@ public class ThriftScanner { } public static boolean getBatchFromServer(ClientContext context, Range range, KeyExtent extent, - String server, SortedMap results, SortedSet fetchedColumns, + ServerId server, SortedMap results, SortedSet fetchedColumns, List serverSideIteratorList, Map> serverSideIteratorOptions, int size, Authorizations authorizations, long batchTimeOut, String classLoaderContext) @@ -126,11 +126,10 @@ public static boolean getBatchFromServer(ClientContext context, Range range, Key throw new AccumuloException(new IOException()); } - final HostAndPort parsedServer = HostAndPort.fromString(server); try { TInfo tinfo = TraceUtil.traceInfo(); TabletScanClientService.Client client = - ThriftUtil.getClient(ThriftClientTypes.TABLET_SCAN, parsedServer, context); + ThriftUtil.getClient(ThriftClientTypes.TABLET_SCAN, server, context); try { // not reading whole rows (or stopping on row boundaries) so there is no need to enable // isolation below @@ -183,11 +182,11 @@ enum ServerType { } static class ScanAddress { - final String serverAddress; + final ServerId serverAddress; final ServerType serverType; final CachedTablet tabletInfo; - public ScanAddress(String serverAddress, ServerType serverType, CachedTablet tabletInfo) { + public ScanAddress(ServerId serverAddress, ServerType serverType, CachedTablet tabletInfo) { this.serverAddress = Objects.requireNonNull(serverAddress); this.serverType = Objects.requireNonNull(serverType); this.tabletInfo = Objects.requireNonNull(tabletInfo); @@ -433,7 +432,7 @@ public Optional waitUntil(Supplier> condition, Duration maxWa Duration delay = null; - String scanServer = actions.getScanServer(tabletId); + ServerId scanServer = actions.getScanServer(tabletId); if (scanServer != null) { addr = new ScanAddress(scanServer, ServerType.SSERVER, loc); delay = actions.getDelay(); @@ -441,10 +440,10 @@ public Optional waitUntil(Supplier> condition, Duration maxWa log.trace("For tablet {} scan server selector chose scan_server:{} delay:{} busyTimeout:{}", loc.getExtent(), scanServer, delay, scanState.busyTimeout); } else { - Optional tserverLoc = loc.getTserverLocation(); + Optional tserverLoc = loc.getTserverLocation(); if (tserverLoc.isPresent()) { - addr = new ScanAddress(loc.getTserverLocation().orElseThrow(), ServerType.TSERVER, loc); + addr = new ScanAddress(tserverLoc.orElseThrow(), ServerType.TSERVER, loc); delay = actions.getDelay(); scanState.busyTimeout = Duration.ZERO; log.trace("For tablet {} scan server selector chose tablet_server: {}", loc.getExtent(), @@ -669,7 +668,7 @@ public static List scan(ClientContext context, ScanState scanState, Du } Span child2 = TraceUtil.startSpan(ThriftScanner.class, "scan::location", - Map.of("tserver", addr.serverAddress)); + Map.of("tserver", addr.serverAddress.toHostPortString())); try (Scope scanLocation = child2.makeCurrent()) { results = scan(addr, scanState, context); } catch (AccumuloSecurityException e) { @@ -854,9 +853,8 @@ private static List scanRpc(ScanAddress addr, ScanState scanState, final TInfo tinfo = TraceUtil.traceInfo(); - final HostAndPort parsedLocation = HostAndPort.fromString(addr.serverAddress); TabletScanClientService.Client client = - ThriftUtil.getClient(ThriftClientTypes.TABLET_SCAN, parsedLocation, context); + ThriftUtil.getClient(ThriftClientTypes.TABLET_SCAN, addr.serverAddress, context); String old = Thread.currentThread().getName(); try { @@ -988,11 +986,10 @@ static void close(ScanState scanState) { log.trace("Closing active scan {} {} {}", scanState.prevLoc.serverType, scanState.prevLoc.serverAddress, scanState.scanID); - HostAndPort parsedLocation = HostAndPort.fromString(scanState.prevLoc.serverAddress); TabletScanClientService.Client client = null; try { - client = - ThriftUtil.getClient(ThriftClientTypes.TABLET_SCAN, parsedLocation, scanState.context); + client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SCAN, + scanState.prevLoc.serverAddress, scanState.context); client.closeScan(tinfo, scanState.scanID); } catch (TException e) { // ignore this is a best effort diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportKey.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportKey.java index ee93871ecf3..5d314788c29 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportKey.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportKey.java @@ -22,17 +22,17 @@ import java.util.Objects; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.rpc.SaslConnectionParams; import org.apache.accumulo.core.rpc.SslConnectionParams; import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; import com.google.common.annotations.VisibleForTesting; -import com.google.common.net.HostAndPort; @VisibleForTesting public class ThriftTransportKey { private final ThriftClientTypes type; - private final HostAndPort server; + private final ServerId server; private final long timeout; private final SslConnectionParams sslParams; private final SaslConnectionParams saslParams; @@ -40,7 +40,7 @@ public class ThriftTransportKey { private final int hash; @VisibleForTesting - public ThriftTransportKey(ThriftClientTypes type, HostAndPort server, long timeout, + public ThriftTransportKey(ThriftClientTypes type, ServerId server, long timeout, ClientContext context) { this(type, server, timeout, context.getClientSslParams(), context.getSaslParams()); } @@ -48,7 +48,7 @@ public ThriftTransportKey(ThriftClientTypes type, HostAndPort server, long ti /** * Visible only for testing */ - ThriftTransportKey(ThriftClientTypes type, HostAndPort server, long timeout, + ThriftTransportKey(ThriftClientTypes type, ServerId server, long timeout, SslConnectionParams sslParams, SaslConnectionParams saslParams) { requireNonNull(server, "location is null"); this.type = type; @@ -69,7 +69,7 @@ public ThriftClientTypes getType() { } @VisibleForTesting - public HostAndPort getServer() { + public ServerId getServer() { return server; } diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportPool.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportPool.java index b5b9ff51edb..fe8fb859861 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportPool.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportPool.java @@ -39,20 +39,17 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; -import java.util.function.Function; import java.util.function.LongSupplier; import java.util.function.Supplier; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.lock.ServiceLockData.ThriftService; -import org.apache.accumulo.core.lock.ServiceLockPaths.AddressSelector; import org.apache.accumulo.core.lock.ServiceLockPaths.ResourceGroupPredicate; -import org.apache.accumulo.core.lock.ServiceLockPaths.ServiceLockPath; import org.apache.accumulo.core.rpc.ThriftUtil; import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; import org.apache.accumulo.core.util.Pair; import org.apache.accumulo.core.util.Timer; import org.apache.accumulo.core.util.threads.Threads; -import org.apache.commons.collections4.set.CompositeSet; import org.apache.thrift.TConfiguration; import org.apache.thrift.transport.TTransport; import org.apache.thrift.transport.TTransportException; @@ -60,7 +57,6 @@ import org.slf4j.LoggerFactory; import com.google.common.base.Preconditions; -import com.google.common.net.HostAndPort; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; @@ -131,10 +127,10 @@ static ThriftTransportPool startNew(LongSupplier maxAgeMillis, boolean shouldHal return pool; } - public TTransport getTransport(ThriftClientTypes type, HostAndPort location, long milliseconds, + public TTransport getTransport(ThriftClientTypes type, ServerId server, long milliseconds, ClientContext context, boolean preferCached) throws TTransportException { - ThriftTransportKey cacheKey = new ThriftTransportKey(type, location, milliseconds, context); + ThriftTransportKey cacheKey = new ThriftTransportKey(type, server, milliseconds, context); if (preferCached) { CachedConnection connection = connectionPool.reserveAny(cacheKey); if (connection != null) { @@ -145,33 +141,14 @@ public TTransport getTransport(ThriftClientTypes type, HostAndPort location, return createNewTransport(cacheKey); } - @SuppressWarnings("unchecked") - public Pair getAnyCachedTransport(ThriftClientTypes type, ClientContext ctx, - ThriftService service, ResourceGroupPredicate rgp) { + public Pair getAnyCachedTransport(ThriftClientTypes type, + ClientContext ctx, ThriftService service, ResourceGroupPredicate rgp) { final Timer timer = Timer.startNew(); final List serversSet = new ArrayList<>(); - Function> paths = switch (service) { - case CLIENT -> (selector) -> new CompositeSet( - ctx.getServerPaths().getCompactor(rgp, selector, true), - ctx.getServerPaths().getScanServer(rgp, selector, true), - ctx.getServerPaths().getTabletServer(rgp, selector, true)); - case COMPACTOR -> (selector) -> ctx.getServerPaths().getCompactor(rgp, selector, true); - case MANAGER, COORDINATOR, FATE -> - (selector) -> Set.of(ctx.getServerPaths().getManager(true)); - case GC -> (selector) -> Set.of(ctx.getServerPaths().getGarbageCollector(true)); - case TABLET_SCAN -> (selector) -> new CompositeSet( - ctx.getServerPaths().getTabletServer(rgp, selector, true), - ctx.getServerPaths().getScanServer(rgp, selector, true)); - case TABLET_INGEST, TABLET_MANAGEMENT, TSERV -> - (selector) -> ctx.getServerPaths().getTabletServer(rgp, selector, true); - default -> throw new IllegalArgumentException("Unhandled thrift service type: " + service); - }; - for (ThriftTransportKey ttk : connectionPool.getThriftTransportKeys()) { - if (ttk.getType().equals(type) - && !paths.apply(AddressSelector.exact(ttk.getServer())).isEmpty()) { + if (ttk.getType().equals(type) && rgp.test(ttk.getServer().getResourceGroup())) { serversSet.add(ttk); } } @@ -182,10 +159,9 @@ public Pair getAnyCachedTransport(ThriftClientTypes type, for (ThriftTransportKey ttk : serversSet) { CachedConnection connection = connectionPool.reserveAny(ttk); if (connection != null) { - final String serverAddr = ttk.getServer().toString(); log.trace("Took {} ms to evaluate existing connection to {}", - timer.elapsed(TimeUnit.MILLISECONDS), serverAddr); - return new Pair<>(serverAddr, connection.transport); + timer.elapsed(TimeUnit.MILLISECONDS), ttk.getServer()); + return new Pair<>(ttk.getServer(), connection.transport); } } return null; diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TimeoutClientTabletCache.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TimeoutClientTabletCache.java index 77c93de7d4c..0ff5d9e7228 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TimeoutClientTabletCache.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TimeoutClientTabletCache.java @@ -26,6 +26,7 @@ import org.apache.accumulo.core.client.InvalidTabletHostingRequestException; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.TimedOutException; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.TableId; @@ -84,7 +85,7 @@ public CachedTablet findTablet(ClientContext context, Text row, boolean skipRow, @Override public void binMutations(ClientContext context, List mutations, - Map> binnedMutations, List failures) + Map> binnedMutations, List failures) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, InvalidTabletHostingRequestException { try { @@ -104,7 +105,7 @@ public void binMutations(ClientContext context, List mut @Override public List binRanges(ClientContext context, List ranges, - Map>> binnedRanges) throws AccumuloException, + Map>> binnedRanges) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, InvalidTabletHostingRequestException { try { List ret = super.binRanges(context, ranges, binnedRanges); diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java index 68382a867c8..7732fe7b250 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java @@ -23,6 +23,7 @@ import java.util.Set; import org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ClientTabletCacheImpl.TabletServerLockChecker; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.lock.ServiceLockPaths; @@ -43,20 +44,20 @@ public class ZookeeperLockChecker implements TabletServerLockChecker { this.lockPaths = new ServiceLockPaths(this.zc); } - public boolean doesTabletServerLockExist(String server) { + public boolean doesTabletServerLockExist(ServerId server) { // ServiceLockPaths only returns items that have a lock - var hostAndPort = HostAndPort.fromString(server); - Set tservers = lockPaths.getTabletServer(ResourceGroupPredicate.ANY, - AddressSelector.exact(hostAndPort), true); + Set tservers = + lockPaths.getTabletServer(ResourceGroupPredicate.exact(server.getResourceGroup()), + AddressSelector.exact(HostAndPort.fromParts(server.getHost(), server.getPort())), true); return !tservers.isEmpty(); } @Override - public boolean isLockHeld(String server, String session) { + public boolean isLockHeld(ServerId server, String session) { // ServiceLockPaths only returns items that have a lock - var hostAndPort = HostAndPort.fromString(server); - Set tservers = lockPaths.getTabletServer(ResourceGroupPredicate.ANY, - AddressSelector.exact(hostAndPort), true); + Set tservers = + lockPaths.getTabletServer(ResourceGroupPredicate.exact(server.getResourceGroup()), + AddressSelector.exact(HostAndPort.fromParts(server.getHost(), server.getPort())), true); for (ServiceLockPath slp : tservers) { if (ServiceLock.getSessionId(zc, slp) == Long.parseLong(session, 16)) { return true; diff --git a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockData.java b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockData.java index c29879f05c7..99d42adce09 100644 --- a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockData.java +++ b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockData.java @@ -31,10 +31,9 @@ import java.util.UUID; import java.util.stream.Collectors; +import org.apache.accumulo.core.client.admin.servers.ServerId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.ResourceGroupId; -import org.apache.accumulo.core.util.AddressUtil; - -import com.google.common.net.HostAndPort; public class ServiceLockData implements Comparable { @@ -63,15 +62,12 @@ public static class ServiceDescriptor { private final UUID uuid; private final ThriftService service; - private final String address; - private final ResourceGroupId group; + private final ServerId server; - public ServiceDescriptor(UUID uuid, ThriftService service, String address, - ResourceGroupId group) { + public ServiceDescriptor(UUID uuid, ThriftService service, ServerId server) { this.uuid = requireNonNull(uuid); this.service = requireNonNull(service); - this.address = requireNonNull(address); - this.group = requireNonNull(group); + this.server = requireNonNull(server); } public UUID getUUID() { @@ -82,12 +78,12 @@ public ThriftService getService() { return service; } - public String getAddress() { - return address; + public ServerId getServer() { + return server; } public ResourceGroupId getGroup() { - return group; + return server.getResourceGroup(); } @Override @@ -116,8 +112,7 @@ public String toString() { } private String serialize() { - return GSON.get() - .toJson(new ServiceDescriptorGson(uuid, service, address, group.canonical())); + return GSON.get().toJson(new ServiceDescriptorGson(uuid, service, server)); } } @@ -152,19 +147,14 @@ public ServiceLockData(ServiceDescriptors sds) { sds.getServices().forEach(sd -> this.services.put(sd.getService(), sd)); } - public ServiceLockData(UUID uuid, String address, ThriftService service, ResourceGroupId group) { - this(new ServiceDescriptors(new HashSet<>( - Collections.singleton(new ServiceDescriptor(uuid, service, address, group))))); + public ServiceLockData(UUID uuid, ServerId address, ThriftService service) { + this(new ServiceDescriptors( + new HashSet<>(Collections.singleton(new ServiceDescriptor(uuid, service, address))))); } - public String getAddressString(ThriftService service) { + public ServerId getServer(ThriftService service) { ServiceDescriptor sd = services.get(service); - return sd == null ? null : sd.getAddress(); - } - - public HostAndPort getAddress(ThriftService service) { - String s = getAddressString(service); - return s == null ? null : AddressUtil.parseAddress(s); + return sd == null ? null : sd.getServer(); } public ResourceGroupId getGroup(ThriftService service) { @@ -179,9 +169,9 @@ public UUID getServerUUID(ThriftService service) { public byte[] serialize() { ServiceDescriptorsGson json = new ServiceDescriptorsGson(); - json.descriptors = services.values().stream() - .map(s -> new ServiceDescriptorGson(s.uuid, s.service, s.address, s.group.canonical())) - .collect(Collectors.toSet()); + json.descriptors = + services.values().stream().map(s -> new ServiceDescriptorGson(s.uuid, s.service, s.server)) + .collect(Collectors.toSet()); return GSON.get().toJson(json).getBytes(UTF_8); } @@ -220,25 +210,23 @@ public static ServiceDescriptors parseServiceDescriptors(String data) { private static ServiceDescriptors deserialize(ServiceDescriptorsGson json) { return new ServiceDescriptors(json.descriptors.stream() - .map(s -> new ServiceDescriptor(s.uuid, s.service, s.address, ResourceGroupId.of(s.group))) + .map(s -> new ServiceDescriptor(s.uuid, s.service, s.address.getServerId())) .collect(Collectors.toSet())); } private static class ServiceDescriptorGson { private UUID uuid; private ThriftService service; - private String address; - private String group; + private ServerIdUtil.ServerIdInfo address; // default constructor required for Gson @SuppressWarnings("unused") public ServiceDescriptorGson() {} - public ServiceDescriptorGson(UUID uuid, ThriftService service, String address, String group) { + public ServiceDescriptorGson(UUID uuid, ThriftService service, ServerId address) { this.uuid = uuid; this.service = service; - this.address = address; - this.group = group; + this.address = ServerIdUtil.toServerIdInfo(address); } } diff --git a/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java b/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java index 99b1e294626..e55f862e9b3 100644 --- a/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java +++ b/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java @@ -27,6 +27,7 @@ import java.util.UUID; import org.apache.accumulo.core.client.admin.compaction.CompactableFile; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.fate.FateId; @@ -46,7 +47,6 @@ import org.slf4j.LoggerFactory; import com.google.common.collect.Collections2; -import com.google.common.net.HostAndPort; /** * This class contains source for logs messages about a tablets internal state, like its location, @@ -79,8 +79,7 @@ public static void loading(KeyExtent extent, TServerInstance server) { locLog.debug("Loading {} on {}", extent, server); } - public static void suspended(KeyExtent extent, HostAndPort server, SteadyTime time, - int numWalogs) { + public static void suspended(KeyExtent extent, ServerId server, SteadyTime time, int numWalogs) { locLog.debug("Suspended {} to {} at {} ms with {} walogs", extent, server, time, numWalogs); } diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletServerIdImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletServerIdImpl.java index 013515a2530..144f5580545 100644 --- a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletServerIdImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletServerIdImpl.java @@ -20,11 +20,11 @@ import static java.util.Objects.requireNonNull; +import org.apache.accumulo.core.client.admin.servers.ServerId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.core.spi.balancer.data.TabletServerId; -import com.google.common.net.HostAndPort; - /** * @since 2.1.0 */ @@ -36,8 +36,12 @@ public static TabletServerIdImpl fromThrift(TServerInstance tsi) { } public TabletServerIdImpl(String host, int port, String session) { - requireNonNull(host); - this.tServerInstance = new TServerInstance(HostAndPort.fromParts(host, port), session); + this.tServerInstance = new TServerInstance(ServerIdUtil.tserver(host, port), session); + } + + public TabletServerIdImpl(ServerId server, String session) { + requireNonNull(server); + this.tServerInstance = new TServerInstance(server, session); } public TabletServerIdImpl(TServerInstance tServerInstance) { @@ -49,13 +53,8 @@ public TServerInstance getTServerInstance() { } @Override - public String getHost() { - return tServerInstance.getHostAndPort().getHost(); - } - - @Override - public int getPort() { - return tServerInstance.getHostAndPort().getPort(); + public ServerId getServer() { + return tServerInstance.getServer(); } @Override @@ -87,7 +86,7 @@ public int hashCode() { @Override public String toString() { - return getHost() + ':' + getPort() + '[' + getSession() + ']'; + return getServer().toHostPortString() + '[' + getSession() + ']'; } public TServerInstance toThrift() { @@ -98,9 +97,7 @@ public static TServerInstance toThrift(TabletServerId tabletServerId) { if (tabletServerId instanceof TabletServerIdImpl) { return ((TabletServerIdImpl) tabletServerId).toThrift(); } else { - return new TServerInstance( - HostAndPort.fromParts(tabletServerId.getHost(), tabletServerId.getPort()), - tabletServerId.getSession()); + return new TServerInstance(tabletServerId.getServer(), tabletServerId.getSession()); } } } diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataCachedTabletObtainer.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataCachedTabletObtainer.java index 0ddc12701c1..5a9212a541c 100644 --- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataCachedTabletObtainer.java +++ b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataCachedTabletObtainer.java @@ -155,7 +155,7 @@ private void decodeRows(TreeMap encodedResults, TreeMap re } public static CachedTablets getMetadataLocationEntries(SortedMap entries) { - Text location = null; + TServerInstance location = null; Text session = null; TabletAvailability tabletAvailability = null; boolean hostingRequested = false; @@ -189,7 +189,7 @@ public static CachedTablets getMetadataLocationEntries(SortedMap entr if (location != null) { throw new IllegalStateException("Tablet has multiple locations : " + lastRowFromKey); } - location = new Text(val.toString()); + location = TServerInstance.deserialize(val.toString()); session = new Text(colq); } else if (TabletColumnFamily.AVAILABILITY_COLUMN.equals(colf, colq)) { tabletAvailability = TabletAvailabilityUtil.fromValue(val); @@ -201,7 +201,7 @@ public static CachedTablets getMetadataLocationEntries(SortedMap entr tabletAvailability = TabletAvailability.HOSTED; } if (location != null) { - results.add(new CachedTablet(ke, location.toString(), session.toString(), + results.add(new CachedTablet(ke, location.getServer(), session.toString(), tabletAvailability, hostingRequested)); } else { results.add(new CachedTablet(ke, tabletAvailability, hostingRequested)); diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/SuspendingTServer.java b/core/src/main/java/org/apache/accumulo/core/metadata/SuspendingTServer.java index 85bafa71792..0e331a017c1 100644 --- a/core/src/main/java/org/apache/accumulo/core/metadata/SuspendingTServer.java +++ b/core/src/main/java/org/apache/accumulo/core/metadata/SuspendingTServer.java @@ -18,39 +18,47 @@ */ package org.apache.accumulo.core.metadata; +import static org.apache.accumulo.core.util.LazySingletons.GSON; + +import java.io.Serializable; +import java.time.Duration; import java.util.Objects; -import java.util.concurrent.TimeUnit; import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.metadata.TServerInstance.TServerInstanceInfo; import org.apache.accumulo.core.util.time.SteadyTime; -import com.google.common.net.HostAndPort; - /** * For a suspended tablet, the time of suspension and the server it was suspended from. */ -public class SuspendingTServer { - public final HostAndPort server; +public class SuspendingTServer implements Serializable { + + public static record SuspendingTServerInfo(TServerInstanceInfo tsi, long millis) { + public SuspendingTServer getSTS() { + return new SuspendingTServer(tsi.getTSI(), SteadyTime.from(Duration.ofMillis(millis))); + } + } + + private static final long serialVersionUID = 1L; + + public final TServerInstance server; public final SteadyTime suspensionTime; - public SuspendingTServer(HostAndPort server, SteadyTime suspensionTime) { + public SuspendingTServer(TServerInstance server, SteadyTime suspensionTime) { this.server = Objects.requireNonNull(server); this.suspensionTime = Objects.requireNonNull(suspensionTime); } public static SuspendingTServer fromValue(Value value) { - String valStr = value.toString(); - String[] parts = valStr.split("[|]", 2); - return new SuspendingTServer(HostAndPort.fromString(parts[0]), - SteadyTime.from(Long.parseLong(parts[1]), TimeUnit.MILLISECONDS)); + return GSON.get().fromJson(value.toString(), SuspendingTServerInfo.class).getSTS(); } - public static Value toValue(TServerInstance tServer, SteadyTime suspensionTime) { - return new Value(tServer.getHostPort() + "|" + suspensionTime.getMillis()); + private SuspendingTServerInfo toSuspendingTServerInfo() { + return new SuspendingTServerInfo(server.getTServerInstanceInfo(), suspensionTime.getMillis()); } public Value toValue() { - return new Value(server + "|" + suspensionTime.getMillis()); + return new Value(GSON.get().toJson(toSuspendingTServerInfo())); } @Override diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/TServerInstance.java b/core/src/main/java/org/apache/accumulo/core/metadata/TServerInstance.java index 9e261025258..d5948cc4b18 100644 --- a/core/src/main/java/org/apache/accumulo/core/metadata/TServerInstance.java +++ b/core/src/main/java/org/apache/accumulo/core/metadata/TServerInstance.java @@ -18,12 +18,19 @@ */ package org.apache.accumulo.core.metadata; -import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.accumulo.core.util.LazySingletons.GSON; -import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.util.AddressUtil; -import org.apache.hadoop.io.Text; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.Serializable; +import java.util.Objects; +import org.apache.accumulo.core.client.admin.servers.ServerId; +import org.apache.accumulo.core.client.admin.servers.ServerId.Type; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; +import org.apache.accumulo.core.data.ResourceGroupId; + +import com.google.common.base.Preconditions; import com.google.common.net.HostAndPort; /** @@ -32,41 +39,67 @@ * Therefore tablet assignments can be considered out-of-date if the tablet server instance * information has been changed. */ -public class TServerInstance implements Comparable { +public class TServerInstance implements Comparable, Serializable { - private final HostAndPort hostAndPort; - private final String hostPort; - private final String session; - private final String hostPortSession; + private static final long serialVersionUID = 1L; - public TServerInstance(HostAndPort address, String session) { - this.hostAndPort = address; - this.session = session; - this.hostPort = hostAndPort.toString(); - this.hostPortSession = hostPort + "[" + session + "]"; + public static record TServerInstanceInfo(ServerIdUtil.ServerIdInfo server, String session) { + public TServerInstance getTSI() { + return new TServerInstance(server.getServerId(), session); + } } - public TServerInstance(String formattedString) { - int pos = formattedString.indexOf("["); - if (pos < 0 || !formattedString.endsWith("]")) { - throw new IllegalArgumentException(formattedString); - } - this.hostAndPort = HostAndPort.fromString(formattedString.substring(0, pos)); - this.session = formattedString.substring(pos + 1, formattedString.length() - 1); - this.hostPort = hostAndPort.toString(); - this.hostPortSession = hostPort + "[" + session + "]"; + public static TServerInstance deserialize(String json) { + return GSON.get().fromJson(json, TServerInstanceInfo.class).getTSI(); + } + + public static TServerInstance fromZooKeeperPathString(String zkPath) { + + // TODO: WAL marker serializations using the old format could present a + // problem. If we change the code here to handle the old format, then + // we have to make a guess at the resource group, which could affect + // callers of this method (GcWalsFilter, WalStateManager, LiveTServerSet) + + String parts[] = zkPath.split("\\+"); + Preconditions.checkArgument(parts.length == 3, + "Invalid tserver instance in zk path: " + zkPath); + var rgid = ResourceGroupId.of(parts[0]); + var hostAndPort = HostAndPort.fromString(parts[1]); + var session = parts[2]; + return new TServerInstance( + ServerIdUtil.tserver(rgid, hostAndPort.getHost(), hostAndPort.getPort()), session); + } + + private final ServerId server; + private final String session; + private transient String hostPortSession; + + public TServerInstance(ServerId address, String session) { + Preconditions.checkArgument(address.getType() == Type.TABLET_SERVER, + "ServerId type must be TABLET_SERVER"); + this.server = address; + this.session = session; + setZooKeeperPathString(); } - public TServerInstance(HostAndPort address, long session) { - this(address, Long.toHexString(session)); + public TServerInstance(ServerId address, long session) { + Preconditions.checkArgument(address.getType() == Type.TABLET_SERVER, + "ServerId type must be TABLET_SERVER"); + this.server = address; + this.session = Long.toHexString(session); + setZooKeeperPathString(); } - public TServerInstance(String address, long session) { - this(AddressUtil.parseAddress(address), Long.toHexString(session)); + public TServerInstance(String json) { + var partial = GSON.get().fromJson(json, TServerInstanceInfo.class).getTSI(); + this.server = partial.server; + this.session = partial.session; + setZooKeeperPathString(); } - public TServerInstance(Value address, Text session) { - this(AddressUtil.parseAddress(new String(address.get(), UTF_8)), session.toString()); + private void setZooKeeperPathString() { + this.hostPortSession = server.getResourceGroup().canonical() + "+" + server.toHostPortString() + + "+" + this.session; } @Override @@ -74,12 +107,16 @@ public int compareTo(TServerInstance other) { if (this == other) { return 0; } - return this.getHostPortSession().compareTo(other.getHostPortSession()); + int result = this.getServer().compareTo(other.getServer()); + if (result == 0) { + return this.getSession().compareTo(other.getSession()); + } + return result; } @Override public int hashCode() { - return getHostPortSession().hashCode(); + return Objects.hash(server, session); } @Override @@ -90,28 +127,33 @@ public boolean equals(Object obj) { return false; } + public String toZooKeeperPathString() { + return hostPortSession; + } + @Override public String toString() { - return hostPortSession; + return toZooKeeperPathString(); } - public String getHostPortSession() { - return hostPortSession; + public String getSession() { + return session; } - public String getHost() { - return hostAndPort.getHost(); + public ServerId getServer() { + return server; } - public String getHostPort() { - return hostPort; + public TServerInstanceInfo getTServerInstanceInfo() { + return new TServerInstanceInfo(ServerIdUtil.toServerIdInfo(server), session); } - public HostAndPort getHostAndPort() { - return hostAndPort; + public String serialize() { + return GSON.get().toJson(getTServerInstanceInfo()); } - public String getSession() { - return session; + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + setZooKeeperPathString(); } } diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java index 3ae83c4eca3..31b8356a37b 100644 --- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java +++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java @@ -109,7 +109,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.google.common.net.HostAndPort; public class TabletMetadata { @@ -381,10 +380,6 @@ public static class Location { private final TServerInstance tServerInstance; private final LocationType lt; - private Location(final String server, final String session, final LocationType lt) { - this(new TServerInstance(HostAndPort.fromString(server), session), lt); - } - private Location(final TServerInstance tServerInstance, final LocationType lt) { this.tServerInstance = Objects.requireNonNull(tServerInstance, "tServerInstance must not be null"); @@ -399,26 +394,6 @@ public TServerInstance getServerInstance() { return tServerInstance; } - public String getHostPortSession() { - return tServerInstance.getHostPortSession(); - } - - public String getHost() { - return tServerInstance.getHost(); - } - - public String getHostPort() { - return tServerInstance.getHostPort(); - } - - public HostAndPort getHostAndPort() { - return tServerInstance.getHostAndPort(); - } - - public String getSession() { - return tServerInstance.getSession(); - } - @Override public boolean equals(Object o) { if (this == o) { @@ -450,26 +425,14 @@ public static Location last(TServerInstance instance) { return new Location(instance, LocationType.LAST); } - public static Location last(final String server, final String session) { - return last(new TServerInstance(HostAndPort.fromString(server), session)); - } - public static Location current(TServerInstance instance) { return new Location(instance, LocationType.CURRENT); } - public static Location current(final String server, final String session) { - return current(new TServerInstance(HostAndPort.fromString(server), session)); - } - public static Location future(TServerInstance instance) { return new Location(instance, LocationType.FUTURE); } - public static Location future(final String server, final String session) { - return future(new TServerInstance(HostAndPort.fromString(server), session)); - } - } public TableId getTableId() { @@ -759,13 +722,13 @@ public static > TabletMetadata convertRow(Iterator BulkFileColumnFamily.getBulkLoadTid(val)); break; case CurrentLocationColumnFamily.STR_NAME: - tmBuilder.location(val, qual, LocationType.CURRENT, suppressLocationError); + tmBuilder.location(val, LocationType.CURRENT, suppressLocationError); break; case FutureLocationColumnFamily.STR_NAME: - tmBuilder.location(val, qual, LocationType.FUTURE, suppressLocationError); + tmBuilder.location(val, LocationType.FUTURE, suppressLocationError); break; case LastLocationColumnFamily.STR_NAME: - tmBuilder.last(Location.last(val, qual)); + tmBuilder.last(Location.last(TServerInstance.deserialize(val))); break; case SuspendLocationColumn.STR_NAME: tmBuilder.suspend(SuspendingTServer.fromValue(kv.getValue())); @@ -849,8 +812,8 @@ private static Optional checkTabletServer(ClientContext context ZcStat stat = new ZcStat(); log.trace("Checking server at ZK path: {}", slp); return ServiceLock.getLockData(context.getZooCache(), slp, stat) - .map(sld -> sld.getAddress(ServiceLockData.ThriftService.TSERV)) - .map(address -> new TServerInstance(address, stat.getEphemeralOwner())); + .map(sld -> sld.getServer(ServiceLockData.ThriftService.TSERV)) + .map(server -> new TServerInstance(server, stat.getEphemeralOwner())); } public static void validate(TabletMetadata tm) { @@ -956,15 +919,15 @@ void migration(TServerInstance tserver) { this.migration = tserver; } - void location(String val, String qual, LocationType lt, boolean suppressError) { + void location(String val, LocationType lt, boolean suppressError) { if (location != null) { if (!suppressError) { throw new IllegalStateException("Attempted to set second location for tableId: " + tableId - + " endrow: " + endRow + " -- " + location + " " + qual + " " + val); + + " endrow: " + endRow + " -- " + location + " " + val); } futureAndCurrentLocationSet = true; } - location = new Location(val, qual, lt); + location = new Location(TServerInstance.deserialize(val), lt); } void last(Location last) { diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMutatorBase.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMutatorBase.java index ea02f57d19d..3e488aac01b 100644 --- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMutatorBase.java +++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMutatorBase.java @@ -171,15 +171,16 @@ protected Text getLocationFamilyText(LocationType type) { @Override public T putLocation(Location location) { Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate."); - mutation.put(getLocationFamily(location.getType()), location.getSession(), - location.getHostPort()); + mutation.put(getLocationFamily(location.getType()), location.getServerInstance().getSession(), + location.getServerInstance().serialize()); return getThis(); } @Override public T deleteLocation(Location location) { Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate."); - mutation.putDelete(getLocationFamily(location.getType()), location.getSession()); + mutation.putDelete(getLocationFamily(location.getType()), + location.getServerInstance().getSession()); return getThis(); } @@ -240,7 +241,7 @@ public T putSuspension(TServerInstance tServer, SteadyTime suspensionTime) { Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate."); mutation.put(SuspendLocationColumn.SUSPEND_COLUMN.getColumnFamily(), SuspendLocationColumn.SUSPEND_COLUMN.getColumnQualifier(), - SuspendingTServer.toValue(tServer, suspensionTime)); + new SuspendingTServer(tServer, suspensionTime).toValue()); return getThis(); } @@ -387,7 +388,7 @@ public T putTabletMergeability(TabletMergeabilityMetadata tabletMergeability) { @Override public T putMigration(TServerInstance tserver) { - ServerColumnFamily.MIGRATION_COLUMN.put(mutation, new Value(tserver.getHostPortSession())); + ServerColumnFamily.MIGRATION_COLUMN.put(mutation, new Value(tserver.serialize())); return getThis(); } diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/filters/GcWalsFilter.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/filters/GcWalsFilter.java index 4e1aca1a06a..1ffbcf3584a 100644 --- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/filters/GcWalsFilter.java +++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/filters/GcWalsFilter.java @@ -52,7 +52,7 @@ public class GcWalsFilter extends TabletMetadataFilter { public GcWalsFilter() {} public GcWalsFilter(Set liveTservers) { - String lts = liveTservers.stream().map(TServerInstance::toString).peek(tsi -> { + String lts = liveTservers.stream().map(TServerInstance::toZooKeeperPathString).peek(tsi -> { if (tsi.contains(",")) { throw new IllegalArgumentException(tsi); } @@ -70,7 +70,7 @@ public void init(SortedKeyValueIterator source, Map op liveTservers = Set.of(); } else { liveTservers = Arrays.stream(options.get(LIVE_TSERVER_OPT).split(",")) - .map(TServerInstance::new).collect(Collectors.toUnmodifiableSet()); + .map(TServerInstance::fromZooKeeperPathString).collect(Collectors.toUnmodifiableSet()); } filter = tm -> !tm.getLogs().isEmpty() diff --git a/core/src/main/java/org/apache/accumulo/core/metrics/MetricsInfo.java b/core/src/main/java/org/apache/accumulo/core/metrics/MetricsInfo.java index ba79378a29e..933dc686b65 100644 --- a/core/src/main/java/org/apache/accumulo/core/metrics/MetricsInfo.java +++ b/core/src/main/java/org/apache/accumulo/core/metrics/MetricsInfo.java @@ -25,10 +25,9 @@ import java.util.Objects; import java.util.Set; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.data.ResourceGroupId; -import com.google.common.net.HostAndPort; - import io.micrometer.core.instrument.Tag; public interface MetricsInfo { @@ -79,15 +78,15 @@ static Tag resourceGroupTag(final ResourceGroupId resourceGroupName) { * Convenience method to create tag name / value pairs for the host and port from address * host:port pair. * - * @param hostAndPort the host:port pair + * @param host the host + * @param port the port */ - static List addressTags(final HostAndPort hostAndPort) { - Objects.requireNonNull(hostAndPort, "cannot create the tag without providing the hostAndPort"); + static List addressTags(final String host, final int port) { + Objects.requireNonNull(host, "cannot create the tag without providing the host"); List tags = new ArrayList<>(2); - tags.add(Tag.of(HOST_TAG_KEY, hostAndPort.getHost())); - int port = hostAndPort.getPort(); + tags.add(Tag.of(HOST_TAG_KEY, host)); if (port != 0) { - tags.add(Tag.of(PORT_TAG_KEY, Integer.toString(hostAndPort.getPort()))); + tags.add(Tag.of(PORT_TAG_KEY, Integer.toString(port))); } return Collections.unmodifiableList(tags); } @@ -98,12 +97,12 @@ static List addressTags(final HostAndPort hostAndPort) { * Common tags for all services. */ static Collection serviceTags(final String instanceName, final String applicationName, - final HostAndPort hostAndPort, final ResourceGroupId resourceGroupName) { + final ServerId server) { List tags = new ArrayList<>(); tags.add(instanceNameTag(instanceName)); tags.add(processTag(applicationName)); - tags.addAll(addressTags(hostAndPort)); - tags.add(resourceGroupTag(resourceGroupName)); + tags.addAll(addressTags(server.getHost(), server.getPort())); + tags.add(resourceGroupTag(server.getResourceGroup())); return tags; } diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/TTimeoutTransport.java b/core/src/main/java/org/apache/accumulo/core/rpc/TTimeoutTransport.java index 88f9adecd99..bfee3f18edc 100644 --- a/core/src/main/java/org/apache/accumulo/core/rpc/TTimeoutTransport.java +++ b/core/src/main/java/org/apache/accumulo/core/rpc/TTimeoutTransport.java @@ -28,6 +28,7 @@ import java.net.SocketAddress; import java.nio.channels.spi.SelectorProvider; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.hadoop.net.NetUtils; import org.apache.thrift.transport.TIOStreamTransport; import org.apache.thrift.transport.TTransport; @@ -35,8 +36,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.net.HostAndPort; - /** * A utility class for setting up a {@link TTransport} with various necessary configurations for * ideal performance in Accumulo. These configurations include: @@ -57,13 +56,13 @@ private TTimeoutTransport() {} * Creates a Thrift TTransport to the given address with the given timeout. All created resources * are closed if an exception is thrown. * - * @param addr The address to connect the client to + * @param server The server to connect the client to * @param timeoutMillis The timeout in milliseconds for the connection * @return A TTransport connected to the given addr * @throws TTransportException If the transport fails to be created/connected */ - public static TTransport create(HostAndPort addr, long timeoutMillis) throws TTransportException { - return INSTANCE.createInternal(new InetSocketAddress(addr.getHost(), addr.getPort()), + public static TTransport create(ServerId server, long timeoutMillis) throws TTransportException { + return INSTANCE.createInternal(new InetSocketAddress(server.getHost(), server.getPort()), timeoutMillis); } diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java b/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java index 4fd6d57e663..1ce1a6dfe2e 100644 --- a/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java @@ -27,6 +27,7 @@ import java.util.HashMap; import java.util.Map; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.rpc.SaslConnectionParams.SaslMechanism; @@ -45,8 +46,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.net.HostAndPort; - /** * Factory methods for creating Thrift client objects */ @@ -113,7 +112,7 @@ public static T createClient(ThriftClientTypes typ * @param context RPC options */ public static T getClientNoTimeout(ThriftClientTypes type, - HostAndPort address, ClientContext context) throws TTransportException { + ServerId address, ClientContext context) throws TTransportException { return getClient(type, address, context, 0); } @@ -125,8 +124,8 @@ public static T getClientNoTimeout(ThriftClientTypes< * @param address Server address for client to connect to * @param context RPC options */ - public static T getClient(ThriftClientTypes type, - HostAndPort address, ClientContext context) throws TTransportException { + public static T getClient(ThriftClientTypes type, ServerId address, + ClientContext context) throws TTransportException { TTransport transport = context.getTransportPool().getTransport(type, address, context.getClientTimeoutInMillis(), context, true); return createClient(type, transport, context.getInstanceID()); @@ -141,8 +140,8 @@ public static T getClient(ThriftClientTypes type, * @param context RPC options * @param timeout Socket timeout which overrides the ClientContext timeout */ - public static T getClient(ThriftClientTypes type, - HostAndPort address, ClientContext context, long timeout) throws TTransportException { + public static T getClient(ThriftClientTypes type, ServerId address, + ClientContext context, long timeout) throws TTransportException { TTransport transport = context.getTransportPool().getTransport(type, address, timeout, context, true); return createClient(type, transport, context.getInstanceID()); @@ -174,7 +173,7 @@ public static void returnClient(TServiceClient iface, ClientContext context) { * @param address Server address to open the transport to * @param context RPC options */ - public static TTransport createTransport(HostAndPort address, ClientContext context) + public static TTransport createTransport(ServerId address, ClientContext context) throws TException { return createClientTransport(address, (int) context.getClientTimeoutInMillis(), context.getClientSslParams(), context.getSaslParams()); @@ -209,7 +208,7 @@ public static synchronized TTransportFactory transportFactory(long maxFrameSize) * @param saslParams RPC options for SASL servers * @return An open TTransport which must be closed when finished */ - public static TTransport createClientTransport(HostAndPort address, int timeout, + public static TTransport createClientTransport(ServerId address, int timeout, SslConnectionParams sslParams, SaslConnectionParams saslParams) throws TTransportException { boolean success = false; TTransport transport = null; diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ClientServiceThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ClientServiceThriftClient.java index a25aee50ee3..b0747c63b4a 100644 --- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ClientServiceThriftClient.java +++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ClientServiceThriftClient.java @@ -22,6 +22,7 @@ import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.clientImpl.thrift.ClientService.Client; import org.apache.accumulo.core.lock.ServiceLockData.ThriftService; @@ -42,7 +43,7 @@ public class ClientServiceThriftClient extends ThriftClientTypes } @Override - public Pair getThriftServerConnection(ClientContext context, + public Pair getThriftServerConnection(ClientContext context, boolean preferCachedConnections, ResourceGroupPredicate rgp) throws TTransportException { return getThriftServerConnection(LOG, this, context, preferCachedConnections, warnedAboutTServersBeingDown, ThriftService.CLIENT, rgp); diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerClient.java index 5127807e0b2..d857016e4bd 100644 --- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerClient.java +++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerClient.java @@ -30,8 +30,6 @@ import org.apache.thrift.transport.TTransportException; import org.slf4j.Logger; -import com.google.common.net.HostAndPort; - public interface ManagerClient { default C getManagerConnection(Logger log, ThriftClientTypes type, ClientContext context) { @@ -44,25 +42,24 @@ default C getManagerConnection(Logger log, ThriftClientTypes type, ClientCont return null; } - final String managerLocation = managers.iterator().next().toHostPortString(); - if (managerLocation.equals("0.0.0.0:0")) { + final ServerId managerLocation = managers.iterator().next(); + if (managerLocation.toHostPortString().equals("0.0.0.0:0")) { // The Manager creates the lock with an initial address of 0.0.0.0:0, then // later updates the lock contents with the actual address after everything // is started. log.debug("Manager is up and lock acquired, waiting for address..."); return null; } - HostAndPort manager = HostAndPort.fromString(managerLocation); try { // Manager requests can take a long time: don't ever time out - return ThriftUtil.getClientNoTimeout(type, manager, context); + return ThriftUtil.getClientNoTimeout(type, managerLocation, context); } catch (TTransportException tte) { Throwable cause = tte.getCause(); if (cause != null && cause instanceof UnknownHostException) { // do not expect to recover from this throw new IllegalStateException(tte); } - log.debug("Failed to connect to manager=" + manager + ", will retry... ", tte); + log.debug("Failed to connect to manager=" + managerLocation + ", will retry... ", tte); return null; } } diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ServerProcessServiceThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ServerProcessServiceThriftClient.java index de21d8d8bc9..be0d5e785d6 100644 --- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ServerProcessServiceThriftClient.java +++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ServerProcessServiceThriftClient.java @@ -21,23 +21,21 @@ import java.io.UncheckedIOException; import java.net.UnknownHostException; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.process.thrift.ServerProcessService.Client; import org.apache.accumulo.core.rpc.ThriftUtil; import org.apache.thrift.transport.TTransportException; import org.slf4j.Logger; -import com.google.common.net.HostAndPort; - public class ServerProcessServiceThriftClient extends ThriftClientTypes { protected ServerProcessServiceThriftClient(String serviceName) { super(serviceName, new Client.Factory()); } - public Client getServerProcessConnection(ClientContext context, Logger log, String hostname, - int port) { - HostAndPort serverProcess = HostAndPort.fromParts(hostname, port); + public Client getServerProcessConnection(ClientContext context, Logger log, + ServerId serverProcess) { try { // Manager requests can take a long time: don't ever time out return ThriftUtil.getClientNoTimeout(this, serverProcess, context); diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TServerClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TServerClient.java index b5104220845..cc2594a5f12 100644 --- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TServerClient.java +++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TServerClient.java @@ -35,6 +35,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.NamespaceNotFoundException; import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.AccumuloServerException; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; @@ -65,10 +66,10 @@ public interface TServerClient { static final String DEBUG_HOST = "org.apache.accumulo.client.rpc.debug.host"; static final String DEBUG_RG = "org.apache.accumulo.client.rpc.debug.group"; - Pair getThriftServerConnection(ClientContext context, boolean preferCachedConnections, + Pair getThriftServerConnection(ClientContext context, boolean preferCachedConnections, ResourceGroupPredicate rgp) throws TTransportException; - default Pair getThriftServerConnection(Logger LOG, ThriftClientTypes type, + default Pair getThriftServerConnection(Logger LOG, ThriftClientTypes type, ClientContext context, boolean preferCachedConnections, AtomicBoolean warned, ThriftService service, ResourceGroupPredicate rgp) throws TTransportException { checkArgument(context != null, "context is null"); @@ -106,13 +107,13 @@ default Pair getThriftServerConnection(Logger LOG, ThriftClientTypes cachedTransport = + Pair cachedTransport = context.getTransportPool().getAnyCachedTransport(type, context, service, rgp); if (cachedTransport != null) { C client = ThriftUtil.createClient(type, cachedTransport.getSecond(), context.getInstanceID()); warned.set(false); - return new Pair(cachedTransport.getFirst(), client); + return new Pair(cachedTransport.getFirst(), client); } } @@ -156,7 +157,7 @@ default Pair getThriftServerConnection(Logger LOG, ThriftClientTypes data = zc.getLockData(path); if (data != null && data.isPresent()) { - HostAndPort tserverClientAddress = data.orElseThrow().getAddress(service); + ServerId tserverClientAddress = data.orElseThrow().getServer(service); if (tserverClientAddress != null) { try { TTransport transport = context.getTransportPool().getTransport(type, @@ -166,7 +167,7 @@ default Pair getThriftServerConnection(Logger LOG, ThriftClientTypes(tserverClientAddress.toString(), client); + return new Pair(tserverClientAddress, client); } catch (TTransportException e) { if (type == ThriftClientTypes.CLIENT && debugHostSpecified) { LOG.error( @@ -209,10 +210,10 @@ default Pair getThriftServerConnection(Logger LOG, ThriftClientTypes R execute(Logger LOG, ClientContext context, Exec exec, ResourceGroupPredicate rgp) throws AccumuloException, AccumuloSecurityException { while (true) { - String server = null; + ServerId server = null; C client = null; try { - Pair pair = getThriftServerConnection(context, true, rgp); + Pair pair = getThriftServerConnection(context, true, rgp); server = pair.getFirst(); client = pair.getSecond(); return exec.execute(client); @@ -249,10 +250,10 @@ default R execute(Logger LOG, ClientContext context, Exec exec, default void executeVoid(Logger LOG, ClientContext context, ExecVoid exec, ResourceGroupPredicate rgp) throws AccumuloException, AccumuloSecurityException { while (true) { - String server = null; + ServerId server = null; C client = null; try { - Pair pair = getThriftServerConnection(context, true, rgp); + Pair pair = getThriftServerConnection(context, true, rgp); server = pair.getFirst(); client = pair.getSecond(); exec.execute(client); diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletManagementClientServiceThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletManagementClientServiceThriftClient.java index 063002f0d72..dcc6c817431 100644 --- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletManagementClientServiceThriftClient.java +++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletManagementClientServiceThriftClient.java @@ -22,6 +22,7 @@ import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.lock.ServiceLockData.ThriftService; import org.apache.accumulo.core.lock.ServiceLockPaths.ResourceGroupPredicate; @@ -47,7 +48,7 @@ public TabletManagementClientServiceThriftClient(String serviceName) { } @Override - public Pair getThriftServerConnection(ClientContext context, + public Pair getThriftServerConnection(ClientContext context, boolean preferCachedConnections, ResourceGroupPredicate rgp) throws TTransportException { return getThriftServerConnection(LOG, this, context, preferCachedConnections, warnedAboutTServersBeingDown, ThriftService.TABLET_MANAGEMENT, rgp); diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletServerThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletServerThriftClient.java index 5cd489651c1..23405f4f271 100644 --- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletServerThriftClient.java +++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletServerThriftClient.java @@ -22,6 +22,7 @@ import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.lock.ServiceLockData.ThriftService; import org.apache.accumulo.core.lock.ServiceLockPaths.ResourceGroupPredicate; @@ -46,7 +47,7 @@ public class TabletServerThriftClient extends ThriftClientTypes } @Override - public Pair getThriftServerConnection(ClientContext context, + public Pair getThriftServerConnection(ClientContext context, boolean preferCachedConnections, ResourceGroupPredicate rgp) throws TTransportException { return getThriftServerConnection(LOG, this, context, preferCachedConnections, warnedAboutTServersBeingDown, ThriftService.TSERV, rgp); diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/GroupBalancer.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/GroupBalancer.java index 86f6c71759f..197816d466c 100644 --- a/core/src/main/java/org/apache/accumulo/core/spi/balancer/GroupBalancer.java +++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/GroupBalancer.java @@ -133,12 +133,11 @@ public void getAssignments(AssignmentParameters params) { if (last != null) { // Maintain locality String fakeSessionID = " "; - TabletServerId simple = - new TabletServerIdImpl(last.getHost(), last.getPort(), fakeSessionID); + TabletServerId simple = new TabletServerIdImpl(entry.getValue().getServer(), fakeSessionID); Iterator find = params.currentStatus().tailMap(simple).keySet().iterator(); if (find.hasNext()) { TabletServerId tserver = find.next(); - if (tserver.getHost().equals(last.getHost())) { + if (tserver.getServer().getHost().equals(last.getServer().getHost())) { params.addAssignment(entry.getKey(), tserver); continue; } diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java index 3afd93886f1..84a780d3dde 100644 --- a/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java +++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java @@ -245,7 +245,7 @@ static class HrtlbConf { * @return pool names, will return default pool if host matches more no regex */ protected List getPoolNamesForHost(TabletServerId tabletServerId) { - final String host = tabletServerId.getHost(); + final String host = tabletServerId.getServer().getHost(); String test = host; if (!hrtlbConf.get().isIpBasedRegex) { try { @@ -459,7 +459,7 @@ public long balance(BalanceParameters params) { } } catch (AccumuloException | AccumuloSecurityException e1) { LOG.error("Error in OOB check getting tablets for table {} from server {} {}", tid, - e.getKey().getHost(), e); + e.getKey().getServer().getHost(), e); } } } diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java index 068e12bfcfe..db58547ab2d 100644 --- a/core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java +++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java @@ -95,11 +95,11 @@ public TabletServerId getAssignment(SortedMap loca if (last != null) { // Maintain locality String fakeSessionID = " "; - TabletServerId simple = new TabletServerIdImpl(last.getHost(), last.getPort(), fakeSessionID); + TabletServerId simple = new TabletServerIdImpl(last.getServer(), fakeSessionID); Iterator find = locations.tailMap(simple).keySet().iterator(); if (find.hasNext()) { TabletServerId current = find.next(); - if (current.getHost().equals(last.getHost())) { + if (current.getServer().getHost().equals(last.getServer().getHost())) { return current; } } diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletServerId.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletServerId.java index 6adcc803f17..6d9003e2296 100644 --- a/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletServerId.java +++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletServerId.java @@ -18,13 +18,14 @@ */ package org.apache.accumulo.core.spi.balancer.data; +import org.apache.accumulo.core.client.admin.servers.ServerId; + /** * @since 2.1.0 */ public interface TabletServerId extends Comparable { - String getHost(); - int getPort(); + ServerId getServer(); String getSession(); } diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerHostSelector.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerHostSelector.java index f43f21e8c26..d952913498c 100644 --- a/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerHostSelector.java +++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerHostSelector.java @@ -29,10 +29,10 @@ import java.util.Map; import java.util.Set; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.data.TabletId; import com.google.common.hash.HashCode; -import com.google.common.net.HostAndPort; /** * Extension of the {@code ConfigurableScanServerSelector} that can be used when there are multiple @@ -58,7 +58,7 @@ public int compare(PriorHostServers o1, PriorHostServers o2) { private static final class PriorHostServers { private final String priorHost; - private final List priorServers = new ArrayList<>(); + private final List priorServers = new ArrayList<>(); public PriorHostServers(String priorHost) { this.priorHost = priorHost; @@ -68,14 +68,14 @@ public String getPriorHost() { return priorHost; } - public List getPriorServers() { + public List getPriorServers() { return priorServers; } } @Override protected int selectServers(SelectorParameters params, Profile profile, - List orderedScanServers, Map serversToUse) { + List orderedScanServers, Map serversToUse) { // orderedScanServers is the set of ScanServers addresses (host:port) // for the resource group designated for the profile being used for @@ -83,11 +83,10 @@ protected int selectServers(SelectorParameters params, Profile profile, // hash the tablet to the hostname, then randomly pick one of the // scan servers in that group. - final Map> scanServerHosts = new HashMap<>(); - for (final String address : orderedScanServers) { - final HostAndPort hp = HostAndPort.fromString(address); - scanServerHosts.computeIfAbsent(hp.getHost(), (k) -> { - return new ArrayList(); + final Map> scanServerHosts = new HashMap<>(); + for (final ServerId address : orderedScanServers) { + scanServerHosts.computeIfAbsent(address.getHost(), (k) -> { + return new ArrayList(); }).add(address); } final List hostIndex = new ArrayList<>(scanServerHosts.keySet()); @@ -107,10 +106,9 @@ protected int selectServers(SelectorParameters params, Profile profile, // scan servers remaining on that host, or has tried them all. final Map priorServers = new HashMap<>(numberOfPreviousAttempts); params.getAttempts(tablet).forEach(ssa -> { - final String priorServerAddress = ssa.getServer(); - final HostAndPort priorHP = HostAndPort.fromString(priorServerAddress); - priorServers.computeIfAbsent(priorHP.getHost(), (k) -> { - return new PriorHostServers(priorHP.getHost()); + final ServerId priorServerAddress = ssa.getServer(); + priorServers.computeIfAbsent(priorServerAddress.getHost(), (k) -> { + return new PriorHostServers(priorServerAddress.getHost()); }).getPriorServers().add(priorServerAddress); }); final List priors = new ArrayList<>(priorServers.values()); @@ -118,7 +116,7 @@ protected int selectServers(SelectorParameters params, Profile profile, Collections.sort(priors, new PriorHostServersComparator()); for (PriorHostServers phs : priors) { - final Set scanServersOnPriorHost = + final Set scanServersOnPriorHost = new HashSet<>(scanServerHosts.get(phs.getPriorHost())); scanServersOnPriorHost.removeAll(phs.getPriorServers()); if (scanServersOnPriorHost.size() > 0) { @@ -144,7 +142,7 @@ protected int selectServers(SelectorParameters params, Profile profile, (Math.abs(hashCode.asInt()) + RANDOM.get().nextInt(numServersToUseInAttemptPlan)) % hostIndex.size(); final String hostToUse = hostIndex.get(serverIndex); - final List scanServersOnHost = scanServerHosts.get(hostToUse); + final List scanServersOnHost = scanServerHosts.get(hostToUse); serversToUse.put(tablet, scanServersOnHost.get(0)); } } diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelector.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelector.java index eecf68b905a..ee7848f2095 100644 --- a/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelector.java +++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelector.java @@ -35,6 +35,7 @@ import java.util.concurrent.TimeUnit; import java.util.function.Supplier; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.conf.ConfigurationTypeHelper; import org.apache.accumulo.core.data.ResourceGroupId; import org.apache.accumulo.core.data.TabletId; @@ -180,7 +181,7 @@ public class ConfigurableScanServerSelector implements ScanServerSelector { + "{'servers':'13', 'busyTimeout':'33ms', 'salt':'two'}," + "{'servers':'100%', 'busyTimeout':'33ms'}]}]"; - private Supplier>> orderedScanServersSupplier; + private Supplier>> orderedScanServersSupplier; private Map profiles; private Profile defaultProfile; @@ -332,9 +333,10 @@ public void init(ScanServerSelector.InitParameters params) { // avoid constantly resorting the scan servers, just do it periodically in case they change orderedScanServersSupplier = Suppliers.memoizeWithExpiration(() -> { Collection scanServers = params.getScanServers().get(); - Map> groupedServers = new HashMap<>(); + Map> groupedServers = new HashMap<>(); scanServers.forEach(sserver -> groupedServers - .computeIfAbsent(sserver.getGroup(), k -> new ArrayList<>()).add(sserver.getAddress())); + .computeIfAbsent(sserver.getServer().getResourceGroup(), k -> new ArrayList<>()) + .add(sserver.getServer())); groupedServers.values().forEach(ssAddrs -> Collections.sort(ssAddrs)); return groupedServers; }, 3, TimeUnit.SECONDS); @@ -363,7 +365,7 @@ public ScanServerSelections selectServers(ScanServerSelector.SelectorParameters // only get this once and use it for the entire method so that the method uses a consistent // snapshot - List orderedScanServers = + List orderedScanServers = orderedScanServersSupplier.get().getOrDefault(ResourceGroupId.of(profile.group), List.of()); Duration scanServerWaitTime = profile.getTimeToWaitForScanServers(); @@ -383,7 +385,7 @@ public ScanServerSelections selectServers(ScanServerSelector.SelectorParameters // there are no scan servers so fall back to the tablet server return new ScanServerSelections() { @Override - public String getScanServer(TabletId tabletId) { + public ServerId getScanServer(TabletId tabletId) { return null; } @@ -399,7 +401,7 @@ public Duration getBusyTimeout() { }; } - Map serversToUse = new HashMap<>(); + Map serversToUse = new HashMap<>(); int maxAttempts = selectServers(params, profile, orderedScanServers, serversToUse); @@ -407,7 +409,7 @@ public Duration getBusyTimeout() { return new ScanServerSelections() { @Override - public String getScanServer(TabletId tabletId) { + public ServerId getScanServer(TabletId tabletId) { return serversToUse.get(tabletId); } @@ -424,7 +426,7 @@ public Duration getBusyTimeout() { } protected int selectServers(ScanServerSelector.SelectorParameters params, Profile profile, - List orderedScanServers, Map serversToUse) { + List orderedScanServers, Map serversToUse) { int attempts = params.getTablets().stream() .mapToInt(tablet -> params.getAttempts(tablet).size()).max().orElse(0); @@ -432,7 +434,7 @@ protected int selectServers(ScanServerSelector.SelectorParameters params, Profil int numServers = profile.getNumServers(attempts, orderedScanServers.size()); for (TabletId tablet : params.getTablets()) { - String serverToUse = null; + ServerId serverToUse = null; var hashCode = hashTablet(tablet, profile.getSalt(attempts)); diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerAttempt.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerAttempt.java index b886e3b26d9..b71edea7aed 100644 --- a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerAttempt.java +++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerAttempt.java @@ -18,6 +18,8 @@ */ package org.apache.accumulo.core.spi.scan; +import org.apache.accumulo.core.client.admin.servers.ServerId; + /** * This object is used to communicate what previous actions were attempted, when they were * attempted, and the result of those attempts @@ -31,7 +33,7 @@ enum Result { BUSY, ERROR } - String getServer(); + ServerId getServer(); ScanServerAttempt.Result getResult(); diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerInfo.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerInfo.java index 2b6d1239d3b..38076db5c94 100644 --- a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerInfo.java +++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerInfo.java @@ -18,7 +18,7 @@ */ package org.apache.accumulo.core.spi.scan; -import org.apache.accumulo.core.data.ResourceGroupId; +import org.apache.accumulo.core.client.admin.servers.ServerId; /** * Information about a scan server. @@ -28,14 +28,8 @@ public interface ScanServerInfo { /** - * @return the address in the form of {@code :} where the scan server is running. + * @return the location where the scan server is running. */ - String getAddress(); - - /** - * @return the group set when the scan server was started. If a group name was not set for the - * scan server, then {@code ResourceGroupId#DEFAULT} is returned. - */ - ResourceGroupId getGroup(); + ServerId getServer(); } diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelections.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelections.java index 0236e21c82a..0766be10bf7 100644 --- a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelections.java +++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelections.java @@ -20,6 +20,7 @@ import java.time.Duration; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.data.TabletId; /** @@ -34,7 +35,7 @@ public interface ScanServerSelections { * @return what scan server to use for a given tablet. Returning null indicates the tablet server * should be used for this tablet. */ - String getScanServer(TabletId tabletId); + ServerId getScanServer(TabletId tabletId); /** * @return The amount of time to wait on the client side before starting to contact servers. diff --git a/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java b/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java index ae60b00dc2a..4e3a4730861 100644 --- a/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java +++ b/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java @@ -88,7 +88,6 @@ import com.github.benmanes.caffeine.cache.Cache; import com.google.common.base.Preconditions; import com.google.common.hash.Hashing; -import com.google.common.net.HostAndPort; /** * This class implements using multiple tservers to gather summaries. @@ -165,7 +164,7 @@ private TSummaryRequest getRequest() { * @return A map of the form : {@code map>} . The ranges * associated with a file represent the tablets that use the file. */ - private Map>> + private Map>> getFilesGroupedByLocation(Predicate fileSelector) { // get a subset of files @@ -185,23 +184,25 @@ private TSummaryRequest getRequest() { // group by location, then file - Map>> locations = new HashMap<>(); + Map>> locations = new HashMap<>(); List tservers = null; for (Entry> entry : files.entrySet()) { - String location = entry.getValue().stream().filter(tm -> tm.getLocation() != null) // filter - // tablets - // w/o a - // location - .map(tm -> tm.getLocation().getHostPort()) // convert to host:port strings - .min(String::compareTo) // find minimum host:port + ServerId location = entry.getValue().stream().filter(tm -> tm.getLocation() != null) // filter + // tablets + // w/o a + // location + .map(tm -> tm.getLocation().getServerInstance().getServer()) // convert to host:port + // strings + .min(ServerId::compareTo) // find minimum host:port .orElse(entry.getValue().stream().filter(tm -> tm.getLast() != null) // if no locations, // then look at last // locations - .map(tm -> tm.getLast().getHostPort()) // convert to host:port strings - .min(String::compareTo).orElse(null)); // find minimum last location or return null + .map(tm -> tm.getLast().getServerInstance().getServer()) // convert to host:port + // strings + .min(ServerId::compareTo).orElse(null)); // find minimum last location or return null if (location == null) { if (tservers == null) { @@ -214,7 +215,7 @@ private TSummaryRequest getRequest() { // same file (as long as the set of tservers is stable). int idx = Math.abs(Hashing.murmur3_32_fixed() .hashString(entry.getKey().getNormalizedPathStr(), UTF_8).asInt()) % tservers.size(); - location = tservers.get(idx).toHostPortString(); + location = tservers.get(idx); } // merge contiguous ranges @@ -287,12 +288,12 @@ static ProcessedFiles merge(ProcessedFiles pf1, ProcessedFiles pf2, SummarizerFa private class FilesProcessor implements Supplier { - final HostAndPort location; + final ServerId location; final Map> allFiles; private final TInfo tinfo; private final AtomicBoolean cancelFlag; - public FilesProcessor(TInfo tinfo, HostAndPort location, + public FilesProcessor(TInfo tinfo, ServerId location, Map> allFiles, AtomicBoolean cancelFlag) { this.location = location; this.allFiles = allFiles; @@ -358,7 +359,7 @@ private class PartitionFuture implements Future { if (previousWork != null) { fileSelector = fileSelector.and(previousWork.failedFiles::contains); } - Map>> filesGBL; + Map>> filesGBL; filesGBL = getFilesGroupedByLocation(fileSelector); List> futures = new ArrayList<>(); @@ -367,12 +368,11 @@ private class PartitionFuture implements Future { .completedFuture(new ProcessedFiles(previousWork.summaries, factory))); } - for (Entry>> entry : filesGBL.entrySet()) { - HostAndPort location = HostAndPort.fromString(entry.getKey()); + for (Entry>> entry : filesGBL.entrySet()) { Map> allFiles = entry.getValue(); - futures.add(CompletableFuture - .supplyAsync(new FilesProcessor(tinfo, location, allFiles, cancelFlag), execSrv)); + futures.add(CompletableFuture.supplyAsync( + new FilesProcessor(tinfo, entry.getKey(), allFiles, cancelFlag), execSrv)); } return CompletableFutureUtil.merge(futures, diff --git a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java index 57820065b07..4bf2c2ccc74 100644 --- a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java +++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java @@ -22,6 +22,8 @@ import java.util.Objects; import java.util.UUID; +import org.apache.accumulo.core.client.admin.servers.ServerId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Value; @@ -31,16 +33,15 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.net.HostAndPort; public final class LogEntry { private final String path; - private final HostAndPort tserver; + private ServerId tserver; private final UUID uniqueId; private final Text columnQualifier; - private LogEntry(String path, HostAndPort tserver, UUID uniqueId, Text columnQualifier) { + private LogEntry(String path, ServerId tserver, UUID uniqueId, Text columnQualifier) { this.path = path; this.tserver = tserver; this.uniqueId = uniqueId; @@ -51,8 +52,8 @@ private LogEntry(String path, HostAndPort tserver, UUID uniqueId, Text columnQua * Creates a new LogEntry object after validating the expected format of the path. We expect the * path to contain a tserver (host+port) followed by a UUID as the file name as the last two * components.
- * For example, file:///some/dir/path/localhost+1234/927ba659-d109-4bce-b0a5-bcbbcb9942a2 is a - * valid path. + * For example, file:///some/dir/path/group+localhost+1234/927ba659-d109-4bce-b0a5-bcbbcb9942a2 is + * a valid path. * * @param path path to validate * @return an object representation of this log entry @@ -74,13 +75,13 @@ private static LogEntry validatedLogEntry(String path, Text columnQualifier) { String uuidPart = parts[parts.length - 1]; String badTServerMsg = - "Invalid tserver in path. Expected: host+port. Found '" + tserverPart + "'"; + "Invalid tserver in path. Expected: group+host+port. Found '" + tserverPart + "'"; if (tserverPart.contains(":") || !tserverPart.contains("+")) { throw new IllegalArgumentException(badTServerMsg); } - HostAndPort tserver; + ServerId tserver; try { - tserver = HostAndPort.fromString(tserverPart.replace("+", ":")); + tserver = ServerIdUtil.fromWalFileName(tserverPart); } catch (IllegalArgumentException e) { throw new IllegalArgumentException(badTServerMsg); } @@ -120,7 +121,7 @@ public static LogEntry fromMetaWalEntry(Entry entry) { @NonNull @VisibleForTesting - HostAndPort getTServer() { + ServerId getTServer() { return tserver; } diff --git a/core/src/main/java/org/apache/accumulo/core/util/AddressUtil.java b/core/src/main/java/org/apache/accumulo/core/util/AddressUtil.java index b6bb6f95346..4144798904c 100644 --- a/core/src/main/java/org/apache/accumulo/core/util/AddressUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/util/AddressUtil.java @@ -25,32 +25,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.net.HostAndPort; - public class AddressUtil { private static final Logger log = LoggerFactory.getLogger(AddressUtil.class); - public static HostAndPort parseAddress(final String address) throws NumberFormatException { - String normalized = normalizePortSeparator(address); - HostAndPort hap = HostAndPort.fromString(normalized); - if (!hap.hasPort()) { - throw new IllegalArgumentException( - "Address was expected to contain port. address=" + address); - } - - return hap; - } - - public static HostAndPort parseAddress(final String address, final int defaultPort) { - String normalized = normalizePortSeparator(address); - return HostAndPort.fromString(normalized).withDefaultPort(defaultPort); - } - - private static String normalizePortSeparator(final String address) { - return address.replace('+', ':'); - } - /** * Fetch the security value that determines how long DNS failures are cached. Looks up the * security property 'networkaddress.cache.negative.ttl'. Should that fail returns the default diff --git a/core/src/main/java/org/apache/accumulo/core/util/cache/Caches.java b/core/src/main/java/org/apache/accumulo/core/util/cache/Caches.java index f58a0a9427f..b7cb1c950a0 100644 --- a/core/src/main/java/org/apache/accumulo/core/util/cache/Caches.java +++ b/core/src/main/java/org/apache/accumulo/core/util/cache/Caches.java @@ -55,6 +55,7 @@ public enum CacheName { RECOVERY_MANAGER_PATH_CACHE, SCAN_SERVER_TABLET_METADATA, SERVICE_ENVIRONMENT_TABLE_CONFIGS, + SERVER_ID, SPACE_AWARE_VOLUME_CHOICE, SPLITTER_FILES, SPLITTER_STARTING, diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java index 7b996d030eb..bf6abf32bb7 100644 --- a/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java @@ -34,7 +34,9 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ClientContext; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; import org.apache.accumulo.core.compaction.thrift.CompactorService; import org.apache.accumulo.core.data.ResourceGroupId; @@ -103,25 +105,26 @@ public static String getHostPortString(HostAndPort address) { * * @return Optional HostAndPort of Coordinator node if found */ - public static Optional findCompactionCoordinator(ClientContext context) { + public static Optional findCompactionCoordinator(ClientContext context) { final ServiceLockPath slp = context.getServerPaths().createManagerPath(); if (slp == null) { return Optional.empty(); } return ServiceLock.getLockData(context.getZooCache(), slp, new ZcStat()) - .map(sld -> sld.getAddress(ThriftService.COORDINATOR)); + .map(sld -> sld.getServer(ThriftService.COORDINATOR)); } /** * @return map of group names to compactor addresses */ - public static Map> getCompactorAddrs(ClientContext context) { - final Map> groupsAndAddresses = new HashMap<>(); + public static Map> getCompactorAddrs(ClientContext context) { + final Map> groupsAndAddresses = new HashMap<>(); context.getServerPaths().getCompactor(ResourceGroupPredicate.ANY, AddressSelector.all(), true) .forEach(slp -> { + var hp = HostAndPort.fromString(slp.getServer()); groupsAndAddresses .computeIfAbsent(slp.getResourceGroup().canonical(), (k) -> new HashSet<>()) - .add(HostAndPort.fromString(slp.getServer())); + .add(ServerIdUtil.compactor(slp.getResourceGroup(), hp.getHost(), hp.getPort())); }); return groupsAndAddresses; } @@ -133,7 +136,7 @@ public static Map> getCompactorAddrs(ClientContext conte * @return list of active compaction * @throws ThriftSecurityException tserver permission error */ - public static List getActiveCompaction(HostAndPort compactor, + public static List getActiveCompaction(ServerId compactor, ClientContext context) throws ThriftSecurityException { CompactorService.Client client = null; try { @@ -156,7 +159,7 @@ public static List getActiveCompaction(HostAndPort compactor, * @param context context * @return external compaction job or null if none running */ - public static TExternalCompactionJob getRunningCompaction(HostAndPort compactorAddr, + public static TExternalCompactionJob getRunningCompaction(ServerId compactorAddr, ClientContext context) { CompactorService.Client client = null; @@ -176,7 +179,7 @@ public static TExternalCompactionJob getRunningCompaction(HostAndPort compactorA return null; } - private static ExternalCompactionId getRunningCompactionId(HostAndPort compactorAddr, + private static ExternalCompactionId getRunningCompactionId(ServerId compactorAddr, ClientContext context) { CompactorService.Client client = null; try { @@ -208,9 +211,10 @@ public static List getCompactionsRunningOnCompactors(ClientCo context.getServerPaths().getCompactor(ResourceGroupPredicate.ANY, AddressSelector.all(), true) .forEach(slp -> { - final HostAndPort hp = HostAndPort.fromString(slp.getServer()); + var hp = HostAndPort.fromString(slp.getServer()); + var sid = ServerIdUtil.compactor(slp.getResourceGroup(), hp.getHost(), hp.getPort()); rcFutures.add(new RunningCompactionFuture(slp, - executor.submit(() -> getRunningCompaction(hp, context)))); + executor.submit(() -> getRunningCompaction(sid, context)))); }); executor.shutdown(); @@ -219,8 +223,8 @@ public static List getCompactionsRunningOnCompactors(ClientCo try { TExternalCompactionJob job = rcf.getFuture().get(); if (null != job && null != job.getExternalCompactionId()) { - var compactorAddress = getHostPortString(rcf.getCompactor()); - results.add(new RunningCompaction(job, compactorAddress, rcf.getGroup())); + results.add(new RunningCompaction(job, ServerIdUtil.compactor(rcf.getGroup(), + rcf.getCompactor().getHost(), rcf.getCompactor().getPort()))); } } catch (InterruptedException | ExecutionException e) { throw new IllegalStateException(e); @@ -237,8 +241,9 @@ public static List getCompactionsRunningOnCompactors(ClientCo context.getServerPaths().getCompactor(ResourceGroupPredicate.ANY, AddressSelector.all(), true) .forEach(slp -> { - final HostAndPort hp = HostAndPort.fromString(slp.getServer()); - futures.add(executor.submit(() -> getRunningCompactionId(hp, context))); + var hp = HostAndPort.fromString(slp.getServer()); + var sid = ServerIdUtil.compactor(slp.getResourceGroup(), hp.getHost(), hp.getPort()); + futures.add(executor.submit(() -> getRunningCompactionId(sid, context))); }); executor.shutdown(); @@ -271,8 +276,7 @@ public static int countCompactors(ResourceGroupId group, ClientContext context) return count; } - public static void cancelCompaction(ClientContext context, HostAndPort compactorAddr, - String ecid) { + public static void cancelCompaction(ClientContext context, ServerId compactorAddr, String ecid) { CompactorService.Client client = null; try { client = ThriftUtil.getClient(ThriftClientTypes.COMPACTOR, compactorAddr, context); diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/RunningCompaction.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/RunningCompaction.java index 809422a5a26..51ac1b57aae 100644 --- a/core/src/main/java/org/apache/accumulo/core/util/compaction/RunningCompaction.java +++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/RunningCompaction.java @@ -22,6 +22,8 @@ import java.util.Objects; import java.util.TreeMap; +import org.apache.accumulo.core.client.admin.servers.ServerId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.compaction.thrift.TCompactionState; import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate; import org.apache.accumulo.core.compaction.thrift.TExternalCompaction; @@ -30,29 +32,30 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.net.HostAndPort; + public class RunningCompaction { private final static Logger LOG = LoggerFactory.getLogger(RunningCompaction.class); private final TExternalCompactionJob job; - private final String compactorAddress; - private final ResourceGroupId groupName; + private final ServerId compactor; private final Map updates = new TreeMap<>(); // If this object were to be added to a time sorted list before the start time // is set, then it will end up at the end of the list. private Long startTime = Long.MAX_VALUE; - public RunningCompaction(TExternalCompactionJob job, String compactorAddress, - ResourceGroupId groupName) { + public RunningCompaction(TExternalCompactionJob job, ServerId compactor) { this.job = Objects.requireNonNull(job, "job cannot be null"); - this.compactorAddress = - Objects.requireNonNull(compactorAddress, "compactor address cannot be null"); - this.groupName = Objects.requireNonNull(groupName, "groupName cannot be null"); + this.compactor = Objects.requireNonNull(compactor, "compactor address cannot be null"); } public RunningCompaction(TExternalCompaction tEC) { - this(tEC.getJob(), tEC.getCompactor(), ResourceGroupId.of(tEC.getGroupName())); + this(tEC.getJob(), + ServerIdUtil.compactor(ResourceGroupId.of(tEC.getGroupName()), + HostAndPort.fromString(tEC.getCompactor()).getHost(), + HostAndPort.fromString(tEC.getCompactor()).getPort())); } public Map getUpdates() { @@ -74,12 +77,8 @@ public TExternalCompactionJob getJob() { return job; } - public String getCompactorAddress() { - return compactorAddress; - } - - public ResourceGroupId getGroup() { - return groupName; + public ServerId getCompactor() { + return compactor; } public boolean isStartTimeSet() { diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientTabletCacheImplTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientTabletCacheImplTest.java index 9c35fd312bd..4f39e8d9938 100644 --- a/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientTabletCacheImplTest.java +++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientTabletCacheImplTest.java @@ -47,11 +47,13 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiConsumer; +import java.util.stream.Collectors; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.admin.TableOperations; import org.apache.accumulo.core.client.admin.TabletAvailability; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ClientTabletCache.CachedTablet; import org.apache.accumulo.core.clientImpl.ClientTabletCache.CachedTablets; import org.apache.accumulo.core.clientImpl.ClientTabletCache.LocationNeed; @@ -69,6 +71,7 @@ import org.apache.accumulo.core.metadata.MetadataCachedTabletObtainer; import org.apache.accumulo.core.metadata.RootTable; import org.apache.accumulo.core.metadata.SystemTables; +import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily; import org.apache.accumulo.core.util.Pair; @@ -83,6 +86,14 @@ public class ClientTabletCacheImplTest { private static final KeyExtent METADATA_TABLE_EXTENT = new KeyExtent(SystemTables.METADATA.tableId(), null, ROOT_TABLE_EXTENT.endRow()); + static ServerId csi(String host, int port) { + return ServerIdUtil.tserver(host, port); + } + + static ServerId csi(String host) { + return ServerIdUtil.tserver(host, 0); + } + static KeyExtent createNewKeyExtent(String table, String endRow, String prevEndRow) { return new KeyExtent(TableId.of(table), endRow == null ? null : new Text(endRow), prevEndRow == null ? null : new Text(prevEndRow)); @@ -128,15 +139,15 @@ static RangeLocation createRangeLocation(String location, KeyExtent extent1, Lis return new RangeLocation(location, extent1, range1, extent2, range2); } - static Map>> + static Map>> createExpectedBinnings(RangeLocation... rangeLocations) { - Map>> expBinnedRanges = new HashMap<>(); + Map>> expBinnedRanges = new HashMap<>(); for (RangeLocation rl : rangeLocations) { Map> binnedKE = - expBinnedRanges.computeIfAbsent(rl.location, k -> new HashMap<>()); - expBinnedRanges.put(rl.location, binnedKE); + expBinnedRanges.computeIfAbsent(csi(rl.location), k -> new HashMap<>()); + expBinnedRanges.put(csi(rl.location), binnedKE); binnedKE.putAll(rl.extents); } return expBinnedRanges; @@ -148,7 +159,7 @@ static TreeMap createMetaCacheKE(Object... data) { for (int i = 0; i < data.length; i += 2) { KeyExtent ke = (KeyExtent) data[i]; String loc = (String) data[i + 1]; - mcke.put(ke, new CachedTablet(ke, loc, "1", TabletAvailability.ONDEMAND, false)); + mcke.put(ke, new CachedTablet(ke, csi(loc), "1", TabletAvailability.ONDEMAND, false)); } return mcke; @@ -189,7 +200,7 @@ static ClientTabletCacheImpl createLocators(TServers tservers, String rootTabLoc for (Entry entry : mcke.entrySet()) { setLocation(tservers, metaTabLoc, METADATA_TABLE_EXTENT, entry.getKey(), - entry.getValue().getTserverLocation().orElseThrow()); + entry.getValue().getTserverLocation().orElseThrow().getHost()); } return tab1TabletCache; @@ -235,15 +246,15 @@ public void setUp() throws AccumuloException, TableNotFoundException { } private void runTest(List ranges, ClientTabletCacheImpl tab1TabletCache, - Map>> expected) throws Exception { + Map>> expected) throws Exception { List failures = Collections.emptyList(); runTest(ranges, tab1TabletCache, expected, failures); } private void runTest(List ranges, ClientTabletCacheImpl tab1TabletCache, - Map>> expected, List efailures) throws Exception { + Map>> expected, List efailures) throws Exception { - Map>> binnedRanges = new HashMap<>(); + Map>> binnedRanges = new HashMap<>(); List f = tab1TabletCache.binRanges(context, ranges, binnedRanges); assertEquals(expected, binnedRanges); @@ -295,8 +306,8 @@ static List createNewMutationList(Mutation... ma) { } private void runTest(ClientTabletCacheImpl metaCache, List ml, - Map>> emb, String... efailures) throws Exception { - Map> binnedMutations = new HashMap<>(); + Map>> emb, String... efailures) throws Exception { + Map> binnedMutations = new HashMap<>(); List afailures = new ArrayList<>(); metaCache.binMutations(context, ml, binnedMutations, afailures); @@ -316,11 +327,11 @@ private void runTest(ClientTabletCacheImpl metaCache, List ml, } - private void verify(Map>> expected, - Map> actual) { + private void verify(Map>> expected, + Map> actual) { assertEquals(expected.keySet(), actual.keySet()); - for (String server : actual.keySet()) { + for (ServerId server : actual.keySet()) { TabletServerMutations atb = actual.get(server); Map> etb = expected.get(server); @@ -359,12 +370,13 @@ static ServerExtent createServerExtent(String row, String location, KeyExtent ex return new ServerExtent(location, row, extent); } - static Map>> createServerExtentMap(ServerExtent... locations) { + static Map>> + createServerExtentMap(ServerExtent... locations) { - Map>> serverExtents = new HashMap<>(); + Map>> serverExtents = new HashMap<>(); for (ServerExtent se : locations) { - serverExtents.computeIfAbsent(se.location, k -> new HashMap<>()) + serverExtents.computeIfAbsent(csi(se.location), k -> new HashMap<>()) .computeIfAbsent(se.extent, k -> new ArrayList<>()).add(se.row); } @@ -505,13 +517,13 @@ public void testRemoveOverlapping2() { } static class TServers { - private final Map>> tservers = new HashMap<>(); + private final Map>> tservers = new HashMap<>(); private volatile BiConsumer lookupConsumer = (cachedTablet, row) -> {}; } static class TestCachedTabletObtainer implements CachedTabletObtainer { - private final Map>> tservers; + private final Map>> tservers; private final BiConsumer lookupConsumer; TestCachedTabletObtainer(TServers tservers) { @@ -552,7 +564,7 @@ public CachedTablets lookupTablet(ClientContext context, CachedTablet src, Text static class YesLockChecker implements TabletServerLockChecker { @Override - public boolean isLockHeld(String tserver, String session) { + public boolean isLockHeld(ServerId tserver, String session) { return true; } @@ -568,14 +580,14 @@ static class TestRootClientTabletCache extends RootClientTabletCache { @Override protected CachedTablet getRootTabletLocation(ClientContext context) { - return new CachedTablet(RootTable.EXTENT, rootTabletLoc, "1", TabletAvailability.HOSTED, + return new CachedTablet(RootTable.EXTENT, csi(rootTabletLoc), "1", TabletAvailability.HOSTED, false); } } static void createEmptyTablet(TServers tservers, String server, KeyExtent tablet) { Map> tablets = - tservers.tservers.computeIfAbsent(server, k -> new HashMap<>()); + tservers.tservers.computeIfAbsent(csi(server), k -> new HashMap<>()); SortedMap tabletData = tablets.computeIfAbsent(tablet, k -> new TreeMap<>()); if (!tabletData.isEmpty()) { throw new IllegalStateException("Asked for empty tablet, but non empty tablet exists"); @@ -584,7 +596,7 @@ static void createEmptyTablet(TServers tservers, String server, KeyExtent tablet static void clearLocation(TServers tservers, String server, KeyExtent tablet, KeyExtent ke, String instance) { - Map> tablets = tservers.tservers.get(server); + Map> tablets = tservers.tservers.get(csi(server)); if (tablets == null) { return; } @@ -601,20 +613,21 @@ static void clearLocation(TServers tservers, String server, KeyExtent tablet, Ke } static void setLocation(TServers tservers, String server, KeyExtent tablet, KeyExtent ke, - String location, String instance) { + String location, String session) { Map> tablets = - tservers.tservers.computeIfAbsent(server, k -> new HashMap<>()); + tservers.tservers.computeIfAbsent(csi(server), k -> new HashMap<>()); SortedMap tabletData = tablets.computeIfAbsent(tablet, k -> new TreeMap<>()); Text mr = ke.toMetaRow(); Value per = TabletColumnFamily.encodePrevEndRow(ke.prevEndRow()); if (location != null) { - if (instance == null) { - instance = ""; + if (session == null) { + session = ""; } - Key lk = new Key(mr, CurrentLocationColumnFamily.NAME, new Text(instance)); - tabletData.put(lk, new Value(location)); + Key lk = new Key(mr, CurrentLocationColumnFamily.NAME, new Text(session)); + var tsi = new TServerInstance(csi(location), session); + tabletData.put(lk, new Value(tsi.serialize())); } Key hk = new Key(mr, TabletColumnFamily.AVAILABILITY_COLUMN.getColumnFamily(), @@ -629,7 +642,7 @@ static void setLocation(TServers tservers, String server, KeyExtent tablet, KeyE static void deleteLocation(TServers tservers, String server, KeyExtent tablet, KeyExtent ke, String instance) { Map> tablets = - tservers.tservers.computeIfAbsent(server, k -> new HashMap<>()); + tservers.tservers.computeIfAbsent(csi(server), k -> new HashMap<>()); SortedMap tabletData = tablets.computeIfAbsent(tablet, k -> new TreeMap<>()); Text mr = ke.toMetaRow(); @@ -643,7 +656,7 @@ static void setLocation(TServers tservers, String server, KeyExtent tablet, KeyE } static void deleteServer(TServers tservers, String server) { - tservers.tservers.remove(server); + tservers.tservers.remove(csi(server)); } @@ -662,7 +675,7 @@ private void locateTabletTest(ClientTabletCacheImpl cache, String row, boolean s assertTrue(tl.getTserverLocation().isEmpty()); assertTrue(tl.getTserverSession().isEmpty()); } else { - assertEquals(server, tl.getTserverLocation().orElseThrow()); + assertEquals(server, tl.getTserverLocation().orElseThrow().getHost()); } assertEquals(expected, tl.getExtent()); } @@ -869,7 +882,7 @@ public void testBinRanges1() throws Exception { createLocators("foo", createNewKeyExtent("foo", null, null), "l1"); List ranges = createNewRangeList(createNewRange(null, null)); - Map>> expected = + Map>> expected = createExpectedBinnings(createRangeLocation("l1", createNewKeyExtent("foo", null, null), createNewRangeList(createNewRange(null, null)))); @@ -899,7 +912,7 @@ public void testBinRanges2() throws Exception { ClientTabletCacheImpl metaCache = createLocators("foo", createNewKeyExtent("foo", "g", null), "l1", createNewKeyExtent("foo", null, "g"), "l2"); - Map>> expected = createExpectedBinnings( createRangeLocation("l1", createNewKeyExtent("foo", "g", null), createNewRangeList(createNewRange(null, null))), @@ -918,7 +931,7 @@ public void testBinRanges3() throws Exception { createLocators("foo", createNewKeyExtent("foo", "g", null), "l1", createNewKeyExtent("foo", "m", "g"), "l2", createNewKeyExtent("foo", null, "m"), "l2"); - Map>> expected = createExpectedBinnings( + Map>> expected = createExpectedBinnings( createRangeLocation("l1", createNewKeyExtent("foo", "g", null), createNewRangeList(createNewRange(null, null))), createRangeLocation("l2", createNewKeyExtent("foo", "m", "g"), @@ -992,7 +1005,7 @@ public void testBinRanges4() throws Exception { "l1", createNewKeyExtent("foo", "1", "0"), "l2", createNewKeyExtent("foo", "2", "1"), "l3", createNewKeyExtent("foo", "3", "2"), "l4", createNewKeyExtent("foo", null, "3"), "l5"); - Map>> expected = + Map>> expected = createExpectedBinnings(createRangeLocation("l2", createNewKeyExtent("foo", "1", "0"), createNewRangeList(new Range(new Text("1"))))); @@ -1003,7 +1016,7 @@ public void testBinRanges4() throws Exception { new Range(rowColKey, true, new Key(new Text("3")).followingKey(PartialKey.ROW), false); ranges = createNewRangeList(range); - Map>> expected4 = createExpectedBinnings( + Map>> expected4 = createExpectedBinnings( createRangeLocation("l4", createNewKeyExtent("foo", "3", "2"), createNewRangeList(range))); runTest(ranges, metaCache, expected4, createNewRangeList()); @@ -1011,7 +1024,7 @@ public void testBinRanges4() throws Exception { range = new Range(rowColKey, true, new Key(new Text("3")).followingKey(PartialKey.ROW), true); ranges = createNewRangeList(range); - Map>> expected5 = createExpectedBinnings( + Map>> expected5 = createExpectedBinnings( createRangeLocation("l4", createNewKeyExtent("foo", "3", "2"), createNewRangeList(range)), createRangeLocation("l5", createNewKeyExtent("foo", null, "3"), createNewRangeList(range))); @@ -1019,26 +1032,26 @@ public void testBinRanges4() throws Exception { range = new Range(new Text("2"), false, new Text("3"), false); ranges = createNewRangeList(range); - Map>> expected6 = createExpectedBinnings( + Map>> expected6 = createExpectedBinnings( createRangeLocation("l4", createNewKeyExtent("foo", "3", "2"), createNewRangeList(range))); runTest(ranges, metaCache, expected6, createNewRangeList()); range = new Range(new Text("2"), true, new Text("3"), false); ranges = createNewRangeList(range); - Map>> expected7 = createExpectedBinnings( + Map>> expected7 = createExpectedBinnings( createRangeLocation("l3", createNewKeyExtent("foo", "2", "1"), createNewRangeList(range)), createRangeLocation("l4", createNewKeyExtent("foo", "3", "2"), createNewRangeList(range))); runTest(ranges, metaCache, expected7, createNewRangeList()); range = new Range(new Text("2"), false, new Text("3"), true); ranges = createNewRangeList(range); - Map>> expected8 = createExpectedBinnings( + Map>> expected8 = createExpectedBinnings( createRangeLocation("l4", createNewKeyExtent("foo", "3", "2"), createNewRangeList(range))); runTest(ranges, metaCache, expected8, createNewRangeList()); range = new Range(new Text("2"), true, new Text("3"), true); ranges = createNewRangeList(range); - Map>> expected9 = createExpectedBinnings( + Map>> expected9 = createExpectedBinnings( createRangeLocation("l3", createNewKeyExtent("foo", "2", "1"), createNewRangeList(range)), createRangeLocation("l4", createNewKeyExtent("foo", "3", "2"), createNewRangeList(range))); runTest(ranges, metaCache, expected9, createNewRangeList()); @@ -1054,14 +1067,14 @@ public void testBinRanges5() throws Exception { "l1", createNewKeyExtent("foo", "1", "0"), "l2", createNewKeyExtent("foo", "3", "2"), "l4", createNewKeyExtent("foo", null, "3"), "l5"); - Map>> expected1 = + Map>> expected1 = createExpectedBinnings(createRangeLocation("l2", createNewKeyExtent("foo", "1", "0"), createNewRangeList(new Range(new Text("1"))))); runTest(ranges, metaCache, expected1); ranges = createNewRangeList(new Range(new Text("2")), new Range(new Text("11"))); - Map>> expected2 = createExpectedBinnings(); + Map>> expected2 = createExpectedBinnings(); runTest(ranges, metaCache, expected2, ranges); @@ -1070,7 +1083,7 @@ public void testBinRanges5() throws Exception { runTest(ranges, metaCache, expected1, createNewRangeList(new Range(new Text("2")))); ranges = createNewRangeList(createNewRange("0", "2"), createNewRange("3", "4")); - Map>> expected3 = createExpectedBinnings( createRangeLocation("l4", createNewKeyExtent("foo", "3", "2"), createNewRangeList(createNewRange("3", "4"))), @@ -1082,7 +1095,7 @@ public void testBinRanges5() throws Exception { ranges = createNewRangeList(createNewRange("0", "1"), createNewRange("0", "11"), createNewRange("1", "2"), createNewRange("0", "4"), createNewRange("2", "4"), createNewRange("21", "4")); - Map>> expected4 = createExpectedBinnings( createRangeLocation("l1", createNewKeyExtent("foo", "0", null), createNewRangeList(createNewRange("0", "1"))), @@ -1111,7 +1124,7 @@ public void testBinRangesNonContiguousExtents() throws Exception { createLocators(tservers, "tserver1", "tserver2", "foo", e1, "l1", e2, "l1"); List ranges = createNewRangeList(createNewRange("01", "07")); - Map>> expected = createExpectedBinnings( createRangeLocation("l1", e1, createNewRangeList(createNewRange("01", "07"))), createRangeLocation("l1", e2, createNewRangeList(createNewRange("01", "07")))); @@ -1159,7 +1172,7 @@ public void testBinRangesNonContiguousExtentsAndMultipleRanges() throws Exceptio Range r2 = createNewRange("loop", "nope"); // overlaps e4 e5 Range r3 = createNewRange("silly", "sunny"); // overlaps e7 - Map>> expected = createExpectedBinnings( + Map>> expected = createExpectedBinnings( createRangeLocation("l1", e1, createNewRangeList(r1)), createRangeLocation("l1", e2, createNewRangeList(r1)), @@ -1193,13 +1206,13 @@ public void testBinRangesNonContiguousExtentsAndMultipleRanges() throws Exceptio @Test public void testIsContiguous() { - CachedTablet e1 = new CachedTablet(createNewKeyExtent("foo", "1", null), "l1", "1", + CachedTablet e1 = new CachedTablet(createNewKeyExtent("foo", "1", null), csi("l1"), "1", TabletAvailability.ONDEMAND, false); - CachedTablet e2 = new CachedTablet(createNewKeyExtent("foo", "2", "1"), "l1", "1", + CachedTablet e2 = new CachedTablet(createNewKeyExtent("foo", "2", "1"), csi("l1"), "1", TabletAvailability.ONDEMAND, false); - CachedTablet e3 = new CachedTablet(createNewKeyExtent("foo", "3", "2"), "l1", "1", + CachedTablet e3 = new CachedTablet(createNewKeyExtent("foo", "3", "2"), csi("l1"), "1", TabletAvailability.ONDEMAND, false); - CachedTablet e4 = new CachedTablet(createNewKeyExtent("foo", null, "3"), "l1", "1", + CachedTablet e4 = new CachedTablet(createNewKeyExtent("foo", null, "3"), csi("l1"), "1", TabletAvailability.ONDEMAND, false); assertTrue(ClientTabletCacheImpl.isContiguous(List.of(e1, e2, e3, e4))); @@ -1213,7 +1226,7 @@ public void testIsContiguous() { assertFalse(ClientTabletCacheImpl.isContiguous(List.of(e1, e2, e4))); assertFalse(ClientTabletCacheImpl.isContiguous(List.of(e1, e3, e4))); - CachedTablet e5 = new CachedTablet(createNewKeyExtent("foo", null, null), "l1", "1", + CachedTablet e5 = new CachedTablet(createNewKeyExtent("foo", null, null), csi("l1"), "1", TabletAvailability.ONDEMAND, false); assertFalse(ClientTabletCacheImpl.isContiguous(List.of(e1, e2, e3, e4, e5))); assertFalse(ClientTabletCacheImpl.isContiguous(List.of(e5, e1, e2, e3, e4))); @@ -1221,12 +1234,12 @@ public void testIsContiguous() { assertFalse(ClientTabletCacheImpl.isContiguous(List.of(e5, e2, e3, e4))); assertTrue(ClientTabletCacheImpl.isContiguous(List.of(e5))); - CachedTablet e6 = new CachedTablet(createNewKeyExtent("foo", null, "1"), "l1", "1", + CachedTablet e6 = new CachedTablet(createNewKeyExtent("foo", null, "1"), csi("l1"), "1", TabletAvailability.ONDEMAND, false); assertFalse(ClientTabletCacheImpl.isContiguous(List.of(e1, e2, e3, e6))); - CachedTablet e7 = new CachedTablet(createNewKeyExtent("foo", "33", "11"), "l1", "1", + CachedTablet e7 = new CachedTablet(createNewKeyExtent("foo", "33", "11"), csi("l1"), "1", TabletAvailability.ONDEMAND, false); assertFalse(ClientTabletCacheImpl.isContiguous(List.of(e1, e2, e7, e4))); @@ -1240,7 +1253,7 @@ public void testBinMutations1() throws Exception { List ml = createNewMutationList(createNewMutation("a", "cf1:cq1=v1", "cf1:cq2=v2"), createNewMutation("c", "cf1:cq1=v3", "cf1:cq2=v4")); - Map>> emb = createServerExtentMap( + Map>> emb = createServerExtentMap( createServerExtent("a", "l1", ke1), createServerExtent("c", "l1", ke1)); runTest(metaCache, ml, emb); @@ -1263,7 +1276,7 @@ public void testBinMutations2() throws Exception { List ml = createNewMutationList(createNewMutation("a", "cf1:cq1=v1", "cf1:cq2=v2"), createNewMutation("c", "cf1:cq1=v3", "cf1:cq2=v4")); - Map>> emb = createServerExtentMap(); + Map>> emb = createServerExtentMap(); runTest(metaCache, ml, emb, "a", "c"); } @@ -1278,7 +1291,7 @@ public void testBinMutations3() throws Exception { List ml = createNewMutationList(createNewMutation("a", "cf1:cq1=v1", "cf1:cq2=v2"), createNewMutation("i", "cf1:cq1=v3", "cf1:cq2=v4")); - Map>> emb = createServerExtentMap( + Map>> emb = createServerExtentMap( createServerExtent("a", "l1", ke1), createServerExtent("i", "l2", ke2)); runTest(metaCache, ml, emb); @@ -1322,7 +1335,7 @@ public void testBinMutations4() throws Exception { List ml = createNewMutationList(createNewMutation("a", "cf1:cq1=v1", "cf1:cq2=v2"), createNewMutation("i", "cf1:cq1=v3", "cf1:cq2=v4")); - Map>> emb = + Map>> emb = createServerExtentMap(createServerExtent("a", "l1", ke1)); runTest(metaCache, ml, emb, "i"); @@ -1372,7 +1385,7 @@ public void testBinSplit() throws Exception { List ml = createNewMutationList(createNewMutation("a", "cf1:cq1=v1", "cf1:cq2=v2"), createNewMutation("m", "cf1:cq1=v3", "cf1:cq2=v4"), createNewMutation("z", "cf1:cq1=v5")); - Map>> emb = + Map>> emb = createServerExtentMap(createServerExtent("a", "l1", ke1), createServerExtent("m", "l1", ke1), createServerExtent("z", "l1", ke1)); if (i == 0 || i == 2) { @@ -1382,7 +1395,7 @@ public void testBinSplit() throws Exception { List ranges = createNewRangeList(new Range(new Text("a")), new Range(new Text("m")), new Range(new Text("z"))); - Map>> expected1 = createExpectedBinnings( + Map>> expected1 = createExpectedBinnings( createRangeLocation("l1", createNewKeyExtent("foo", null, null), ranges)); if (i == 1 || i == 2) { @@ -1401,7 +1414,7 @@ public void testBinSplit() throws Exception { runTest(metaCache, ml, emb, "a", "m"); } - Map>> expected2 = + Map>> expected2 = createExpectedBinnings(createRangeLocation("l2", createNewKeyExtent("foo", null, "n"), createNewRangeList(new Range(new Text("z"))))); @@ -1417,7 +1430,7 @@ public void testBinSplit() throws Exception { runTest(metaCache, ml, emb); } - Map>> expected3 = createExpectedBinnings( createRangeLocation("l2", createNewKeyExtent("foo", null, "n"), createNewRangeList(new Range(new Text("z")))), @@ -1484,7 +1497,7 @@ public void testBug2() throws Exception { // create the ~ tablet so it exists Map> ts3 = new HashMap<>(); ts3.put(mte2, new TreeMap<>()); - tservers.tservers.put("tserver3", ts3); + tservers.tservers.put(csi("tserver3"), ts3); assertNull(tab0TabletCache.findTablet(context, new Text("row_0000000000"), false, LocationNeed.REQUIRED)); @@ -1561,8 +1574,8 @@ public void testLostLock() throws Exception { ClientTabletCacheImpl metaCache = createLocators(tservers, "tserver1", "tserver2", "foo", new TabletServerLockChecker() { @Override - public boolean isLockHeld(String tserver, String session) { - return activeLocks.contains(tserver + ":" + session); + public boolean isLockHeld(ServerId tserver, String session) { + return activeLocks.contains(tserver.getHost() + ":" + session); } @Override @@ -1611,7 +1624,7 @@ public void invalidateCache(String server) {} List ml = createNewMutationList(createNewMutation("a", "cf1:cq1=v1", "cf1:cq2=v2"), createNewMutation("w", "cf1:cq3=v3")); - Map>> emb = createServerExtentMap( + Map>> emb = createServerExtentMap( createServerExtent("a", "L3", ke1), createServerExtent("w", "L3", ke1)); runTest(metaCache, ml, emb); @@ -1647,7 +1660,7 @@ public void invalidateCache(String server) {} List ranges = createNewRangeList(new Range("a"), createNewRange("b", "o"), createNewRange("r", "z")); - Map>> expected = createExpectedBinnings( createRangeLocation("L1", ke11, createNewRangeList(new Range("a"), createNewRange("b", "o"))), @@ -1728,7 +1741,7 @@ public void testCachingUnhosted() throws Exception { assertEquals(List.of(), failures); var tl1 = new CachedTablet(ke1, TabletAvailability.ONDEMAND, false); var tl2 = new CachedTablet(ke2, TabletAvailability.ONDEMAND, false); - var tl3 = new CachedTablet(ke3, "L2", "I2", TabletAvailability.ONDEMAND, false); + var tl3 = new CachedTablet(ke3, csi("L2"), "I2", TabletAvailability.ONDEMAND, false); var expected = Set.of(new Pair<>(tl1, r1), new Pair<>(tl1, r2), new Pair<>(tl2, r2), new Pair<>(tl3, r2)); assertEquals(expected, actual); @@ -1755,7 +1768,7 @@ public void testCachingUnhosted() throws Exception { failures = metaCache.findTablets(context, ranges, (tl, r) -> actual.add(new Pair<>(tl, r)), LocationNeed.NOT_REQUIRED); assertEquals(List.of(), failures); - tl1 = new CachedTablet(ke1, "L3", "I3", TabletAvailability.ONDEMAND, false); + tl1 = new CachedTablet(ke1, csi("L3"), "I3", TabletAvailability.ONDEMAND, false); expected = Set.of(new Pair<>(tl1, r1), new Pair<>(tl1, r2), new Pair<>(tl2, r2), new Pair<>(tl3, r2)); assertEquals(expected, actual); @@ -1764,7 +1777,7 @@ public void testCachingUnhosted() throws Exception { failures = metaCache.findTablets(context, ranges, (tl, r) -> actual.add(new Pair<>(tl, r)), LocationNeed.REQUIRED); assertEquals(List.of(), failures); - tl2 = new CachedTablet(ke2, "L4", "I4", TabletAvailability.ONDEMAND, false); + tl2 = new CachedTablet(ke2, csi("L4"), "I4", TabletAvailability.ONDEMAND, false); expected = Set.of(new Pair<>(tl1, r1), new Pair<>(tl1, r2), new Pair<>(tl2, r2), new Pair<>(tl3, r2)); assertEquals(expected, actual); @@ -1783,7 +1796,7 @@ public void testCachingUnhosted() throws Exception { // one mutation should fail because there is no location for ke2 List ml = createNewMutationList(createNewMutation("a", "cf1:cq1=v1", "cf1:cq2=v2"), createNewMutation("i", "cf1:cq1=v3", "cf1:cq2=v4")); - Map>> emb = + Map>> emb = createServerExtentMap(createServerExtent("a", "L3", ke1)); runTest(metaCache, ml, emb, "i"); @@ -1823,7 +1836,7 @@ public void testLookupCounts() throws Exception { for (char c = 'a'; c <= 'z'; c++) { ml.add(createNewMutation("" + c, "cf1:cq1=v3", "cf1:cq2=v4")); } - Map> binnedMutations = new HashMap<>(); + Map> binnedMutations = new HashMap<>(); List afailures = new ArrayList<>(); assertEquals(List.of(), lookups); metaCache.binMutations(context, ml, binnedMutations, afailures); @@ -1891,11 +1904,12 @@ public void testLookupCounts() throws Exception { afailures.clear(); assertEquals(List.of(), lookups); metaCache.binMutations(context, ml, binnedMutations, afailures); - assertEquals(Set.of("T1", "T2", "T3"), binnedMutations.keySet()); - assertEquals(7, binnedMutations.get("T1").getMutations().get(ke1).size()); - assertEquals(6, binnedMutations.get("T2").getMutations().get(ke2).size()); - assertEquals(5, binnedMutations.get("T3").getMutations().get(ke3).size()); - assertEquals(8, binnedMutations.get("T3").getMutations().get(ke4).size()); + assertEquals(Set.of("T1", "T2", "T3"), + binnedMutations.keySet().stream().map(ServerId::getHost).collect(Collectors.toSet())); + assertEquals(7, binnedMutations.get(csi("T1")).getMutations().get(ke1).size()); + assertEquals(6, binnedMutations.get(csi("T2")).getMutations().get(ke2).size()); + assertEquals(5, binnedMutations.get(csi("T3")).getMutations().get(ke3).size()); + assertEquals(8, binnedMutations.get(csi("T3")).getMutations().get(ke4).size()); assertEquals(0, afailures.size()); assertEquals(List.of(METADATA_TABLE_EXTENT), lookups); @@ -1904,11 +1918,12 @@ public void testLookupCounts() throws Exception { binnedMutations.clear(); afailures.clear(); metaCache.binMutations(context, ml, binnedMutations, afailures); - assertEquals(Set.of("T1", "T2", "T3"), binnedMutations.keySet()); - assertEquals(7, binnedMutations.get("T1").getMutations().get(ke1).size()); - assertEquals(6, binnedMutations.get("T2").getMutations().get(ke2).size()); - assertEquals(5, binnedMutations.get("T3").getMutations().get(ke3).size()); - assertEquals(8, binnedMutations.get("T3").getMutations().get(ke4).size()); + assertEquals(Set.of("T1", "T2", "T3"), + binnedMutations.keySet().stream().map(ServerId::getHost).collect(Collectors.toSet())); + assertEquals(7, binnedMutations.get(csi("T1")).getMutations().get(ke1).size()); + assertEquals(6, binnedMutations.get(csi("T2")).getMutations().get(ke2).size()); + assertEquals(5, binnedMutations.get(csi("T3")).getMutations().get(ke3).size()); + assertEquals(8, binnedMutations.get(csi("T3")).getMutations().get(ke4).size()); assertEquals(0, afailures.size()); assertEquals(List.of(), lookups); @@ -2020,11 +2035,11 @@ public void testMultithreadedLookups() throws Exception { } var loc = metaCache.findTablet(context, new Text(lookup), false, LocationNeed.REQUIRED); if (lookup.compareTo("m") <= 0) { - assertEquals("tserver7", loc.getTserverLocation().orElseThrow()); + assertEquals("tserver7", loc.getTserverLocation().orElseThrow().getHost()); } else if (lookup.compareTo("q") <= 0) { - assertEquals("tserver8", loc.getTserverLocation().orElseThrow()); + assertEquals("tserver8", loc.getTserverLocation().orElseThrow().getHost()); } else { - assertEquals("tserver9", loc.getTserverLocation().orElseThrow()); + assertEquals("tserver9", loc.getTserverLocation().orElseThrow().getHost()); } return loc; }); @@ -2086,16 +2101,16 @@ public void testNonBlocking() throws Exception { assertEquals("tserver3", metaCache.findTablet(context, new Text("a"), false, LocationNeed.REQUIRED) - .getTserverLocation().orElseThrow()); + .getTserverLocation().orElseThrow().getHost()); assertEquals("tserver4", metaCache.findTablet(context, new Text("h"), false, LocationNeed.REQUIRED) - .getTserverLocation().orElseThrow()); + .getTserverLocation().orElseThrow().getHost()); assertEquals("tserver5", metaCache.findTablet(context, new Text("n"), false, LocationNeed.REQUIRED) - .getTserverLocation().orElseThrow()); + .getTserverLocation().orElseThrow().getHost()); assertEquals("tserver6", metaCache.findTablet(context, new Text("s"), false, LocationNeed.REQUIRED) - .getTserverLocation().orElseThrow()); + .getTserverLocation().orElseThrow().getHost()); // clear this extent from cache to cause it to be looked up again in the metadata table metaCache.invalidateCache(ke2); @@ -2125,13 +2140,13 @@ public void testNonBlocking() throws Exception { // should be able to get the tablet locations that are still in the cache w/o blocking assertEquals("tserver3", metaCache.findTablet(context, new Text("a"), false, LocationNeed.REQUIRED) - .getTserverLocation().orElseThrow()); + .getTserverLocation().orElseThrow().getHost()); assertEquals("tserver5", metaCache.findTablet(context, new Text("n"), false, LocationNeed.REQUIRED) - .getTserverLocation().orElseThrow()); + .getTserverLocation().orElseThrow().getHost()); assertEquals("tserver6", metaCache.findTablet(context, new Text("s"), false, LocationNeed.REQUIRED) - .getTserverLocation().orElseThrow()); + .getTserverLocation().orElseThrow().getHost()); // The lookup task should still be blocked assertFalse(lookupFuture.isDone()); @@ -2141,7 +2156,7 @@ public void testNonBlocking() throws Exception { lookupLock.unlock(); // The future should be able to run and complete - assertEquals("tserver4", lookupFuture.get()); + assertEquals("tserver4", lookupFuture.get().getHost()); // verify test assumptions assertTrue(lookupFuture.isDone()); @@ -2218,7 +2233,7 @@ private void checkLocations(Map expectedLocations, ClientTabletCa String location = entry.getValue(); assertEquals(location, metaCache.findTablet(context, new Text(row), false, LocationNeed.REQUIRED) - .getTserverLocation().orElseThrow()); + .getTserverLocation().orElseThrow().getHost()); } } } diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/ScanAttemptsImplTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/ScanAttemptsImplTest.java index eee7d2ce77c..63ca1bbab6e 100644 --- a/core/src/test/java/org/apache/accumulo/core/clientImpl/ScanAttemptsImplTest.java +++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/ScanAttemptsImplTest.java @@ -40,7 +40,7 @@ public class ScanAttemptsImplTest { map.forEach((tabletId, scanAttempts) -> { Set stringAttempts = new HashSet<>(); scanAttempts.forEach(scanAttempt -> stringAttempts - .add(scanAttempt.getServer() + "_" + scanAttempt.getResult())); + .add(scanAttempt.getServer().toHostPortString() + "_" + scanAttempt.getResult())); ret.put(tabletId, stringAttempts); }); @@ -57,7 +57,7 @@ public void testBasic() { var tablet1 = nti("1", "a"); - var reporter1 = sai.createReporter("ss1:1", tablet1); + var reporter1 = sai.createReporter(ServerIdUtil.sserver("ss1", 1101), tablet1); reporter1.report(ScanServerAttempt.Result.BUSY); @@ -65,21 +65,21 @@ public void testBasic() { var snap2 = sai.snapshot(); - assertEquals(Map.of(tablet1, Set.of("ss1:1_BUSY")), simplify(snap2)); + assertEquals(Map.of(tablet1, Set.of("ss1:1101_BUSY")), simplify(snap2)); reporter1.report(ScanServerAttempt.Result.ERROR); assertEquals(Map.of(), snap1); - assertEquals(Map.of(tablet1, Set.of("ss1:1_BUSY")), simplify(snap2)); + assertEquals(Map.of(tablet1, Set.of("ss1:1101_BUSY")), simplify(snap2)); var snap3 = sai.snapshot(); - assertEquals(Map.of(tablet1, Set.of("ss1:1_BUSY", "ss1:1_ERROR")), simplify(snap3)); + assertEquals(Map.of(tablet1, Set.of("ss1:1101_BUSY", "ss1:1101_ERROR")), simplify(snap3)); var tablet2 = nti("1", "m"); - var reporter2 = sai.createReporter("ss1:1", tablet2); + var reporter2 = sai.createReporter(ServerIdUtil.sserver("ss1", 1101), tablet2); var tablet3 = nti("2", "r"); - var reporter3 = sai.createReporter("ss2:2", tablet3); + var reporter3 = sai.createReporter(ServerIdUtil.sserver("ss2", 1102), tablet3); reporter2.report(ScanServerAttempt.Result.BUSY); reporter3.report(ScanServerAttempt.Result.ERROR); @@ -87,10 +87,10 @@ public void testBasic() { var snap4 = sai.snapshot(); assertEquals(Map.of(), snap1); - assertEquals(Map.of(tablet1, Set.of("ss1:1_BUSY")), simplify(snap2)); - assertEquals(Map.of(tablet1, Set.of("ss1:1_BUSY", "ss1:1_ERROR")), simplify(snap3)); - assertEquals(Map.of(tablet1, Set.of("ss1:1_BUSY", "ss1:1_ERROR"), tablet2, Set.of("ss1:1_BUSY"), - tablet3, Set.of("ss2:2_ERROR")), simplify(snap4)); + assertEquals(Map.of(tablet1, Set.of("ss1:1101_BUSY")), simplify(snap2)); + assertEquals(Map.of(tablet1, Set.of("ss1:1101_BUSY", "ss1:1101_ERROR")), simplify(snap3)); + assertEquals(Map.of(tablet1, Set.of("ss1:1101_BUSY", "ss1:1101_ERROR"), tablet2, + Set.of("ss1:1101_BUSY"), tablet3, Set.of("ss2:1102_ERROR")), simplify(snap4)); } } diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/ThriftTransportKeyTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/ThriftTransportKeyTest.java index 858bb64771c..46c0c54b504 100644 --- a/core/src/test/java/org/apache/accumulo/core/clientImpl/ThriftTransportKeyTest.java +++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/ThriftTransportKeyTest.java @@ -43,8 +43,6 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import com.google.common.net.HostAndPort; - public class ThriftTransportKeyTest { private static final String primary = "accumulo"; @@ -79,7 +77,7 @@ public void testSslAndSaslErrors() { try { assertThrows(RuntimeException.class, () -> new ThriftTransportKey(ThriftClientTypes.CLIENT, - HostAndPort.fromParts("localhost", 9999), 120_000, clientCtx)); + ServerIdUtil.tserver("localhost", 9999), 120_000, clientCtx)); } finally { verify(clientCtx); } @@ -99,9 +97,9 @@ public void testConnectionCaching() throws IOException, InterruptedException { user1.doAs((PrivilegedExceptionAction) () -> createSaslParams(token)); ThriftTransportKey ttk1 = new ThriftTransportKey(ThriftClientTypes.CLIENT, - HostAndPort.fromParts("localhost", 9997), 1L, null, saslParams1); + ServerIdUtil.tserver("localhost", 9997), 1L, null, saslParams1); ThriftTransportKey ttk2 = new ThriftTransportKey(ThriftClientTypes.CLIENT, - HostAndPort.fromParts("localhost", 9997), 1L, null, saslParams2); + ServerIdUtil.tserver("localhost", 9997), 1L, null, saslParams2); // Should equals() and hashCode() to make sure we don't throw away thrift cnxns assertEquals(ttk1, ttk2); @@ -120,9 +118,9 @@ public void testSaslPrincipalIsSignificant() throws IOException, InterruptedExce user2.doAs((PrivilegedExceptionAction) () -> createSaslParams(token)); ThriftTransportKey ttk1 = new ThriftTransportKey(ThriftClientTypes.CLIENT, - HostAndPort.fromParts("localhost", 9997), 1L, null, saslParams1); + ServerIdUtil.tserver("localhost", 9997), 1L, null, saslParams1); ThriftTransportKey ttk2 = new ThriftTransportKey(ThriftClientTypes.CLIENT, - HostAndPort.fromParts("localhost", 9997), 1L, null, saslParams2); + ServerIdUtil.tserver("localhost", 9997), 1L, null, saslParams2); assertNotEquals(ttk1, ttk2); assertNotEquals(ttk1.hashCode(), ttk2.hashCode()); @@ -138,7 +136,7 @@ public void testSimpleEquivalence() { replay(clientCtx); ThriftTransportKey ttk = new ThriftTransportKey(ThriftClientTypes.CLIENT, - HostAndPort.fromParts("localhost", 9999), 120_000, clientCtx); + ServerIdUtil.tserver("localhost", 9999), 120_000, clientCtx); assertEquals(ttk, ttk, "Normal ThriftTransportKey doesn't equal itself"); assertEquals(ttk.hashCode(), ttk.hashCode()); diff --git a/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockDataTest.java b/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockDataTest.java index 6a628690577..c221d60635d 100644 --- a/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockDataTest.java +++ b/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockDataTest.java @@ -21,12 +21,12 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.Optional; import java.util.UUID; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.ResourceGroupId; import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptor; import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptors; @@ -41,84 +41,83 @@ public class ServiceLockDataTest { @Test public void testSingleServiceConstructor() throws Exception { - ServiceLockData ss = - new ServiceLockData(serverUUID, "127.0.0.1", ThriftService.TSERV, ResourceGroupId.DEFAULT); + ServiceLockData ss = new ServiceLockData(serverUUID, ServerIdUtil.tserver("127.0.0.1", 9997), + ThriftService.TSERV); assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV)); - assertEquals("127.0.0.1", ss.getAddressString(ThriftService.TSERV)); - assertThrows(IllegalArgumentException.class, () -> ss.getAddress(ThriftService.TSERV)); + assertEquals("127.0.0.1", ss.getServer(ThriftService.TSERV).getHost()); assertEquals(ResourceGroupId.DEFAULT, ss.getGroup(ThriftService.TSERV)); assertNull(ss.getServerUUID(ThriftService.TABLET_SCAN)); - assertNull(ss.getAddressString(ThriftService.TABLET_SCAN)); - assertNull(ss.getAddress(ThriftService.TABLET_SCAN)); + assertNull(ss.getServer(ThriftService.TABLET_SCAN)); assertNull(ss.getGroup(ThriftService.TABLET_SCAN)); } @Test public void testMultipleServiceConstructor() throws Exception { ServiceDescriptors sds = new ServiceDescriptors(); - sds.addService(new ServiceDescriptor(serverUUID, ThriftService.TSERV, "127.0.0.1:9997", - ResourceGroupId.DEFAULT)); - sds.addService(new ServiceDescriptor(serverUUID, ThriftService.TABLET_SCAN, "127.0.0.1:9998", - ResourceGroupId.DEFAULT)); + sds.addService(new ServiceDescriptor(serverUUID, ThriftService.TSERV, + ServerIdUtil.tserver("127.0.0.1", 9997))); + sds.addService(new ServiceDescriptor(serverUUID, ThriftService.TABLET_SCAN, + ServerIdUtil.tserver("127.0.0.1", 9998))); ServiceLockData ss = new ServiceLockData(sds); assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV)); - assertEquals("127.0.0.1:9997", ss.getAddressString(ThriftService.TSERV)); - assertEquals(HostAndPort.fromString("127.0.0.1:9997"), ss.getAddress(ThriftService.TSERV)); + assertEquals("127.0.0.1:9997", ss.getServer(ThriftService.TSERV).toHostPortString()); + assertEquals(HostAndPort.fromString("127.0.0.1:9997"), HostAndPort.fromParts( + ss.getServer(ThriftService.TSERV).getHost(), ss.getServer(ThriftService.TSERV).getPort())); assertEquals(ResourceGroupId.DEFAULT, ss.getGroup(ThriftService.TSERV)); assertEquals(serverUUID, ss.getServerUUID(ThriftService.TABLET_SCAN)); - assertEquals("127.0.0.1:9998", ss.getAddressString(ThriftService.TABLET_SCAN)); + assertEquals("127.0.0.1:9998", ss.getServer(ThriftService.TABLET_SCAN).toHostPortString()); assertEquals(HostAndPort.fromString("127.0.0.1:9998"), - ss.getAddress(ThriftService.TABLET_SCAN)); + HostAndPort.fromParts(ss.getServer(ThriftService.TABLET_SCAN).getHost(), + ss.getServer(ThriftService.TABLET_SCAN).getPort())); assertEquals(ResourceGroupId.DEFAULT, ss.getGroup(ThriftService.TSERV)); } @Test public void testSingleServiceConstructorWithGroup() throws Exception { - ServiceLockData ss = new ServiceLockData(serverUUID, "127.0.0.1", ThriftService.TSERV, - ResourceGroupId.of("meta")); + ServiceLockData ss = new ServiceLockData(serverUUID, + ServerIdUtil.tserver(ResourceGroupId.of("meta"), "127.0.0.1", 9997), ThriftService.TSERV); assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV)); - assertEquals("127.0.0.1", ss.getAddressString(ThriftService.TSERV)); - assertThrows(IllegalArgumentException.class, () -> ss.getAddress(ThriftService.TSERV)); + assertEquals("127.0.0.1:9997", ss.getServer(ThriftService.TSERV).toHostPortString()); assertEquals(ResourceGroupId.of("meta"), ss.getGroup(ThriftService.TSERV)); assertNull(ss.getServerUUID(ThriftService.TABLET_SCAN)); - assertNull(ss.getAddressString(ThriftService.TABLET_SCAN)); - assertNull(ss.getAddress(ThriftService.TABLET_SCAN)); + assertNull(ss.getServer(ThriftService.TABLET_SCAN)); assertNull(ss.getGroup(ThriftService.TABLET_SCAN)); } @Test public void testSingleServiceConstructor2WithGroup() throws Exception { - ServiceLockData ss = new ServiceLockData(serverUUID, "127.0.0.1", ThriftService.TSERV, - ResourceGroupId.of("meta")); + ServiceLockData ss = new ServiceLockData(serverUUID, + ServerIdUtil.tserver(ResourceGroupId.of("meta"), "127.0.0.1", 9997), ThriftService.TSERV); assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV)); - assertEquals("127.0.0.1", ss.getAddressString(ThriftService.TSERV)); - assertThrows(IllegalArgumentException.class, () -> ss.getAddress(ThriftService.TSERV)); + assertEquals("127.0.0.1:9997", ss.getServer(ThriftService.TSERV).toHostPortString()); assertEquals(ResourceGroupId.of("meta"), ss.getGroup(ThriftService.TSERV)); assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV)); - assertNull(ss.getAddressString(ThriftService.TABLET_SCAN)); - assertNull(ss.getAddress(ThriftService.TABLET_SCAN)); + assertNull(ss.getServerUUID(ThriftService.TABLET_SCAN)); + assertNull(ss.getServer(ThriftService.TABLET_SCAN)); assertNull(ss.getGroup(ThriftService.TABLET_SCAN)); } @Test public void testMultipleServiceConstructorWithGroup() throws Exception { ServiceDescriptors sds = new ServiceDescriptors(); - sds.addService(new ServiceDescriptor(serverUUID, ThriftService.TSERV, "127.0.0.1:9997", - ResourceGroupId.of("meta"))); - sds.addService(new ServiceDescriptor(serverUUID, ThriftService.TABLET_SCAN, "127.0.0.1:9998", - ResourceGroupId.of("ns1"))); + sds.addService(new ServiceDescriptor(serverUUID, ThriftService.TSERV, + ServerIdUtil.tserver(ResourceGroupId.of("meta"), "127.0.0.1", 9997))); + sds.addService(new ServiceDescriptor(serverUUID, ThriftService.TABLET_SCAN, + ServerIdUtil.tserver(ResourceGroupId.of("ns1"), "127.0.0.1", 9998))); ServiceLockData ss = new ServiceLockData(sds); assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV)); - assertEquals("127.0.0.1:9997", ss.getAddressString(ThriftService.TSERV)); - assertEquals(HostAndPort.fromString("127.0.0.1:9997"), ss.getAddress(ThriftService.TSERV)); + assertEquals("127.0.0.1:9997", ss.getServer(ThriftService.TSERV).toHostPortString()); + assertEquals(HostAndPort.fromString("127.0.0.1:9997"), HostAndPort.fromParts( + ss.getServer(ThriftService.TSERV).getHost(), ss.getServer(ThriftService.TSERV).getPort())); assertEquals(ResourceGroupId.of("meta"), ss.getGroup(ThriftService.TSERV)); assertEquals(serverUUID, ss.getServerUUID(ThriftService.TABLET_SCAN)); - assertEquals("127.0.0.1:9998", ss.getAddressString(ThriftService.TABLET_SCAN)); + assertEquals("127.0.0.1:9998", ss.getServer(ThriftService.TABLET_SCAN).toHostPortString()); assertEquals(HostAndPort.fromString("127.0.0.1:9998"), - ss.getAddress(ThriftService.TABLET_SCAN)); + HostAndPort.fromParts(ss.getServer(ThriftService.TABLET_SCAN).getHost(), + ss.getServer(ThriftService.TABLET_SCAN).getPort())); assertEquals(ResourceGroupId.of("ns1"), ss.getGroup(ThriftService.TABLET_SCAN)); - assertNull(ss.getAddressString(ThriftService.COMPACTOR)); - assertNull(ss.getAddress(ThriftService.COMPACTOR)); + assertNull(ss.getServerUUID(ThriftService.COMPACTOR)); + assertNull(ss.getServer(ThriftService.COMPACTOR)); assertNull(ss.getGroup(ThriftService.COMPACTOR)); } diff --git a/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockPathsTest.java b/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockPathsTest.java index c4043e5fb4e..defcf167796 100644 --- a/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockPathsTest.java +++ b/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockPathsTest.java @@ -44,6 +44,7 @@ import java.util.Set; import java.util.UUID; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.ResourceGroupId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.lock.ServiceLockData.ThriftService; @@ -179,7 +180,7 @@ public void testGetGarbageCollector() { UUID uuid = UUID.randomUUID(); String svcLock1 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + "#0000000001"; String svcLock2 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + "#0000000002"; - var sld = new ServiceLockData(uuid, HOSTNAME, ThriftService.GC, TEST_RESOURCE_GROUP); + var sld = new ServiceLockData(uuid, ServerIdUtil.gc("localhost", 1234), ThriftService.GC); expect(zc.getChildren(ZGC_LOCK)).andReturn(List.of(svcLock1, svcLock2)).anyTimes(); expect(zc.get(EasyMock.eq(ZGC_LOCK + "/" + svcLock1), EasyMock.isA(ZcStat.class))) @@ -221,7 +222,8 @@ public void testGetManager() { UUID uuid = UUID.randomUUID(); String svcLock1 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + "#0000000001"; String svcLock2 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + "#0000000002"; - var sld = new ServiceLockData(uuid, HOSTNAME, ThriftService.MANAGER, TEST_RESOURCE_GROUP); + var sld = + new ServiceLockData(uuid, ServerIdUtil.manager("localhost", 9995), ThriftService.MANAGER); expect(zc.getChildren(ZMANAGER_LOCK)).andReturn(List.of(svcLock1, svcLock2)).anyTimes(); expect(zc.get(EasyMock.eq(ZMANAGER_LOCK + "/" + svcLock1), EasyMock.isA(ZcStat.class))) @@ -263,7 +265,8 @@ public void testGetMonitor() { UUID uuid = UUID.randomUUID(); String svcLock1 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + "#0000000001"; String svcLock2 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + "#0000000002"; - var sld = new ServiceLockData(uuid, HOSTNAME, ThriftService.NONE, TEST_RESOURCE_GROUP); + var sld = + new ServiceLockData(uuid, ServerIdUtil.monitor("localhost", 9996), ThriftService.NONE); expect(zc.getChildren(ZMONITOR_LOCK)).andReturn(List.of(svcLock1, svcLock2)).anyTimes(); expect(zc.get(EasyMock.eq(ZMONITOR_LOCK + "/" + svcLock1), EasyMock.isA(ZcStat.class))) @@ -299,9 +302,10 @@ public void testGetCompactors() { UUID uuid = UUID.randomUUID(); String svcLock1 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + "#0000000001"; String svcLock2 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + "#0000000002"; - var sld1 = new ServiceLockData(uuid, HOSTNAME, ThriftService.COMPACTOR, TEST_RESOURCE_GROUP); - var sld2 = - new ServiceLockData(uuid, HOSTNAME, ThriftService.COMPACTOR, ResourceGroupId.DEFAULT); + var sld1 = new ServiceLockData(uuid, + ServerIdUtil.compactor(TEST_RESOURCE_GROUP, "localhost", 9876), ThriftService.COMPACTOR); + var sld2 = new ServiceLockData(uuid, ServerIdUtil.compactor("localhost", 9877), + ThriftService.COMPACTOR); expect(zc.getChildren(ZCOMPACTORS)) .andReturn(List.of(TEST_RESOURCE_GROUP.canonical(), ResourceGroupId.DEFAULT.canonical())) @@ -444,9 +448,10 @@ public void testGetScanServers() { UUID uuid = UUID.randomUUID(); String svcLock1 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + "#0000000001"; String svcLock2 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + "#0000000002"; - var sld1 = new ServiceLockData(uuid, HOSTNAME, ThriftService.TABLET_SCAN, TEST_RESOURCE_GROUP); - var sld2 = - new ServiceLockData(uuid, HOSTNAME, ThriftService.TABLET_SCAN, ResourceGroupId.DEFAULT); + var sld1 = new ServiceLockData(uuid, + ServerIdUtil.sserver(TEST_RESOURCE_GROUP, "localhost", 9876), ThriftService.TABLET_SCAN); + var sld2 = new ServiceLockData(uuid, ServerIdUtil.sserver("localhost", 9877), + ThriftService.TABLET_SCAN); expect(zc.getChildren(ZSSERVERS)) .andReturn(List.of(TEST_RESOURCE_GROUP.canonical(), ResourceGroupId.DEFAULT.canonical())) @@ -582,9 +587,10 @@ public void testGetTabletServers() { UUID uuid = UUID.randomUUID(); String svcLock1 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + "#0000000001"; String svcLock2 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + "#0000000002"; - var sld1 = new ServiceLockData(uuid, HOSTNAME, ThriftService.TABLET_SCAN, TEST_RESOURCE_GROUP); - var sld2 = - new ServiceLockData(uuid, HOSTNAME, ThriftService.TABLET_SCAN, ResourceGroupId.DEFAULT); + var sld1 = new ServiceLockData(uuid, + ServerIdUtil.tserver(TEST_RESOURCE_GROUP, "localhost", 9876), ThriftService.TABLET_SCAN); + var sld2 = new ServiceLockData(uuid, ServerIdUtil.tserver("localhost", 9877), + ThriftService.TABLET_SCAN); expect(zc.getChildren(ZTSERVERS)) .andReturn(List.of(TEST_RESOURCE_GROUP.canonical(), ResourceGroupId.DEFAULT.canonical())) @@ -720,9 +726,10 @@ public void testGetDeadTabletServers() { UUID uuid = UUID.randomUUID(); String svcLock1 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + "#0000000001"; String svcLock2 = ServiceLock.ZLOCK_PREFIX + uuid.toString() + "#0000000002"; - var sld1 = new ServiceLockData(uuid, HOSTNAME, ThriftService.TABLET_SCAN, TEST_RESOURCE_GROUP); - var sld2 = - new ServiceLockData(uuid, HOSTNAME, ThriftService.TABLET_SCAN, ResourceGroupId.DEFAULT); + var sld1 = new ServiceLockData(uuid, + ServerIdUtil.tserver(TEST_RESOURCE_GROUP, "localhost", 9876), ThriftService.TABLET_SCAN); + var sld2 = new ServiceLockData(uuid, ServerIdUtil.tserver("localhost", 9877), + ThriftService.TABLET_SCAN); expect(zc.getChildren(ZDEADTSERVERS)) .andReturn(List.of(TEST_RESOURCE_GROUP.canonical(), ResourceGroupId.DEFAULT.canonical())) diff --git a/core/src/test/java/org/apache/accumulo/core/metadata/SuspendingTServerTest.java b/core/src/test/java/org/apache/accumulo/core/metadata/SuspendingTServerTest.java index 7826915ed57..bd1ea435789 100644 --- a/core/src/test/java/org/apache/accumulo/core/metadata/SuspendingTServerTest.java +++ b/core/src/test/java/org/apache/accumulo/core/metadata/SuspendingTServerTest.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.concurrent.TimeUnit; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.util.time.SteadyTime; import org.junit.jupiter.api.Test; @@ -33,14 +34,16 @@ public class SuspendingTServerTest { @Test public void testToFromValue() { SteadyTime suspensionTime = SteadyTime.from(System.currentTimeMillis(), TimeUnit.MILLISECONDS); - TServerInstance ser1 = new TServerInstance(HostAndPort.fromParts("server1", 8555), "s001"); + TServerInstance ser1 = new TServerInstance(ServerIdUtil.tserver("server1", 8555), "s001"); - var val1 = SuspendingTServer.toValue(ser1, suspensionTime); + var val1 = new SuspendingTServer(ser1, suspensionTime).toValue(); var st1 = SuspendingTServer.fromValue(val1); - assertEquals(HostAndPort.fromParts("server1", 8555), st1.server); + assertEquals(HostAndPort.fromParts("server1", 8555), + HostAndPort.fromParts(st1.server.getServer().getHost(), st1.server.getServer().getPort())); assertEquals(suspensionTime, st1.suspensionTime); assertEquals(val1, st1.toValue()); - var st2 = new SuspendingTServer(HostAndPort.fromParts("server1", 8555), suspensionTime); + var st2 = new SuspendingTServer( + new TServerInstance(ServerIdUtil.tserver("server1", 8555), "s001"), suspensionTime); assertEquals(st1, st2); assertEquals(st1.hashCode(), st2.hashCode()); assertEquals(st1.toString(), st2.toString()); @@ -48,11 +51,14 @@ public void testToFromValue() { // Create three SuspendingTServer objs that differ in one field. Ensure each field is considered // in equality checks. - var st3 = new SuspendingTServer(HostAndPort.fromParts("server2", 8555), suspensionTime); - var st4 = new SuspendingTServer(HostAndPort.fromParts("server1", 9555), suspensionTime); + var st3 = new SuspendingTServer( + new TServerInstance(ServerIdUtil.tserver("server2", 8555), "s001"), suspensionTime); + var st4 = new SuspendingTServer( + new TServerInstance(ServerIdUtil.tserver("server1", 9555), "s001"), suspensionTime); SteadyTime suspensionTime2 = SteadyTime.from(System.currentTimeMillis() + 100, TimeUnit.MILLISECONDS); - var st5 = new SuspendingTServer(HostAndPort.fromParts("server1", 8555), suspensionTime2); + var st5 = new SuspendingTServer( + new TServerInstance(ServerIdUtil.tserver("server2", 8555), "s001"), suspensionTime2); for (var stne : List.of(st3, st4, st5)) { assertNotEquals(st1, stne); assertNotEquals(st1.toValue(), stne.toValue()); diff --git 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 index b56315d8d70..c09037cf3e9 100644 --- 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 @@ -81,6 +81,7 @@ import org.apache.accumulo.core.client.admin.TabletAvailability; import org.apache.accumulo.core.client.admin.TimeType; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.clientImpl.TabletAvailabilityUtil; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; @@ -134,6 +135,9 @@ public void testAllColumns() { FateId fateId1 = FateId.from(type, UUID.randomUUID()); FateId fateId2 = FateId.from(type, UUID.randomUUID()); + TServerInstance current = new TServerInstance(ServerIdUtil.tserver("server1", 8555), "s001"); + TServerInstance last = new TServerInstance(ServerIdUtil.tserver("server2", 8555), "s000"); + mutation.put(MetadataSchema.TabletsSection.CompactedColumnFamily.STR_NAME, fateId1.canonical(), ""); @@ -167,13 +171,14 @@ public void testAllColumns() { 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(CurrentLocationColumnFamily.NAME).qualifier("s001") + .put(current.serialize()); - mutation.at().family(LastLocationColumnFamily.NAME).qualifier("s000").put("server2:8555"); + mutation.at().family(LastLocationColumnFamily.NAME).qualifier("s000").put(last.serialize()); - LogEntry le1 = LogEntry.fromPath("localhost+8020/" + UUID.randomUUID()); + LogEntry le1 = LogEntry.fromPath("default+localhost+8020/" + UUID.randomUUID()); le1.addToMutation(mutation); - LogEntry le2 = LogEntry.fromPath("localhost+8020/" + UUID.randomUUID()); + LogEntry le2 = LogEntry.fromPath("default+localhost+8020/" + UUID.randomUUID()); le2.addToMutation(mutation); StoredTabletFile sf1 = StoredTabletFile.of(new Path("hdfs://nn1/acc/tables/1/t-0001/sf1.rf")); @@ -189,10 +194,8 @@ public void testAllColumns() { SplitColumnFamily.UNSPLITTABLE_COLUMN.put(mutation, new Value(unsplittableMeta.toBase64())); SteadyTime suspensionTime = SteadyTime.from(1000L, TimeUnit.MILLISECONDS); - TServerInstance ser1 = new TServerInstance(HostAndPort.fromParts("server1", 8555), "s001"); - SuspendingTServer suspendingTServer = - new SuspendingTServer(HostAndPort.fromParts("server1", 8555), suspensionTime); - Value suspend = SuspendingTServer.toValue(ser1, suspensionTime); + SuspendingTServer suspendingTServer = new SuspendingTServer(current, suspensionTime); + Value suspend = suspendingTServer.toValue(); SUSPEND_COLUMN.put(mutation, suspend); FLUSH_NONCE_COLUMN.put(mutation, new Value(Long.toHexString(10L))); @@ -207,9 +210,9 @@ public void testAllColumns() { ResourceGroupId.of("Q1"), true, FateId.from(FateInstanceType.USER, UUID.randomUUID())); mutation.put(ExternalCompactionColumnFamily.STR_NAME, ecid.canonical(), ecMeta.toJson()); - TServerInstance tsi = new TServerInstance("localhost:9997", 5000L); + TServerInstance tsi = new TServerInstance(ServerIdUtil.tserver("localhost", 9997), 5000L); - MIGRATION_COLUMN.put(mutation, new Value(tsi.getHostPortSession())); + MIGRATION_COLUMN.put(mutation, new Value(tsi.serialize())); SortedMap rowMap = toRowMap(mutation); @@ -252,13 +255,17 @@ public void testAllColumns() { assertEquals(Map.of(new StoredTabletFile(bf1), fateId1, new StoredTabletFile(bf2), fateId2), tm.getLoaded()); allColumns.remove(LOADED); - assertEquals(HostAndPort.fromParts("server1", 8555), tm.getLocation().getHostAndPort()); + assertEquals(HostAndPort.fromParts("server1", 8555), + HostAndPort.fromParts(tm.getLocation().getServerInstance().getServer().getHost(), + tm.getLocation().getServerInstance().getServer().getPort())); allColumns.remove(LOCATION); - assertEquals("s001", tm.getLocation().getSession()); + assertEquals("s001", tm.getLocation().getServerInstance().getSession()); assertEquals(LocationType.CURRENT, tm.getLocation().getType()); assertTrue(tm.hasCurrent()); - assertEquals(HostAndPort.fromParts("server2", 8555), tm.getLast().getHostAndPort()); - assertEquals("s000", tm.getLast().getSession()); + assertEquals(HostAndPort.fromParts("server2", 8555), + HostAndPort.fromParts(tm.getLast().getServerInstance().getServer().getHost(), + tm.getLast().getServerInstance().getServer().getPort())); + assertEquals("s000", tm.getLast().getServerInstance().getSession()); allColumns.remove(LAST); assertEquals(LocationType.LAST, tm.getLast().getType()); assertEquals(Set.of(le1, le2), tm.getLogs().stream().collect(toSet())); @@ -292,8 +299,9 @@ public void testAllColumns() { public void testFuture() { KeyExtent extent = new KeyExtent(TableId.of("5"), new Text("df"), new Text("da")); + TServerInstance tsi = new TServerInstance(ServerIdUtil.tserver("server1", 8555), "s001"); Mutation mutation = TabletColumnFamily.createPrevRowMutation(extent); - mutation.at().family(FutureLocationColumnFamily.NAME).qualifier("s001").put("server1:8555"); + mutation.at().family(FutureLocationColumnFamily.NAME).qualifier("s001").put(tsi.serialize()); SortedMap rowMap = toRowMap(mutation); @@ -301,8 +309,10 @@ public void testFuture() { EnumSet.allOf(ColumnType.class), false, false); assertEquals(extent, tm.getExtent()); - assertEquals(HostAndPort.fromParts("server1", 8555), tm.getLocation().getHostAndPort()); - assertEquals("s001", tm.getLocation().getSession()); + assertEquals(HostAndPort.fromParts("server1", 8555), + HostAndPort.fromParts(tm.getLocation().getServerInstance().getServer().getHost(), + tm.getLocation().getServerInstance().getServer().getPort())); + assertEquals("s001", tm.getLocation().getServerInstance().getSession()); assertEquals(LocationType.FUTURE, tm.getLocation().getType()); assertFalse(tm.hasCurrent()); } @@ -311,9 +321,10 @@ public void testFuture() { public void testFutureAndCurrent() { KeyExtent extent = new KeyExtent(TableId.of("5"), new Text("df"), new Text("da")); + TServerInstance tsi = new TServerInstance(ServerIdUtil.tserver("server1", 8555), "s001"); Mutation mutation = TabletColumnFamily.createPrevRowMutation(extent); - mutation.at().family(CurrentLocationColumnFamily.NAME).qualifier("s001").put("server1:8555"); - mutation.at().family(FutureLocationColumnFamily.NAME).qualifier("s001").put("server1:8555"); + mutation.at().family(CurrentLocationColumnFamily.NAME).qualifier("s001").put(tsi.serialize()); + mutation.at().family(FutureLocationColumnFamily.NAME).qualifier("s001").put(tsi.serialize()); SortedMap rowMap = toRowMap(mutation); @@ -328,9 +339,9 @@ public void testFutureAndCurrent() { @Test public void testLocationStates() { KeyExtent extent = new KeyExtent(TableId.of("5"), new Text("df"), new Text("da")); - TServerInstance ser1 = new TServerInstance(HostAndPort.fromParts("server1", 8555), "s001"); - TServerInstance ser2 = new TServerInstance(HostAndPort.fromParts("server2", 8111), "s002"); - TServerInstance deadSer = new TServerInstance(HostAndPort.fromParts("server3", 8000), "s003"); + TServerInstance ser1 = new TServerInstance(ServerIdUtil.tserver("server1", 8555), "s001"); + TServerInstance ser2 = new TServerInstance(ServerIdUtil.tserver("server2", 8111), "s002"); + TServerInstance deadSer = new TServerInstance(ServerIdUtil.tserver("server3", 8000), "s003"); Set tservers = new LinkedHashSet<>(); tservers.add(ser1); tservers.add(ser2); @@ -340,7 +351,7 @@ public void testLocationStates() { // test assigned Mutation mutation = TabletColumnFamily.createPrevRowMutation(extent); mutation.at().family(FutureLocationColumnFamily.NAME).qualifier(ser1.getSession()) - .put(ser1.getHostPort()); + .put(ser1.serialize()); SortedMap rowMap = toRowMap(mutation); TabletMetadata tm = @@ -349,35 +360,35 @@ public void testLocationStates() { assertEquals(TabletState.ASSIGNED, state); assertEquals(ser1, tm.getLocation().getServerInstance()); - assertEquals(ser1.getSession(), tm.getLocation().getSession()); + assertEquals(ser1.getSession(), tm.getLocation().getServerInstance().getSession()); assertEquals(LocationType.FUTURE, tm.getLocation().getType()); assertFalse(tm.hasCurrent()); // test hosted mutation = TabletColumnFamily.createPrevRowMutation(extent); mutation.at().family(CurrentLocationColumnFamily.NAME).qualifier(ser2.getSession()) - .put(ser2.getHostPort()); + .put(ser2.serialize()); rowMap = toRowMap(mutation); tm = TabletMetadata.convertRow(rowMap.entrySet().iterator(), colsToFetch, false, false); assertEquals(TabletState.HOSTED, TabletState.compute(tm, tservers)); assertEquals(ser2, tm.getLocation().getServerInstance()); - assertEquals(ser2.getSession(), tm.getLocation().getSession()); + assertEquals(ser2.getSession(), tm.getLocation().getServerInstance().getSession()); assertEquals(LocationType.CURRENT, tm.getLocation().getType()); assertTrue(tm.hasCurrent()); // test ASSIGNED_TO_DEAD_SERVER mutation = TabletColumnFamily.createPrevRowMutation(extent); mutation.at().family(CurrentLocationColumnFamily.NAME).qualifier(deadSer.getSession()) - .put(deadSer.getHostPort()); + .put(deadSer.serialize()); rowMap = toRowMap(mutation); tm = TabletMetadata.convertRow(rowMap.entrySet().iterator(), colsToFetch, false, false); assertEquals(TabletState.ASSIGNED_TO_DEAD_SERVER, TabletState.compute(tm, tservers)); assertEquals(deadSer, tm.getLocation().getServerInstance()); - assertEquals(deadSer.getSession(), tm.getLocation().getSession()); + assertEquals(deadSer.getSession(), tm.getLocation().getServerInstance().getSession()); assertEquals(LocationType.CURRENT, tm.getLocation().getType()); assertTrue(tm.hasCurrent()); @@ -395,14 +406,14 @@ public void testLocationStates() { mutation = TabletColumnFamily.createPrevRowMutation(extent); mutation.at().family(SUSPEND_COLUMN.getColumnFamily()) .qualifier(SUSPEND_COLUMN.getColumnQualifier()) - .put(SuspendingTServer.toValue(ser2, SteadyTime.from(1000L, TimeUnit.MILLISECONDS))); + .put(new SuspendingTServer(ser2, SteadyTime.from(1000L, TimeUnit.MILLISECONDS)).toValue()); rowMap = toRowMap(mutation); tm = TabletMetadata.convertRow(rowMap.entrySet().iterator(), colsToFetch, false, false); assertEquals(TabletState.SUSPENDED, TabletState.compute(tm, tservers)); assertEquals(1000L, tm.getSuspend().suspensionTime.getMillis()); - assertEquals(ser2.getHostAndPort(), tm.getSuspend().server); + assertEquals(ser2.getServer(), tm.getSuspend().server.getServer()); assertNull(tm.getLocation()); assertFalse(tm.hasCurrent()); } @@ -516,7 +527,7 @@ public void testTmBuilderImmutable() { () -> tm.getFilesMap().put(stf, new DataFileValue(0, 0, 0))); assertTrue(tm.getLogs().isEmpty()); assertThrows(UnsupportedOperationException.class, - () -> tm.getLogs().add(LogEntry.fromPath("localhost+8020/" + UUID.randomUUID()))); + () -> tm.getLogs().add(LogEntry.fromPath("default+localhost+8020/" + UUID.randomUUID()))); assertTrue(tm.getScans().isEmpty()); assertThrows(UnsupportedOperationException.class, () -> tm.getScans().add(stf)); assertTrue(tm.getLoaded().isEmpty()); @@ -534,7 +545,7 @@ public void testTmBuilderImmutable() { b.table(TableId.of("4")); b.extCompaction(ecid, ecMeta); b.file(stf, new DataFileValue(0, 0, 0)); - b.log(LogEntry.fromPath("localhost+8020/" + UUID.randomUUID())); + b.log(LogEntry.fromPath("default+localhost+8020/" + UUID.randomUUID())); b.scan(stf); b.loadedFile(stf, FateId.from(FateInstanceType.USER, UUID.randomUUID())); b.compacted(FateId.from(FateInstanceType.USER, UUID.randomUUID())); @@ -553,7 +564,7 @@ public void testTmBuilderImmutable() { () -> tm2.getFilesMap().put(stf, new DataFileValue(0, 0, 0))); assertEquals(1, tm2.getLogs().size()); assertThrows(UnsupportedOperationException.class, - () -> tm2.getLogs().add(LogEntry.fromPath("localhost+8020/" + UUID.randomUUID()))); + () -> tm2.getLogs().add(LogEntry.fromPath("default+localhost+8020/" + UUID.randomUUID()))); assertEquals(1, tm2.getScans().size()); assertThrows(UnsupportedOperationException.class, () -> tm2.getScans().add(stf)); assertEquals(1, tm2.getLoaded().size()); @@ -734,7 +745,7 @@ private SortedMap toRowMap(Mutation mutation) { @Test public void testBuilder() { - TServerInstance ser1 = new TServerInstance(HostAndPort.fromParts("server1", 8555), "s001"); + TServerInstance ser1 = new TServerInstance(ServerIdUtil.tserver("server1", 8555), "s001"); KeyExtent extent = new KeyExtent(TableId.of("5"), new Text("df"), new Text("da")); @@ -763,7 +774,7 @@ public void testBuilder() { FateId compactFateId1 = FateId.from(type, UUID.randomUUID()); FateId compactFateId2 = FateId.from(type, UUID.randomUUID()); - TServerInstance migration = new TServerInstance("localhost:9999", 1000L); + TServerInstance migration = new TServerInstance(ServerIdUtil.tserver("localhost", 9999), 1000L); TabletMetadata tm = TabletMetadata.builder(extent) .putTabletAvailability(TabletAvailability.UNHOSTED).putLocation(Location.future(ser1)) @@ -829,8 +840,8 @@ public void testBuilder() { new CompactionMetadata(Set.of(sf1, sf2), rf1, "cid1", CompactionKind.USER, (short) 3, ResourceGroupId.of("Q1"), true, FateId.from(type, UUID.randomUUID())); - LogEntry le1 = LogEntry.fromPath("localhost+8020/" + UUID.randomUUID()); - LogEntry le2 = LogEntry.fromPath("localhost+8020/" + UUID.randomUUID()); + LogEntry le1 = LogEntry.fromPath("default+localhost+8020/" + UUID.randomUUID()); + LogEntry le2 = LogEntry.fromPath("default+localhost+8020/" + UUID.randomUUID()); FateId selFilesFateId = FateId.from(type, UUID.randomUUID()); SelectedFiles selFiles = new SelectedFiles(Set.of(sf1, sf4), false, selFilesFateId, @@ -849,7 +860,8 @@ public void testBuilder() { assertEquals(Set.of(ecid1), tm3.getExternalCompactions().keySet()); assertEquals(Set.of(sf1, sf2), tm3.getExternalCompactions().get(ecid1).getJobFiles()); - assertEquals(ser1.getHostAndPort(), tm3.getSuspend().server); + assertEquals(ser1.getServer().toHostPortString(), + tm3.getSuspend().server.getServer().toHostPortString()); assertEquals(SteadyTime.from(45L, TimeUnit.MILLISECONDS), tm3.getSuspend().suspensionTime); assertEquals(new MetadataTime(479, TimeType.LOGICAL), tm3.getTime()); assertTrue(tm3.getHostingRequested()); diff --git a/core/src/test/java/org/apache/accumulo/core/spi/balancer/BaseHostRegexTableLoadBalancerTest.java b/core/src/test/java/org/apache/accumulo/core/spi/balancer/BaseHostRegexTableLoadBalancerTest.java index 5e4408ace5b..23272667a80 100644 --- a/core/src/test/java/org/apache/accumulo/core/spi/balancer/BaseHostRegexTableLoadBalancerTest.java +++ b/core/src/test/java/org/apache/accumulo/core/spi/balancer/BaseHostRegexTableLoadBalancerTest.java @@ -203,7 +203,7 @@ public List getOnlineTabletsForTable(TabletServerId tserver, protected boolean tabletInBounds(TabletId tabletId, TabletServerId tsi) { String tid = tabletId.getTable().canonical(); - String host = tsi.getHost(); + String host = tsi.getServer().getHost(); if (tid.equals("1") && (host.equals("192.168.0.1") || host.equals("192.168.0.2") || host.equals("192.168.0.3") || host.equals("192.168.0.4") || host.equals("192.168.0.5"))) { diff --git a/core/src/test/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancerReconfigurationTest.java b/core/src/test/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancerReconfigurationTest.java index 89ef8c4c782..d526d4b05b1 100644 --- a/core/src/test/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancerReconfigurationTest.java +++ b/core/src/test/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancerReconfigurationTest.java @@ -102,7 +102,7 @@ public void testConfigurationChanges() { // Ensure assignments are correct for (Entry e : assignments.entrySet()) { if (!tabletInBounds(e.getKey(), e.getValue())) { - fail("tablet not in bounds: " + e.getKey() + " -> " + e.getValue().getHost()); + fail("tablet not in bounds: " + e.getKey() + " -> " + e.getValue().getServer().getHost()); } } Set migrations = new HashSet<>(); @@ -128,11 +128,11 @@ public void testConfigurationChanges() { DataLevel.USER, tables)); assertEquals(5, migrationsOut.size()); for (TabletMigration migration : migrationsOut) { - assertTrue(migration.getNewTabletServer().getHost().startsWith("192.168.0.1") - || migration.getNewTabletServer().getHost().startsWith("192.168.0.2") - || migration.getNewTabletServer().getHost().startsWith("192.168.0.3") - || migration.getNewTabletServer().getHost().startsWith("192.168.0.4") - || migration.getNewTabletServer().getHost().startsWith("192.168.0.5")); + assertTrue(migration.getNewTabletServer().getServer().getHost().startsWith("192.168.0.1") + || migration.getNewTabletServer().getServer().getHost().startsWith("192.168.0.2") + || migration.getNewTabletServer().getServer().getHost().startsWith("192.168.0.3") + || migration.getNewTabletServer().getServer().getHost().startsWith("192.168.0.4") + || migration.getNewTabletServer().getServer().getHost().startsWith("192.168.0.5")); } } diff --git a/core/src/test/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancerTest.java b/core/src/test/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancerTest.java index 4facdeda5c8..38ab3c16945 100644 --- a/core/src/test/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancerTest.java +++ b/core/src/test/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancerTest.java @@ -539,7 +539,7 @@ public void testAllUnassigned() { // Ensure assignments are correct for (Entry e : assignments.entrySet()) { if (!tabletInBounds(e.getKey(), e.getValue())) { - fail("tablet not in bounds: " + e.getKey() + " -> " + e.getValue().getHost()); + fail("tablet not in bounds: " + e.getKey() + " -> " + e.getValue().getServer().getHost()); } } } @@ -587,7 +587,7 @@ public void testPartiallyAssigned() { // Ensure assignments are correct for (Entry e : assignments.entrySet()) { if (!tabletInBounds(e.getKey(), e.getValue())) { - fail("tablet not in bounds: " + e.getKey() + " -> " + e.getValue().getHost()); + fail("tablet not in bounds: " + e.getKey() + " -> " + e.getValue().getServer().getHost()); } } } @@ -604,10 +604,11 @@ public void testUnassignedWithNoTServers() { // Remove the BAR tablet servers from current List removals = new ArrayList<>(); for (Entry e : current.entrySet()) { - if (e.getKey().getHost().equals("192.168.0.6") || e.getKey().getHost().equals("192.168.0.7") - || e.getKey().getHost().equals("192.168.0.8") - || e.getKey().getHost().equals("192.168.0.9") - || e.getKey().getHost().equals("192.168.0.10")) { + if (e.getKey().getServer().getHost().equals("192.168.0.6") + || e.getKey().getServer().getHost().equals("192.168.0.7") + || e.getKey().getServer().getHost().equals("192.168.0.8") + || e.getKey().getServer().getHost().equals("192.168.0.9") + || e.getKey().getServer().getHost().equals("192.168.0.10")) { removals.add(e.getKey()); } } @@ -622,7 +623,8 @@ public void testUnassignedWithNoTServers() { // Ensure tablets are assigned in default pool for (Entry e : assignments.entrySet()) { if (tabletInBounds(e.getKey(), e.getValue())) { - fail("tablet unexpectedly in bounds: " + e.getKey() + " -> " + e.getValue().getHost()); + fail("tablet unexpectedly in bounds: " + e.getKey() + " -> " + + e.getValue().getServer().getHost()); } } } @@ -640,15 +642,16 @@ public void testUnassignedWithNoDefaultPool() { // Remove the BAR tablet servers and default pool from current List removals = new ArrayList<>(); for (Entry e : current.entrySet()) { - if (e.getKey().getHost().equals("192.168.0.6") || e.getKey().getHost().equals("192.168.0.7") - || e.getKey().getHost().equals("192.168.0.8") - || e.getKey().getHost().equals("192.168.0.9") - || e.getKey().getHost().equals("192.168.0.10") - || e.getKey().getHost().equals("192.168.0.11") - || e.getKey().getHost().equals("192.168.0.12") - || e.getKey().getHost().equals("192.168.0.13") - || e.getKey().getHost().equals("192.168.0.14") - || e.getKey().getHost().equals("192.168.0.15")) { + if (e.getKey().getServer().getHost().equals("192.168.0.6") + || e.getKey().getServer().getHost().equals("192.168.0.7") + || e.getKey().getServer().getHost().equals("192.168.0.8") + || e.getKey().getServer().getHost().equals("192.168.0.9") + || e.getKey().getServer().getHost().equals("192.168.0.10") + || e.getKey().getServer().getHost().equals("192.168.0.11") + || e.getKey().getServer().getHost().equals("192.168.0.12") + || e.getKey().getServer().getHost().equals("192.168.0.13") + || e.getKey().getServer().getHost().equals("192.168.0.14") + || e.getKey().getServer().getHost().equals("192.168.0.15")) { removals.add(e.getKey()); } } @@ -665,7 +668,8 @@ public void testUnassignedWithNoDefaultPool() { // Ensure tablets are assigned in default pool for (Entry e : assignments.entrySet()) { if (tabletInBounds(e.getKey(), e.getValue())) { - fail("tablet unexpectedly in bounds: " + e.getKey() + " -> " + e.getValue().getHost()); + fail("tablet unexpectedly in bounds: " + e.getKey() + " -> " + + e.getValue().getServer().getHost()); } } } @@ -690,7 +694,7 @@ public void testOutOfBoundsTablets() { public List getOnlineTabletsForTable(TabletServerId tserver, TableId tableId) { // Report incorrect information so that balance will create an assignment List tablets = new ArrayList<>(); - if (tableId.equals(BAR.getId()) && tserver.getHost().equals("192.168.0.1")) { + if (tableId.equals(BAR.getId()) && tserver.getServer().getHost().equals("192.168.0.1")) { // Report that we have a bar tablet on this server TKeyExtent tke = new TKeyExtent(); tke.setTable(BAR.getId().canonical().getBytes(UTF_8)); @@ -700,7 +704,7 @@ public List getOnlineTabletsForTable(TabletServerId tserver, T tstats.setExtent(tke); TabletStatistics ts = new TabletStatisticsImpl(tstats); tablets.add(ts); - } else if (tableId.equals(FOO.getId()) && tserver.getHost().equals("192.168.0.6")) { + } else if (tableId.equals(FOO.getId()) && tserver.getServer().getHost().equals("192.168.0.6")) { // Report that we have a foo tablet on this server TKeyExtent tke = new TKeyExtent(); tke.setTable(FOO.getId().canonical().getBytes(UTF_8)); diff --git a/core/src/test/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancerTest.java b/core/src/test/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancerTest.java index d7bbbe8f866..659e8b6d83e 100644 --- a/core/src/test/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancerTest.java +++ b/core/src/test/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancerTest.java @@ -181,7 +181,7 @@ SortedMap getAssignments(Map generateFakeTablets(TabletServerId tserver, TableI for (int i = 0; i < tableInfo.getTableMap().get(tableId.canonical()).getOnlineTabletCount(); i++) { TabletStats stats = new TabletStats(); - stats.extent = - new KeyExtent(tableId, new Text(tserver.getHost() + String.format("%03d", i + 1)), - new Text(tserver.getHost() + String.format("%03d", i))).toThrift(); + stats.extent = new KeyExtent(tableId, + new Text(tserver.getServer().getHost() + String.format("%03d", i + 1)), + new Text(tserver.getServer().getHost() + String.format("%03d", i))).toThrift(); result.add(new TabletStatisticsImpl(stats)); } return result; diff --git a/core/src/test/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerHostSelectorTest.java b/core/src/test/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerHostSelectorTest.java index 69753a5d98a..cf17bb5997d 100644 --- a/core/src/test/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerHostSelectorTest.java +++ b/core/src/test/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerHostSelectorTest.java @@ -31,6 +31,8 @@ import java.util.Map; import java.util.Set; +import org.apache.accumulo.core.client.admin.servers.ServerId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.TabletId; import org.apache.accumulo.core.spi.scan.ConfigurableScanServerSelectorTest.InitParams; import org.apache.accumulo.core.spi.scan.ConfigurableScanServerSelectorTest.SelectorParams; @@ -38,23 +40,21 @@ import org.apache.accumulo.core.spi.scan.ScanServerAttempt.Result; import org.junit.jupiter.api.Test; -import com.google.common.net.HostAndPort; - public class ConfigurableScanServerHostSelectorTest { - private final String ss1 = "host1:2000"; - private final String ss2 = "host1:2001"; - private final String ss3 = "host1:2002"; - private final String ss4 = "host1:2003"; - private final String ss5 = "host2:2000"; - private final String ss6 = "host2:2001"; - private final String ss7 = "host2:2002"; - private final String ss8 = "host2:2003"; - private final String ss9 = "host3:2000"; + private final ServerId ss1 = ServerIdUtil.sserver("host1", 2000); + private final ServerId ss2 = ServerIdUtil.sserver("host1", 2001); + private final ServerId ss3 = ServerIdUtil.sserver("host1", 2002); + private final ServerId ss4 = ServerIdUtil.sserver("host1", 2003); + private final ServerId ss5 = ServerIdUtil.sserver("host2", 2000); + private final ServerId ss6 = ServerIdUtil.sserver("host2", 2001); + private final ServerId ss7 = ServerIdUtil.sserver("host2", 2002); + private final ServerId ss8 = ServerIdUtil.sserver("host2", 2003); + private final ServerId ss9 = ServerIdUtil.sserver("host3", 2000); - private final Set host1Servers = Set.of(ss1, ss2, ss3, ss4); - private final Set host2Servers = Set.of(ss5, ss6, ss7, ss8); - private final Set host3Servers = Set.of(ss9); + private final Set host1Servers = Set.of(ss1, ss2, ss3, ss4); + private final Set host2Servers = Set.of(ss5, ss6, ss7, ss8); + private final Set host3Servers = Set.of(ss9); @Test public void test() { @@ -66,15 +66,17 @@ public void test() { final TabletId tId = ConfigurableScanServerSelectorTest.nti("1", "m"); final ConfigurableScanServerHostSelector selector = new ConfigurableScanServerHostSelector(); - selector.init(new InitParams(Set.of(ss1, ss2, ss3, ss4, ss5, ss6, ss7, ss8, ss9))); + selector.init(new InitParams( + Set.of(ss1.toHostPortString(), ss2.toHostPortString(), ss3.toHostPortString(), + ss4.toHostPortString(), ss5.toHostPortString(), ss6.toHostPortString(), + ss7.toHostPortString(), ss8.toHostPortString(), ss9.toHostPortString()))); ScanServerSelections selection = selector.selectServers(new SelectorParams(tId)); assertNotNull(selection); - final String firstServer = selection.getScanServer(tId); + final ServerId firstServer = selection.getScanServer(tId); assertNotNull(firstServer); - final HostAndPort firstHP = HostAndPort.fromString(firstServer); - final Set remainingServers = new HashSet<>(); + final Set remainingServers = new HashSet<>(); if (host1Servers.contains(firstServer)) { remainingServers.addAll(host1Servers); firstHostSeen = true; @@ -93,18 +95,16 @@ public void test() { attempts.add(new TestScanServerAttempt(firstServer, Result.BUSY)); while (!remainingServers.isEmpty()) { selection = selector.selectServers(new SelectorParams(tId, Map.of(tId, attempts), Map.of())); - String selectedServer = selection.getScanServer(tId); - HostAndPort selectedHP = HostAndPort.fromString(selectedServer); - assertEquals(selectedHP.getHost(), firstHP.getHost()); + ServerId selectedServer = selection.getScanServer(tId); + assertEquals(selectedServer.getHost(), firstServer.getHost()); assertTrue(remainingServers.remove(selectedServer)); attempts.add(new TestScanServerAttempt(selectedServer, Result.BUSY)); } // At this point we should have exhausted all of the scan servers on the first selected host selection = selector.selectServers(new SelectorParams(tId, Map.of(tId, attempts), Map.of())); - String secondServer = selection.getScanServer(tId); - final HostAndPort secondHP = HostAndPort.fromString(secondServer); - assertFalse(secondHP.getHost().equals(firstHP.getHost())); + ServerId secondServer = selection.getScanServer(tId); + assertFalse(secondServer.getHost().equals(firstServer.getHost())); if (host1Servers.contains(secondServer)) { if (firstHostSeen) { @@ -132,9 +132,8 @@ public void test() { attempts.add(new TestScanServerAttempt(secondServer, Result.BUSY)); while (!remainingServers.isEmpty()) { selection = selector.selectServers(new SelectorParams(tId, Map.of(tId, attempts), Map.of())); - String selectedServer = selection.getScanServer(tId); - HostAndPort selectedHP = HostAndPort.fromString(selectedServer); - assertEquals(selectedHP.getHost(), secondHP.getHost()); + ServerId selectedServer = selection.getScanServer(tId); + assertEquals(selectedServer.getHost(), secondServer.getHost()); assertTrue(remainingServers.remove(selectedServer)); attempts.add(new TestScanServerAttempt(selectedServer, Result.BUSY)); } @@ -142,10 +141,9 @@ public void test() { // At this point we should have exhausted all of the scan servers on the first and second // selected host selection = selector.selectServers(new SelectorParams(tId, Map.of(tId, attempts), Map.of())); - String thirdServer = selection.getScanServer(tId); - final HostAndPort thirdHP = HostAndPort.fromString(thirdServer); - assertFalse(thirdHP.getHost().equals(firstHP.getHost())); - assertFalse(thirdHP.getHost().equals(secondHP.getHost())); + ServerId thirdServer = selection.getScanServer(tId); + assertFalse(thirdServer.getHost().equals(firstServer.getHost())); + assertFalse(thirdServer.getHost().equals(secondServer.getHost())); if (host1Servers.contains(thirdServer)) { if (firstHostSeen) { @@ -173,9 +171,8 @@ public void test() { attempts.add(new TestScanServerAttempt(thirdServer, Result.BUSY)); while (!remainingServers.isEmpty()) { selection = selector.selectServers(new SelectorParams(tId, Map.of(tId, attempts), Map.of())); - String selectedServer = selection.getScanServer(tId); - HostAndPort selectedHP = HostAndPort.fromString(selectedServer); - assertEquals(selectedHP.getHost(), thirdHP.getHost()); + ServerId selectedServer = selection.getScanServer(tId); + assertEquals(selectedServer.getHost(), thirdServer.getHost()); assertTrue(remainingServers.remove(selectedServer)); attempts.add(new TestScanServerAttempt(selectedServer, Result.BUSY)); } diff --git a/core/src/test/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelectorTest.java b/core/src/test/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelectorTest.java index b77bd026bac..33810a70fe6 100644 --- a/core/src/test/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelectorTest.java +++ b/core/src/test/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelectorTest.java @@ -38,6 +38,8 @@ import java.util.stream.Stream; import org.apache.accumulo.core.client.TimedOutException; +import org.apache.accumulo.core.client.admin.servers.ServerId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.ResourceGroupId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.data.TabletId; @@ -49,13 +51,14 @@ import org.junit.jupiter.api.Test; import com.google.common.base.Preconditions; +import com.google.common.net.HostAndPort; public class ConfigurableScanServerSelectorTest { static class InitParams implements ScanServerSelector.InitParameters { private final Map opts; - private final Supplier> scanServers; + private final Supplier> scanServers; InitParams(Set scanServers) { this(scanServers, Map.of()); @@ -63,17 +66,22 @@ static class InitParams implements ScanServerSelector.InitParameters { InitParams(Set scanServers, Map opts) { this.opts = opts; - var scanServersMap = new HashMap(); - scanServers.forEach(sserv -> scanServersMap.put(sserv, ResourceGroupId.DEFAULT)); + var scanServersMap = new HashMap(); + scanServers.forEach(sserv -> { + var hp = HostAndPort.fromString(sserv); + scanServersMap.put( + ServerIdUtil.sserver(ResourceGroupId.DEFAULT, hp.getHost(), hp.getPort()), + ResourceGroupId.DEFAULT); + }); this.scanServers = () -> scanServersMap; } - InitParams(Map scanServers, Map opts) { + InitParams(Map scanServers, Map opts) { this.opts = opts; this.scanServers = () -> scanServers; } - InitParams(Supplier> scanServers, Map opts) { + InitParams(Supplier> scanServers, Map opts) { this.opts = opts; this.scanServers = scanServers; } @@ -93,15 +101,9 @@ public Supplier> getScanServers() { return () -> scanServers.get().entrySet().stream().map(entry -> new ScanServerInfo() { @Override - public String getAddress() { + public ServerId getServer() { return entry.getKey(); } - - @Override - public ResourceGroupId getGroup() { - return entry.getValue(); - } - }).collect(Collectors.toSet()); } } @@ -151,16 +153,22 @@ public Optional waitUntil(Supplier> condition, Duration maxWa static class TestScanServerAttempt implements ScanServerAttempt { - private final String server; + private final ServerId server; private final Result result; - TestScanServerAttempt(String server, Result result) { + TestScanServerAttempt(ServerId server, Result result) { this.server = server; this.result = result; } + TestScanServerAttempt(String server, Result result) { + var hp = HostAndPort.fromString(server); + this.server = ServerIdUtil.sserver(hp.getHost(), hp.getPort()); + this.result = result; + } + @Override - public String getServer() { + public ServerId getServer() { return server; } @@ -178,10 +186,10 @@ public static TabletId nti(String tableId, String endRow) { @Test public void testBasic() { ConfigurableScanServerSelector selector = new ConfigurableScanServerSelector(); - selector.init(new InitParams( - Set.of("ss1:1", "ss2:2", "ss3:3", "ss4:4", "ss5:5", "ss6:6", "ss7:7", "ss8:8"))); + selector.init(new InitParams(Set.of("ss1:1101", "ss2:1102", "ss3:1103", "ss4:1104", "ss5:1105", + "ss6:1106", "ss7:1107", "ss8:1108"))); - Set servers = new HashSet<>(); + Set servers = new HashSet<>(); for (int i = 0; i < 100; i++) { var tabletId = nti("1", "m"); @@ -208,17 +216,18 @@ private void runBusyTest(int numServers, int busyAttempts, int expectedServers, long expectedBusyTimeout, Map opts, Map hints) { ConfigurableScanServerSelector selector = new ConfigurableScanServerSelector(); - var servers = Stream.iterate(1, i -> i <= numServers, i -> i + 1).map(i -> "s" + i + ":" + i) - .collect(Collectors.toSet()); + var servers = Stream.iterate(1, i -> i <= numServers, i -> i + 1) + .map(i -> "s" + i + ":" + (1100 + i)).collect(Collectors.toSet()); selector.init(new InitParams(servers, opts)); - Set serversSeen = new HashSet<>(); + Set serversSeen = new HashSet<>(); var tabletId = nti("1", "m"); var tabletAttempts = Stream.iterate(1, i -> i <= busyAttempts, i -> i + 1) - .map(i -> (new TestScanServerAttempt("ss" + i + ":" + i, ScanServerAttempt.Result.BUSY))) + .map(i -> (new TestScanServerAttempt("ss" + i + ":" + (1100 + i), + ScanServerAttempt.Result.BUSY))) .collect(Collectors.toList()); Map> attempts = new HashMap<>(); @@ -268,14 +277,14 @@ public void testBusy() { @Test public void testCoverage() { ConfigurableScanServerSelector selector = new ConfigurableScanServerSelector(); - var servers = Stream.iterate(1, i -> i <= 20, i -> i + 1).map(i -> "s" + i + ":" + i) + var servers = Stream.iterate(1, i -> i <= 20, i -> i + 1).map(i -> "s" + i + ":" + (1100 + i)) .collect(Collectors.toSet()); selector.init(new InitParams(servers)); - Map allServersSeen = new HashMap<>(); + Map allServersSeen = new HashMap<>(); for (int t = 0; t < 10000; t++) { - Set serversSeen = new HashSet<>(); + Set serversSeen = new HashSet<>(); String endRow = Long.toString(Math.abs(Math.max(RANDOM.get().nextLong(), Long.MIN_VALUE + 1)), 36); @@ -301,18 +310,40 @@ public void testCoverage() { @Test public void testOpts() { - String defaultProfile = - "{'isDefault':true,'maxBusyTimeout':'5m','busyTimeoutMultiplier':4, 'attemptPlans':" - + "[{'servers':'5', 'busyTimeout':'5ms'},{'servers':'20', 'busyTimeout':'33ms'}," - + "{'servers':'50%', 'busyTimeout':'100ms'},{'servers':'100%', 'busyTimeout':'200ms'}]}"; - - String profile1 = "{'scanTypeActivations':['long','st9'],'maxBusyTimeout':'30m'," - + "'busyTimeoutMultiplier':4, 'attemptPlans':[{'servers':'2', 'busyTimeout':'10s'}," - + "{'servers':'4', 'busyTimeout':'2m'},{'servers':'10%', 'busyTimeout':'5m'}]}"; - - String profile2 = - "{'scanTypeActivations':['mega'],'maxBusyTimeout':'60m','busyTimeoutMultiplier':2, " - + "'attemptPlans':[{'servers':'100%', 'busyTimeout':'10m'}]}"; + String defaultProfile = """ + {'isDefault':true,'maxBusyTimeout':'5m', + 'busyTimeoutMultiplier':4, + 'attemptPlans': + [ + {'servers':'5', 'busyTimeout':'5ms'},{'servers':'20', 'busyTimeout':'33ms'}, + {'servers':'50%', 'busyTimeout':'100ms'},{'servers':'100%', 'busyTimeout':'200ms'} + ] + } + """; + + String profile1 = """ + {'scanTypeActivations':['long','st9'], + 'maxBusyTimeout':'30m', + 'busyTimeoutMultiplier':4, + 'attemptPlans': + [ + {'servers':'2', 'busyTimeout':'10s'}, + {'servers':'4', 'busyTimeout':'2m'}, + {'servers':'10%', 'busyTimeout':'5m'} + ] + } + """; + + String profile2 = """ + {'scanTypeActivations':['mega'], + 'maxBusyTimeout':'60m', + 'busyTimeoutMultiplier':2, + 'attemptPlans': + [ + {'servers':'100%', 'busyTimeout':'10m'} + ] + } + """; // Intentionally put the default profile in 2nd position. There was a bug where config parsing // would fail if the default did not come first. @@ -435,9 +466,13 @@ public void testGroups() { ConfigurableScanServerSelector selector = new ConfigurableScanServerSelector(); var dg = ResourceGroupId.DEFAULT; selector.init(new InitParams( - Map.of("ss1:1", dg, "ss2:2", dg, "ss3:3", dg, "ss4:4", ResourceGroupId.of("g1"), "ss5:5", - ResourceGroupId.of("g1"), "ss6:6", ResourceGroupId.of("g2"), "ss7:7", - ResourceGroupId.of("g2"), "ss8:8", ResourceGroupId.of("g2")), + Map.of(ServerIdUtil.sserver("ss1", 1101), dg, ServerIdUtil.sserver("ss2", 1102), dg, + ServerIdUtil.sserver("ss3", 1103), dg, + ServerIdUtil.sserver(ResourceGroupId.of("g1"), "ss4", 1104), ResourceGroupId.of("g1"), + ServerIdUtil.sserver(ResourceGroupId.of("g1"), "ss5", 1105), ResourceGroupId.of("g1"), + ServerIdUtil.sserver(ResourceGroupId.of("g2"), "ss6", 1106), ResourceGroupId.of("g2"), + ServerIdUtil.sserver(ResourceGroupId.of("g2"), "ss7", 1107), ResourceGroupId.of("g2"), + ServerIdUtil.sserver(ResourceGroupId.of("g2"), "ss8", 1108), ResourceGroupId.of("g2")), opts)); Set servers = new HashSet<>(); @@ -447,10 +482,10 @@ public void testGroups() { ScanServerSelections actions = selector.selectServers(new SelectorParams(tabletId)); - servers.add(actions.getScanServer(tabletId)); + servers.add(actions.getScanServer(tabletId).toHostPortString()); } - assertEquals(Set.of("ss1:1", "ss2:2", "ss3:3"), servers); + assertEquals(Set.of("ss1:1101", "ss2:1102", "ss3:1103"), servers); // config should map this scan type to the group of scan servers g1 var hints = Map.of("scan_type", "long"); @@ -463,10 +498,10 @@ public void testGroups() { ScanServerSelections actions = selector.selectServers(new SelectorParams(tabletId, Map.of(), hints)); - servers.add(actions.getScanServer(tabletId)); + servers.add(actions.getScanServer(tabletId).toHostPortString()); } - assertEquals(Set.of("ss4:4", "ss5:5"), servers); + assertEquals(Set.of("ss4:1104", "ss5:1105"), servers); // config should map this scan type to the group of scan servers g2 hints = Map.of("scan_type", "mega"); @@ -479,10 +514,10 @@ public void testGroups() { ScanServerSelections actions = selector.selectServers(new SelectorParams(tabletId, Map.of(), hints)); - servers.add(actions.getScanServer(tabletId)); + servers.add(actions.getScanServer(tabletId).toHostPortString()); } - assertEquals(Set.of("ss6:6", "ss7:7", "ss8:8"), servers); + assertEquals(Set.of("ss6:1106", "ss7:1107", "ss8:1108"), servers); // config does map this scan type to anything, so should use the default group of scan servers hints = Map.of("scan_type", "rust"); @@ -495,10 +530,10 @@ public void testGroups() { ScanServerSelections actions = selector.selectServers(new SelectorParams(tabletId, Map.of(), hints)); - servers.add(actions.getScanServer(tabletId)); + servers.add(actions.getScanServer(tabletId).toHostPortString()); } - assertEquals(Set.of("ss1:1", "ss2:2", "ss3:3"), servers); + assertEquals(Set.of("ss1:1101", "ss2:1102", "ss3:1103"), servers); } @Test @@ -514,7 +549,7 @@ public void testWaitForScanServers() { ConfigurableScanServerSelector selector = new ConfigurableScanServerSelector(); - AtomicReference> scanServers = new AtomicReference<>(Map.of()); + AtomicReference> scanServers = new AtomicReference<>(Map.of()); selector.init(new InitParams(scanServers::get, opts)); @@ -527,7 +562,8 @@ public void testWaitForScanServers() { public Optional waitUntil(Supplier> condition, Duration maxWaitTime, String description) { // make some scan servers available now that wait was called - scanServers.set(Map.of("ss1:1", dg, "ss2:2", dg, "ss3:3", dg)); + scanServers.set(Map.of(ServerIdUtil.sserver("ss1", 1101), dg, + ServerIdUtil.sserver("ss2", 1102), dg, ServerIdUtil.sserver("ss3", 1103), dg)); Optional optional = condition.get(); @@ -542,7 +578,8 @@ public Optional waitUntil(Supplier> condition, Duration maxWa ScanServerSelections actions = selector.selectServers(params); - assertTrue(Set.of("ss1:1", "ss2:2", "ss3:3").contains(actions.getScanServer(tabletId))); + assertTrue(Set.of(ServerIdUtil.sserver("ss1", 1101), ServerIdUtil.sserver("ss2", 1102), + ServerIdUtil.sserver("ss3", 1103)).contains(actions.getScanServer(tabletId))); assertFalse(scanServers.get().isEmpty()); } } diff --git a/core/src/test/java/org/apache/accumulo/core/tabletserver/log/LogEntryTest.java b/core/src/test/java/org/apache/accumulo/core/tabletserver/log/LogEntryTest.java index 579571d81de..a0bf94aea3e 100644 --- a/core/src/test/java/org/apache/accumulo/core/tabletserver/log/LogEntryTest.java +++ b/core/src/test/java/org/apache/accumulo/core/tabletserver/log/LogEntryTest.java @@ -28,16 +28,15 @@ import java.util.UUID; import java.util.stream.Stream; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily; import org.apache.hadoop.io.Text; import org.junit.jupiter.api.Test; -import com.google.common.net.HostAndPort; - public class LogEntryTest { - private final String validHost = "localhost+9997"; + private final String validHost = "default+localhost+9997"; private final UUID validUUID = UUID.randomUUID(); private final String validPath = "viewfs:/a/accumulo/wal/" + validHost + "/" + validUUID; @@ -64,7 +63,7 @@ public void testFromMetadata() { private void verifyLogEntry(LogEntry logEntry, Text expectedColumnQualifier) { assertEquals(validPath, logEntry.toString()); assertEquals(validPath, logEntry.getPath()); - assertEquals(HostAndPort.fromString(validHost.replace('+', ':')), logEntry.getTServer()); + assertEquals(ServerIdUtil.fromWalFileName(validHost), logEntry.getTServer()); assertEquals(expectedColumnQualifier, logEntry.getColumnQualifier()); assertEquals(validUUID, logEntry.getUniqueID()); } @@ -109,7 +108,8 @@ public void testInvalidHostPort() { Stream.of("default:9997", "default+badPort").forEach(badHostAndPort -> { var badPath = badHostAndPort + "/" + validUUID; var e = assertThrows(IllegalArgumentException.class, () -> LogEntry.fromPath(badPath)); - assertTrue(e.getMessage().contains("Expected: host+port. Found '" + badHostAndPort + "'")); + assertTrue( + e.getMessage().contains("Expected: group+host+port. Found '" + badHostAndPort + "'")); }); } diff --git a/core/src/test/java/org/apache/accumulo/core/util/AddressUtilTest.java b/core/src/test/java/org/apache/accumulo/core/util/AddressUtilTest.java index 4c1303aa08b..9add26510dd 100644 --- a/core/src/test/java/org/apache/accumulo/core/util/AddressUtilTest.java +++ b/core/src/test/java/org/apache/accumulo/core/util/AddressUtilTest.java @@ -28,8 +28,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.net.HostAndPort; - /** * Test the AddressUtil class. */ @@ -105,27 +103,4 @@ public void testGetNegativeTtlThrowsOnForever() { } - @Test - public void normalizeAddressRequirePortTest() { - HostAndPort hostAndPort = AddressUtil.parseAddress("127.0.1.2+8080"); - assertEquals("127.0.1.2", hostAndPort.getHost()); - assertEquals(8080, hostAndPort.getPort()); - - HostAndPort hostAndPort2 = AddressUtil.parseAddress("127.0.1.2:9123"); - assertEquals("127.0.1.2", hostAndPort2.getHost()); - assertEquals(9123, hostAndPort2.getPort()); - - assertThrows(IllegalArgumentException.class, () -> AddressUtil.parseAddress("127.0.1.2")); - } - - @Test - public void normalizeAddressWithDefaultTest() { - HostAndPort hostAndPort = AddressUtil.parseAddress("127.0.1.2+8080", 9123); - assertEquals("127.0.1.2", hostAndPort.getHost()); - assertEquals(8080, hostAndPort.getPort()); - - HostAndPort hostAndPort2 = AddressUtil.parseAddress("127.0.1.2", 9123); - assertEquals("127.0.1.2", hostAndPort2.getHost()); - assertEquals(9123, hostAndPort2.getPort()); - } } diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java index ab9797879e2..017738cc677 100644 --- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java +++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java @@ -47,6 +47,7 @@ import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.TableOfflineException; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.client.sample.SamplerConfiguration; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.clientImpl.ClientTabletCache; @@ -54,6 +55,7 @@ import org.apache.accumulo.core.clientImpl.ClientTabletCache.LocationNeed; import org.apache.accumulo.core.clientImpl.OfflineScanner; import org.apache.accumulo.core.clientImpl.ScannerImpl; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.TableId; @@ -281,7 +283,7 @@ public float getProgress() { protected Key currentKey = null; - private static Map>> binOfflineTable(JobConf job, + private static Map>> binOfflineTable(JobConf job, TableId tableId, List ranges, Class callingClass) throws TableNotFoundException, AccumuloException { try (AccumuloClient client = createClient(job, callingClass)) { @@ -335,7 +337,7 @@ public static InputSplit[] getSplits(JobConf job, Class callingClass) throws } // get the metadata information for these ranges - Map>> binnedRanges = new HashMap<>(); + Map>> binnedRanges = new HashMap<>(); ClientTabletCache tl; try { if (tableConfig.isOfflineScan()) { @@ -365,11 +367,12 @@ public static InputSplit[] getSplits(JobConf job, Class callingClass) throws tl.invalidateCache(); } } else { - Map>> unhostedRanges = new HashMap<>(); - unhostedRanges.put("", new HashMap<>()); + final ServerId unhostedKey = ServerIdUtil.tserver("", 0); + Map>> unhostedRanges = new HashMap<>(); + unhostedRanges.put(unhostedKey, new HashMap<>()); BiConsumer consumer = (ct, r) -> { - unhostedRanges.get("").computeIfAbsent(ct.getExtent(), k -> new ArrayList<>()) - .add(r); + unhostedRanges.get(unhostedKey) + .computeIfAbsent(ct.getExtent(), k -> new ArrayList<>()).add(r); }; List failures = tl.findTablets(context, ranges, consumer, LocationNeed.NOT_REQUIRED); @@ -389,7 +392,7 @@ public static InputSplit[] getSplits(JobConf job, Class callingClass) throws } catch (InterruptedException e) { throw new RuntimeException(e); } - unhostedRanges.get("").clear(); + unhostedRanges.get(unhostedKey).clear(); tl.invalidateCache(); failures = tl.findTablets(context, ranges, consumer, LocationNeed.NOT_REQUIRED); } @@ -407,14 +410,14 @@ public static InputSplit[] getSplits(JobConf job, Class callingClass) throws splitsToAdd = new HashMap<>(); } - HashMap hostNameCache = new HashMap<>(); - for (Map.Entry>> tserverBin : binnedRanges.entrySet()) { - String ip = tserverBin.getKey().split(":", 2)[0]; - String location = hostNameCache.get(ip); + HashMap hostNameCache = new HashMap<>(); + for (Map.Entry>> tserverBin : binnedRanges.entrySet()) { + ServerId server = tserverBin.getKey(); + String location = hostNameCache.get(server); if (location == null) { - InetAddress inetAddress = InetAddress.getByName(ip); + InetAddress inetAddress = InetAddress.getByName(server.getHost()); location = inetAddress.getCanonicalHostName(); - hostNameCache.put(ip, location); + hostNameCache.put(server, location); } for (Map.Entry> extentRanges : tserverBin.getValue().entrySet()) { Range ke = extentRanges.getKey().toDataRange(); diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java index 19c03d808f5..41d491b5991 100644 --- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java +++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java @@ -47,6 +47,7 @@ import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.TableOfflineException; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.client.sample.SamplerConfiguration; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.clientImpl.ClientTabletCache; @@ -54,6 +55,7 @@ import org.apache.accumulo.core.clientImpl.ClientTabletCache.LocationNeed; import org.apache.accumulo.core.clientImpl.OfflineScanner; import org.apache.accumulo.core.clientImpl.ScannerImpl; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.TableId; @@ -316,7 +318,7 @@ private static void validateOptions(JobContext context, Class callingClass) } } - private static Map>> binOfflineTable(JobContext context, + private static Map>> binOfflineTable(JobContext context, TableId tableId, List ranges, Class callingClass) throws TableNotFoundException, AccumuloException { try (AccumuloClient client = createClient(context, callingClass)) { @@ -367,7 +369,7 @@ public static List getSplits(JobContext context, Class callingCla } // get the metadata information for these ranges - Map>> binnedRanges = new HashMap<>(); + Map>> binnedRanges = new HashMap<>(); ClientTabletCache tl; try { if (tableConfig.isOfflineScan()) { @@ -398,11 +400,12 @@ public static List getSplits(JobContext context, Class callingCla tl.invalidateCache(); } } else { - Map>> unhostedRanges = new HashMap<>(); - unhostedRanges.put("", new HashMap<>()); + final ServerId unhostedKey = ServerIdUtil.tserver("", 0); + Map>> unhostedRanges = new HashMap<>(); + unhostedRanges.put(unhostedKey, new HashMap<>()); BiConsumer consumer = (ct, r) -> { - unhostedRanges.get("").computeIfAbsent(ct.getExtent(), k -> new ArrayList<>()) - .add(r); + unhostedRanges.get(unhostedKey) + .computeIfAbsent(ct.getExtent(), k -> new ArrayList<>()).add(r); }; List failures = tl.findTablets(clientContext, ranges, consumer, LocationNeed.NOT_REQUIRED); @@ -422,7 +425,7 @@ public static List getSplits(JobContext context, Class callingCla } catch (InterruptedException e) { throw new RuntimeException(e); } - unhostedRanges.get("").clear(); + unhostedRanges.get(unhostedKey).clear(); tl.invalidateCache(); failures = tl.findTablets(clientContext, ranges, consumer, LocationNeed.NOT_REQUIRED); @@ -444,14 +447,14 @@ public static List getSplits(JobContext context, Class callingCla splitsToAdd = new HashMap<>(); } - HashMap hostNameCache = new HashMap<>(); - for (Map.Entry>> tserverBin : binnedRanges.entrySet()) { - String ip = tserverBin.getKey().split(":", 2)[0]; - String location = hostNameCache.get(ip); + HashMap hostNameCache = new HashMap<>(); + for (Map.Entry>> tserverBin : binnedRanges.entrySet()) { + ServerId server = tserverBin.getKey(); + String location = hostNameCache.get(server); if (location == null) { - InetAddress inetAddress = InetAddress.getByName(ip); + InetAddress inetAddress = InetAddress.getByName(server.getHost()); location = inetAddress.getCanonicalHostName(); - hostNameCache.put(ip, location); + hostNameCache.put(server, location); } for (Map.Entry> extentRanges : tserverBin.getValue().entrySet()) { Range ke = extentRanges.getKey().toDataRange(); diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java index 6caff916385..52b92d9d9a3 100644 --- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java +++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java @@ -51,6 +51,7 @@ import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel; import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.client.sample.SamplerConfiguration; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.clientImpl.ClientTabletCache; @@ -64,10 +65,12 @@ import org.apache.accumulo.core.iterators.SortedKeyValueIterator; import org.apache.accumulo.core.manager.state.tables.TableState; import org.apache.accumulo.core.metadata.SystemTables; +import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LastLocationColumnFamily; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily; +import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location; import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl; import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.security.NamespacePermission; @@ -832,9 +835,9 @@ public static void validatePermissions(Class implementingClass, Configuration return null; } - public static Map>> binOffline(TableId tableId, + public static Map>> binOffline(TableId tableId, List ranges, ClientContext context) throws AccumuloException, TableNotFoundException { - Map>> binnedRanges = new HashMap<>(); + Map>> binnedRanges = new HashMap<>(); if (context.getTableState(tableId) != TableState.OFFLINE) { context.clearTableListCache(); @@ -867,21 +870,21 @@ public static Map>> binOffline(TableId tableId, KeyExtent lastExtent = null; while (rowIter.hasNext()) { Iterator> row = rowIter.next(); - String last = ""; + Location last = null; KeyExtent extent = null; - String location = null; + Location location = null; while (row.hasNext()) { Map.Entry entry = row.next(); Key key = entry.getKey(); if (key.getColumnFamily().equals(LastLocationColumnFamily.NAME)) { - last = entry.getValue().toString(); + last = Location.last(TServerInstance.deserialize(entry.getValue().toString())); } if (key.getColumnFamily().equals(CurrentLocationColumnFamily.NAME) || key.getColumnFamily().equals(FutureLocationColumnFamily.NAME)) { - location = entry.getValue().toString(); + location = Location.current(TServerInstance.deserialize(entry.getValue().toString())); } if (TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key)) { @@ -902,7 +905,7 @@ public static Map>> binOffline(TableId tableId, throw new AccumuloException(" " + lastExtent + " is not previous extent " + extent); } - binnedRanges.computeIfAbsent(last, k -> new HashMap<>()) + binnedRanges.computeIfAbsent(last.getServerInstance().getServer(), k -> new HashMap<>()) .computeIfAbsent(extent, k -> new ArrayList<>()).add(range); if (extent.endRow() == null diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java index 63aa27227df..7fce5abafb1 100644 --- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java +++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java @@ -68,6 +68,7 @@ import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.security.tokens.AuthenticationToken; import org.apache.accumulo.core.clientImpl.ClientContext; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.ClientProperty; import org.apache.accumulo.core.conf.ConfigurationCopy; @@ -76,7 +77,6 @@ import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.conf.SiteConfiguration; import org.apache.accumulo.core.data.InstanceId; -import org.apache.accumulo.core.data.ResourceGroupId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; @@ -722,7 +722,7 @@ public void failedToAcquireLock(Exception e) { throw new IllegalStateException("Error creating path in ZooKeeper", e); } ServiceLockData sld = - new ServiceLockData(miniUUID, "localhost", ThriftService.NONE, ResourceGroupId.DEFAULT); + new ServiceLockData(miniUUID, ServerIdUtil.mini("localhost", 0), ThriftService.NONE); miniLock = new ServiceLock(miniLockZk, slp, miniUUID); miniLock.lock(miniLockWatcher, sld); diff --git a/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java b/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java index 157698997fd..980c64eee4e 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java +++ b/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java @@ -31,6 +31,7 @@ import org.apache.accumulo.core.classloader.ClassLoaderUtil; import org.apache.accumulo.core.cli.ConfigOpts; import org.apache.accumulo.core.client.admin.servers.ServerId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode; import org.apache.accumulo.core.clientImpl.thrift.TInfo; import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; @@ -78,12 +79,13 @@ public static void startServer(AbstractServer server, Logger LOG) throws Excepti server.runServer(); } + private final ServerId.Type serverType; private final MetricSource metricSource; private final ServerContext context; protected final String applicationName; private volatile ServerAddress thriftServer; - private final AtomicReference advertiseAddress; // used for everything but the Thrift - // server (e.g. ZK, metadata, etc). + private final AtomicReference advertiseAddress; // used for everything but the Thrift + // server (e.g. ZK, metadata, etc). private final String bindAddress; // used for the Thrift server private final ResourceGroupId resourceGroup; private final Logger log; @@ -99,10 +101,12 @@ public static void startServer(AbstractServer server, Logger LOG) throws Excepti protected AbstractServer(ServerId.Type serverType, ConfigOpts opts, BiFunction serverContextFactory, String[] args) { + this.serverType = serverType; log = LoggerFactory.getLogger(getClass()); this.applicationName = serverType.name(); opts.parseArgs(applicationName, args); var siteConfig = opts.getSiteConfiguration(); + this.resourceGroup = ResourceGroupId.of(getResourceGroupPropertyValue(siteConfig)); final String newBindParameter = siteConfig.get(Property.RPC_PROCESS_BIND_ADDRESS); // If new bind parameter passed on command line or in file, then use it. if (newBindParameter != null @@ -113,16 +117,16 @@ protected AbstractServer(ServerId.Type serverType, ConfigOpts opts, } String advertAddr = siteConfig.get(Property.RPC_PROCESS_ADVERTISE_ADDRESS); if (advertAddr != null && !advertAddr.isBlank()) { - HostAndPort advertHP = HostAndPort.fromString(advertAddr); + HostAndPort advertHP = HostAndPort.fromString(advertAddr).withDefaultPort(0); if (advertHP.getHost().equals(ConfigOpts.BIND_ALL_ADDRESSES)) { throw new IllegalArgumentException("Advertise address cannot be 0.0.0.0"); } - advertiseAddress = new AtomicReference<>(advertHP); + advertiseAddress = new AtomicReference<>( + ServerIdUtil.dynamic(serverType, resourceGroup, advertHP.getHost(), advertHP.getPort())); } else { advertiseAddress = new AtomicReference<>(); } log.info("Bind address: {}, advertise address: {}", bindAddress, getAdvertiseAddress()); - this.resourceGroup = ResourceGroupId.of(getResourceGroupPropertyValue(siteConfig)); SecurityUtil.serverLogin(siteConfig); context = serverContextFactory.apply(siteConfig, resourceGroup); try { @@ -310,7 +314,7 @@ public void registerMetrics(MeterRegistry registry) { getContext().setMeterRegistry(registry); } - public HostAndPort getAdvertiseAddress() { + public ServerId getAdvertiseAddress() { return advertiseAddress.get(); } @@ -330,12 +334,13 @@ protected ServerAddress getThriftServerAddress() { return thriftServer; } - protected void updateAdvertiseAddress(HostAndPort thriftBindAddress) { + protected void updateAdvertiseAddress(ServerId thriftBindAddress) { advertiseAddress.accumulateAndGet(thriftBindAddress, (curr, update) -> { if (curr == null) { return thriftBindAddress; - } else if (!curr.hasPort()) { - return HostAndPort.fromParts(curr.getHost(), update.getPort()); + } else if (curr.getPort() == 0) { + return ServerIdUtil.dynamic(curr.getType(), curr.getResourceGroup(), curr.getHost(), + thriftBindAddress.getPort()); } else { return curr; } @@ -358,7 +363,8 @@ protected void updateThriftServer(ThriftServerSupplier supplier, boolean start) log.info("Starting {} Thrift server, listening on {}", this.getClass().getSimpleName(), thriftServer.address); } - updateAdvertiseAddress(thriftServer.address); + updateAdvertiseAddress(ServerIdUtil.dynamic(this.serverType, this.resourceGroup, + thriftServer.address.getHost(), thriftServer.address.getPort())); } public ServerContext getContext() { diff --git a/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java b/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java index 21cfef697aa..d9f6c7b6133 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java +++ b/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java @@ -111,9 +111,9 @@ private String root() throws WalMarkerException { // Tablet server exists public void initWalMarker(TServerInstance tsi) throws WalMarkerException { byte[] data = new byte[0]; - try { - zoo.putPersistentData(root() + "/" + tsi, data, NodeExistsPolicy.FAIL); + zoo.putPersistentData(root() + "/" + tsi.toZooKeeperPathString(), data, + NodeExistsPolicy.FAIL); } catch (KeeperException | InterruptedException e) { throw new WalMarkerException(e); } @@ -132,8 +132,9 @@ private void updateState(TServerInstance tsi, Path path, WalState state) if (state == WalState.OPEN) { policy = NodeExistsPolicy.FAIL; } - log.debug("Setting {} to {}", path.getName(), state); - zoo.putPersistentData(root() + "/" + tsi + "/" + path.getName(), data, policy); + final String hostPath = root() + "/" + tsi.toZooKeeperPathString(); + log.debug("Setting {} to {} at WAL marker host znode {}", path.getName(), state, hostPath); + zoo.putPersistentData(hostPath + "/" + path.getName(), data, policy); } catch (KeeperException | InterruptedException e) { throw new WalMarkerException(e); } @@ -153,7 +154,7 @@ private static Pair parse(byte[] data) { public List getWalsInUse(TServerInstance tsi) throws WalMarkerException { List result = new ArrayList<>(); try { - String zpath = root() + "/" + tsi; + final String zpath = root() + "/" + tsi.toZooKeeperPathString(); zoo.sync(zpath); for (String child : zoo.getChildren(zpath)) { byte[] zdata = null; @@ -188,7 +189,7 @@ public Map> getAllMarkers() throws WalMarkerException try { String path = root(); for (String child : zoo.getChildren(path)) { - TServerInstance inst = new TServerInstance(child); + TServerInstance inst = TServerInstance.fromZooKeeperPathString(child); List logs = result.computeIfAbsent(inst, k -> new ArrayList<>()); // This function is called by the Accumulo GC which deletes WAL markers. Therefore we do not @@ -206,7 +207,7 @@ public Map> getAllMarkers() throws WalMarkerException // garbage collector wants to know the state (open/closed) of a log, and the filename to delete public Pair state(TServerInstance instance, UUID uuid) throws WalMarkerException { try { - String path = root() + "/" + instance + "/" + uuid; + String path = root() + "/" + instance.toZooKeeperPathString() + "/" + uuid; return parse(zoo.getData(path)); } catch (KeeperException | InterruptedException e) { throw new WalMarkerException(e); @@ -230,9 +231,9 @@ public Map getAllState() throws WalMarkerException { // garbage collector knows it's safe to remove the marker for a closed log public void removeWalMarker(TServerInstance instance, UUID uuid) throws WalMarkerException { try { - log.debug("Removing {}", uuid); - String path = root() + "/" + instance + "/" + uuid; + String path = root() + "/" + instance.toZooKeeperPathString() + "/" + uuid; zoo.delete(path); + log.debug("Removed {}", path); } catch (InterruptedException | KeeperException e) { throw new WalMarkerException(e); } @@ -240,9 +241,10 @@ public void removeWalMarker(TServerInstance instance, UUID uuid) throws WalMarke // garbage collector knows the instance is dead, and has no markers public void forget(TServerInstance instance) throws WalMarkerException { - String path = root() + "/" + instance; + String path = root() + "/" + instance.toZooKeeperPathString(); try { zoo.recursiveDelete(path, NodeMissingPolicy.FAIL); + log.debug("Removed {} recursively", path); } catch (InterruptedException | KeeperException e) { throw new WalMarkerException(e); } diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java b/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java index 8d7f5ee6539..c2fb441cf75 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java +++ b/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java @@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; import org.apache.accumulo.core.data.ResourceGroupId; import org.apache.accumulo.core.data.TableId; @@ -52,7 +53,6 @@ import org.apache.accumulo.core.tablet.thrift.TabletManagementClientService; import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService; import org.apache.accumulo.core.trace.TraceUtil; -import org.apache.accumulo.core.util.AddressUtil; import org.apache.accumulo.core.util.Halt; import org.apache.accumulo.core.util.threads.ThreadPools; import org.apache.accumulo.core.zookeeper.ZcStat; @@ -82,14 +82,14 @@ public interface Listener { private final ServerContext context; public class TServerConnection { - private final HostAndPort address; + private final ServerId server; - public TServerConnection(HostAndPort addr) { - address = addr; + public TServerConnection(ServerId addr) { + server = addr; } - public HostAndPort getAddress() { - return address; + public ServerId getAddress() { + return server; } private String lockString(ServiceLock mlock) { @@ -105,14 +105,14 @@ private void loadTablet(TabletManagementClientService.Client client, ServiceLock public void assignTablet(ServiceLock lock, KeyExtent extent) throws TException { if (extent.isMeta()) { // see ACCUMULO-3597 - try (TTransport transport = ThriftUtil.createTransport(address, context)) { + try (TTransport transport = ThriftUtil.createTransport(server, context)) { TabletManagementClientService.Client client = ThriftUtil .createClient(ThriftClientTypes.TABLET_MGMT, transport, context.getInstanceID()); loadTablet(client, lock, extent); } } else { TabletManagementClientService.Client client = - ThriftUtil.getClient(ThriftClientTypes.TABLET_MGMT, address, context); + ThriftUtil.getClient(ThriftClientTypes.TABLET_MGMT, server, context); try { loadTablet(client, lock, extent); } finally { @@ -124,7 +124,7 @@ public void assignTablet(ServiceLock lock, KeyExtent extent) throws TException { public void unloadTablet(ServiceLock lock, KeyExtent extent, TUnloadTabletGoal goal, long requestTime) throws TException { TabletManagementClientService.Client client = - ThriftUtil.getClient(ThriftClientTypes.TABLET_MGMT, address, context); + ThriftUtil.getClient(ThriftClientTypes.TABLET_MGMT, server, context); try { client.unloadTablet(TraceUtil.traceInfo(), context.rpcCreds(), lockString(lock), extent.toThrift(), goal, requestTime); @@ -142,7 +142,7 @@ public TabletServerStatus getTableMap(boolean usePooledConnection) long start = System.currentTimeMillis(); - try (TTransport transport = ThriftUtil.createTransport(address, context)) { + try (TTransport transport = ThriftUtil.createTransport(server, context)) { TabletServerClientService.Client client = ThriftUtil .createClient(ThriftClientTypes.TABLET_SERVER, transport, context.getInstanceID()); TabletServerStatus status = @@ -156,7 +156,7 @@ public TabletServerStatus getTableMap(boolean usePooledConnection) public void halt(ServiceLock lock) throws TException, ThriftSecurityException { TabletServerClientService.Client client = - ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context); + ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, server, context); try { client.halt(TraceUtil.traceInfo(), context.rpcCreds(), lockString(lock)); } finally { @@ -166,7 +166,7 @@ public void halt(ServiceLock lock) throws TException, ThriftSecurityException { public void fastHalt(ServiceLock lock) throws TException { TabletServerClientService.Client client = - ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context); + ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, server, context); try { client.fastHalt(TraceUtil.traceInfo(), context.rpcCreds(), lockString(lock)); } finally { @@ -177,7 +177,7 @@ public void fastHalt(ServiceLock lock) throws TException { public void flush(ServiceLock lock, TableId tableId, byte[] startRow, byte[] endRow) throws TException { TabletServerClientService.Client client = - ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context); + ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, server, context); try { client.flush(TraceUtil.traceInfo(), context.rpcCreds(), lockString(lock), tableId.canonical(), startRow == null ? null : ByteBuffer.wrap(startRow), @@ -192,13 +192,10 @@ public void flush(ServiceLock lock, TableId tableId, byte[] startRow, byte[] end static class TServerInfo { final TServerConnection connection; final TServerInstance instance; - final ResourceGroupId resourceGroup; - TServerInfo(TServerInstance instance, TServerConnection connection, - ResourceGroupId resourceGroup) { + TServerInfo(TServerInstance instance, TServerConnection connection) { this.connection = connection; this.instance = instance; - this.resourceGroup = resourceGroup; } } @@ -297,21 +294,17 @@ private synchronized void checkServer(final Set updates, } else { log.trace("Lock exists for server: {}, adding to current set", tserverPath.getServer()); locklessServers.remove(tserverPath); - HostAndPort address = sld.orElseThrow().getAddress(ServiceLockData.ThriftService.TSERV); - ResourceGroupId resourceGroup = - sld.orElseThrow().getGroup(ServiceLockData.ThriftService.TSERV); - TServerInstance instance = new TServerInstance(address, stat.getEphemeralOwner()); + ServerId server = sld.orElseThrow().getServer(ServiceLockData.ThriftService.TSERV); + TServerInstance instance = new TServerInstance(server, stat.getEphemeralOwner()); if (info == null) { updates.add(instance); - TServerInfo tServerInfo = - new TServerInfo(instance, new TServerConnection(address), resourceGroup); + TServerInfo tServerInfo = new TServerInfo(instance, new TServerConnection(server)); current.put(tserverPath.getServer(), tServerInfo); } else if (!info.instance.equals(instance)) { doomed.add(info.instance); updates.add(instance); - TServerInfo tServerInfo = - new TServerInfo(instance, new TServerConnection(address), resourceGroup); + TServerInfo tServerInfo = new TServerInfo(instance, new TServerConnection(server)); current.put(tserverPath.getServer(), tServerInfo); } } @@ -386,17 +379,6 @@ public synchronized TServerConnection getConnection(TServerInstance server) { return tServerInfo.connection; } - public synchronized ResourceGroupId getResourceGroup(TServerInstance server) { - if (server == null) { - return null; - } - TServerInfo tServerInfo = getSnapshot().tserversInfo.get(server); - if (tServerInfo == null) { - return null; - } - return tServerInfo.resourceGroup; - } - public static class LiveTServersSnapshot { private final Set tservers; private final Map> tserverGroups; @@ -438,8 +420,8 @@ public synchronized LiveTServersSnapshot getSnapshot() { Map> tserversGroups = new HashMap<>(); current.values().forEach(tServerInfo -> { tServerInstances.put(tServerInfo.instance, tServerInfo); - tserversGroups.computeIfAbsent(tServerInfo.resourceGroup, rg -> new HashSet<>()) - .add(tServerInfo.instance); + tserversGroups.computeIfAbsent(tServerInfo.instance.getServer().getResourceGroup(), + rg -> new HashSet<>()).add(tServerInfo.instance); }); tServersSnapshot = new LiveTServersSnapshot(tServerInstances, tserversGroups); } @@ -460,23 +442,13 @@ public synchronized TServerInstance find(String tabletServer) { } static TServerInstance find(Map servers, String tabletServer) { - HostAndPort addr; - String sessionId = null; - if (tabletServer.charAt(tabletServer.length() - 1) == ']') { - int index = tabletServer.indexOf('['); - if (index == -1) { - throw new IllegalArgumentException("Could not parse tabletserver '" + tabletServer + "'"); - } - addr = AddressUtil.parseAddress(tabletServer.substring(0, index)); - // Strip off the last bracket - sessionId = tabletServer.substring(index + 1, tabletServer.length() - 1); - } else { - addr = AddressUtil.parseAddress(tabletServer); - } + var target = TServerInstance.fromZooKeeperPathString(tabletServer); for (Entry entry : servers.entrySet()) { - if (entry.getValue().instance.getHostAndPort().equals(addr)) { + if (entry.getValue().instance.getServer().getHost().equals(target.getServer().getHost()) + && (entry.getValue().instance.getServer().getPort() == target.getServer().getPort())) { // Return the instance if we have no desired session ID, or we match the desired session ID - if (sessionId == null || sessionId.equals(entry.getValue().instance.getSession())) { + if (target.getSession() == null || target.getSession().equals(Long.toHexString(0)) + || target.getSession().equals(entry.getValue().instance.getSession())) { return entry.getValue().instance; } } @@ -494,7 +466,7 @@ public synchronized void remove(TServerInstance server) { for (Entry entry : current.entrySet()) { if (entry.getValue().instance.equals(server)) { address = Optional.of(HostAndPort.fromString(entry.getKey())); - resourceGroup = Optional.of(entry.getValue().resourceGroup); + resourceGroup = Optional.of(entry.getValue().instance.getServer().getResourceGroup()); break; } } diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/balancer/BalancerEnvironmentImpl.java b/server/base/src/main/java/org/apache/accumulo/server/manager/balancer/BalancerEnvironmentImpl.java index cdab4586496..91e16e516a9 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/manager/balancer/BalancerEnvironmentImpl.java +++ b/server/base/src/main/java/org/apache/accumulo/server/manager/balancer/BalancerEnvironmentImpl.java @@ -53,8 +53,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.net.HostAndPort; - public class BalancerEnvironmentImpl extends ServiceEnvironmentImpl implements BalancerEnvironment { private static final Logger log = LoggerFactory.getLogger(BalancerEnvironmentImpl.class); @@ -91,9 +89,8 @@ public List listOnlineTabletsForTable(TabletServerId tabletSer TableId tableId) throws AccumuloException, AccumuloSecurityException { log.trace("Scanning tablet server {} for table {}", tabletServerId, tableId); try { - TabletServerClientService.Client client = ThriftUtil.getClient( - ThriftClientTypes.TABLET_SERVER, - HostAndPort.fromParts(tabletServerId.getHost(), tabletServerId.getPort()), getContext()); + TabletServerClientService.Client client = ThriftUtil + .getClient(ThriftClientTypes.TABLET_SERVER, tabletServerId.getServer(), getContext()); try { return client .getTabletStats(TraceUtil.traceInfo(), getContext().rpcCreds(), tableId.canonical()) diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/LoggingTabletStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/LoggingTabletStateStore.java index b5da85b3ad9..3d9b498e2d2 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/LoggingTabletStateStore.java +++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/LoggingTabletStateStore.java @@ -23,6 +23,7 @@ import java.util.Map; import java.util.Set; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.logging.TabletLogger; @@ -33,8 +34,6 @@ import org.apache.accumulo.core.util.time.SteadyTime; import org.apache.hadoop.fs.Path; -import com.google.common.net.HostAndPort; - /** * Wraps a tablet state store and logs important events. */ @@ -106,9 +105,9 @@ public void suspend(Collection tablets, for (TabletMetadata tm : tablets) { var location = tm.getLocation(); - HostAndPort server = null; + ServerId server = null; if (location != null) { - server = location.getHostAndPort(); + server = location.getServerInstance().getServer(); } TabletLogger.suspended(tm.getExtent(), server, suspensionTimestamp, logsForDeadServers.size()); diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletManagementParameters.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletManagementParameters.java index 0030228568e..409ac4decab 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletManagementParameters.java +++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletManagementParameters.java @@ -41,6 +41,7 @@ import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.manager.thrift.ManagerState; import org.apache.accumulo.core.metadata.TServerInstance; +import org.apache.accumulo.core.metadata.TServerInstance.TServerInstanceInfo; import org.apache.accumulo.core.metadata.schema.Ample; import org.apache.accumulo.core.util.time.SteadyTime; import org.apache.accumulo.server.manager.LiveTServerSet; @@ -107,15 +108,15 @@ private TabletManagementParameters(JsonData jdata) { this.parentUpgradeMap = Map.copyOf(jdata.parentUpgradeMap); this.onlineTables = jdata.onlineTables.stream().map(TableId::of).collect(toUnmodifiableSet()); this.onlineTservers = - jdata.onlineTservers.stream().map(TServerInstance::new).collect(toUnmodifiableSet()); - this.serversToShutdown = - jdata.serversToShutdown.stream().map(TServerInstance::new).collect(toUnmodifiableSet()); + jdata.onlineTservers.stream().map(TServerInstanceInfo::getTSI).collect(toUnmodifiableSet()); + this.serversToShutdown = jdata.serversToShutdown.stream().map(TServerInstanceInfo::getTSI) + .collect(toUnmodifiableSet()); this.level = jdata.level; this.compactionHints = makeImmutable(jdata.compactionHints.entrySet().stream() .collect(Collectors.toMap(entry -> FateId.from(entry.getKey()), Map.Entry::getValue))); - this.tserverGroups = jdata.tserverGroups.entrySet().stream().collect(toUnmodifiableMap( - entry -> ResourceGroupId.of(entry.getKey()), - entry -> entry.getValue().stream().map(TServerInstance::new).collect(toUnmodifiableSet()))); + this.tserverGroups = jdata.tserverGroups.entrySet().stream() + .collect(toUnmodifiableMap(entry -> ResourceGroupId.of(entry.getKey()), entry -> entry + .getValue().stream().map(TServerInstanceInfo::getTSI).collect(toUnmodifiableSet()))); this.resourceGroups = Suppliers.memoize(() -> { Map resourceGroups = new HashMap<>(); TabletManagementParameters.this.tserverGroups.forEach((resourceGroup, tservers) -> tservers @@ -196,12 +197,12 @@ private static class JsonData { ManagerState managerState; Map parentUpgradeMap; Collection onlineTables; - Collection onlineTservers; - Collection serversToShutdown; + Collection onlineTservers; + Collection serversToShutdown; Ample.DataLevel level; - Map> tserverGroups; + Map> tserverGroups; Map> compactionHints; @@ -217,14 +218,14 @@ private JsonData() {} managerState = params.managerState; parentUpgradeMap = params.parentUpgradeMap; onlineTables = params.onlineTables.stream().map(AbstractId::canonical).collect(toList()); - onlineTservers = params.getOnlineTsevers().stream().map(TServerInstance::getHostPortSession) + onlineTservers = params.getOnlineTsevers().stream().map(tsi -> tsi.getTServerInstanceInfo()) .collect(toList()); - serversToShutdown = params.serversToShutdown.stream().map(TServerInstance::getHostPortSession) + serversToShutdown = params.serversToShutdown.stream().map(tsi -> tsi.getTServerInstanceInfo()) .collect(toList()); level = params.level; tserverGroups = params.getGroupedTServers().entrySet().stream() .collect(toMap(entry -> entry.getKey().canonical(), entry -> entry.getValue().stream() - .map(TServerInstance::getHostPortSession).collect(toSet()))); + .map(tsi -> tsi.getTServerInstanceInfo()).collect(toSet()))); compactionHints = params.compactionHints.entrySet().stream() .collect(Collectors.toMap(entry -> entry.getKey().canonical(), Map.Entry::getValue)); canSuspendTablets = params.canSuspendTablets; diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletManagementScanner.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletManagementScanner.java index 7be790e2694..9fd38bb916b 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletManagementScanner.java +++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletManagementScanner.java @@ -33,6 +33,7 @@ import org.apache.accumulo.core.client.BatchScanner; import org.apache.accumulo.core.client.InvalidTabletHostingRequestException; import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.clientImpl.ClientTabletCache; import org.apache.accumulo.core.clientImpl.ClientTabletCache.LocationNeed; @@ -68,7 +69,7 @@ public TabletManagementScanner(ClientContext context, List ranges, try { final TableId tid = context.getTableId(tableName); final ClientTabletCache locator = context.getTabletLocationCache(tid); - final Set locations = new HashSet<>(); + final Set locations = new HashSet<>(); final List failures = locator.findTablets(context, ALL_TABLETS_RANGE, (ct, r) -> ct.getTserverLocation().ifPresent(locations::add), LocationNeed.NOT_REQUIRED); diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletMutatorImpl.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletMutatorImpl.java index 50acebc727d..412e9745eae 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletMutatorImpl.java +++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletMutatorImpl.java @@ -143,8 +143,8 @@ public Ample.ConditionalTabletMutator requireLocation(Location location) { sawOperationRequirement = true; Function> encoder = - l -> new Pair<>(location.getSession().getBytes(UTF_8), - location.getHostPort().getBytes(UTF_8)); + l -> new Pair<>(location.getServerInstance().getSession().getBytes(UTF_8), + location.getServerInstance().serialize().getBytes(UTF_8)); // The location column family can have multiple column qualifiers set. When requiring a location // we want to check the location is set AND that no other location qualifiers are set on the @@ -394,7 +394,7 @@ public ConditionalTabletMutator requireMigration(TServerInstance tserver) { Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate."); Condition condition = new Condition(MIGRATION_COLUMN.getColumnFamily(), MIGRATION_COLUMN.getColumnQualifier()) - .setValue(tserver.getHostPortSession()); + .setValue(tserver.serialize()); mutation.addCondition(condition); return this; } diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/iterators/CurrentLocationNotEqualToIterator.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/iterators/CurrentLocationNotEqualToIterator.java index 3ade6465cec..f6d058e9c91 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/metadata/iterators/CurrentLocationNotEqualToIterator.java +++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/iterators/CurrentLocationNotEqualToIterator.java @@ -18,8 +18,6 @@ */ package org.apache.accumulo.server.metadata.iterators; -import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily; - import java.io.IOException; import java.util.Map; @@ -30,8 +28,8 @@ import org.apache.accumulo.core.iterators.IteratorEnvironment; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; import org.apache.accumulo.core.metadata.TServerInstance; +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily; import org.apache.accumulo.server.metadata.ConditionalTabletMutatorImpl; -import org.apache.hadoop.io.Text; public class CurrentLocationNotEqualToIterator extends ColumnFamilyTransformationIterator { private static final String TSERVER_INSTANCE_OPTION = "tsi_option"; @@ -51,8 +49,7 @@ protected Value transform(SortedKeyValueIterator source) throws IOExc TServerInstance tsiSeen; while (source.hasTop()) { Value address = source.getTopValue(); - Text session = source.getTopKey().getColumnQualifier(); - tsiSeen = new TServerInstance(address, session); + tsiSeen = new TServerInstance(address.toString()); if (tsiSeen.equals(tsi)) { return new Value(EQUAL); } @@ -69,7 +66,7 @@ protected Value transform(SortedKeyValueIterator source) throws IOExc public static Condition createCondition(TServerInstance tsi) { IteratorSetting is = new IteratorSetting(ConditionalTabletMutatorImpl.INITIAL_ITERATOR_PRIO, CurrentLocationNotEqualToIterator.class); - is.addOption(TSERVER_INSTANCE_OPTION, tsi.getHostPortSession()); + is.addOption(TSERVER_INSTANCE_OPTION, tsi.serialize()); return new Condition(CurrentLocationColumnFamily.NAME, EMPTY).setValue(NOT_EQUAL) .setIterators(is); } diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java index 64a2d4896b0..424469e89c2 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java @@ -59,6 +59,7 @@ import org.apache.accumulo.core.client.admin.InstanceOperations; import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ClientContext; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.DefaultConfiguration; import org.apache.accumulo.core.conf.Property; @@ -81,6 +82,7 @@ import org.apache.accumulo.core.manager.thrift.FateService; import org.apache.accumulo.core.manager.thrift.TFateId; import org.apache.accumulo.core.metadata.SystemTables; +import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.core.metadata.schema.TabletMetadata; import org.apache.accumulo.core.process.thrift.ServerProcessService; import org.apache.accumulo.core.rpc.ThriftUtil; @@ -90,7 +92,6 @@ import org.apache.accumulo.core.security.SystemPermission; import org.apache.accumulo.core.security.TablePermission; import org.apache.accumulo.core.trace.TraceUtil; -import org.apache.accumulo.core.util.AddressUtil; import org.apache.accumulo.core.util.Halt; import org.apache.accumulo.core.zookeeper.ZooCache; import org.apache.accumulo.core.zookeeper.ZooSession; @@ -715,27 +716,33 @@ private static void stopServers(final ServerContext context, List server } } else { for (var server : hostAndPort) { - signalGracefulShutdown(context, HostAndPort.fromString(server)); + for (ServerId.Type t : ServerId.Type.values()) { + Set matchingServers = + context.instanceOperations().getServers(t, ResourceGroupPredicate.ANY, (h, p) -> { + var hp = HostAndPort.fromString(server); + return hp.getHost().equals(h) && hp.getPort() == p; + }); + matchingServers.forEach(ms -> signalGracefulShutdown(context, ms)); + } } } } } // Visible for tests - public static void signalGracefulShutdown(final ClientContext context, HostAndPort hp) { - Objects.requireNonNull(hp, "address not set"); + public static void signalGracefulShutdown(final ClientContext context, ServerId server) { + Objects.requireNonNull(server, "address not set"); ServerProcessService.Client client = null; try { - client = ThriftClientTypes.SERVER_PROCESS.getServerProcessConnection(context, log, - hp.getHost(), hp.getPort()); + client = ThriftClientTypes.SERVER_PROCESS.getServerProcessConnection(context, log, server); if (client == null) { - log.warn("Failed to initiate shutdown for {}", hp); + log.warn("Failed to initiate shutdown for {}", server); return; } client.gracefulShutdown(context.rpcCreds()); - log.info("Initiated shutdown for {}", hp); + log.info("Initiated shutdown for {}", server); } catch (TException e) { - log.warn("Failed to initiate shutdown for {}", hp, e); + log.warn("Failed to initiate shutdown for {}", server, e); } finally { if (client != null) { ThriftUtil.returnClient(client, context); @@ -773,15 +780,14 @@ private static void stopTabletServer(final ClientContext context, List s log.info("Only 1 tablet server running. Not attempting shutdown of {}", server); return; } - for (int port : context.getConfiguration().getPort(Property.TSERV_CLIENTPORT)) { - HostAndPort address = AddressUtil.parseAddress(server, port); - final String finalServer = qualifyWithZooKeeperSessionId(context, zc, address.toString()); + for (ServerId sid : runningServers) { + final TServerInstance finalServer = + qualifyWithZooKeeperSessionId(context, zc, sid.toHostPortString()); log.info("Stopping server {}", finalServer); - ThriftClientTypes.MANAGER - .executeVoid( - context, client -> client.shutdownTabletServer(TraceUtil.traceInfo(), - context.rpcCreds(), finalServer, force), - ResourceGroupPredicate.DEFAULT_RG_ONLY); + ThriftClientTypes.MANAGER.executeVoid(context, + client -> client.shutdownTabletServer(TraceUtil.traceInfo(), context.rpcCreds(), + finalServer.toZooKeeperPathString(), force), + ResourceGroupPredicate.DEFAULT_RG_ONLY); } } } @@ -793,19 +799,23 @@ private static void stopTabletServer(final ClientContext context, List s * @return The host and port with the session ID in square-brackets appended, or the original * value. */ - static String qualifyWithZooKeeperSessionId(ClientContext context, ZooCache zooCache, + static TServerInstance qualifyWithZooKeeperSessionId(ClientContext context, ZooCache zooCache, String hostAndPort) { var hpObj = HostAndPort.fromString(hostAndPort); Set paths = context.getServerPaths() .getTabletServer(ResourceGroupPredicate.ANY, AddressSelector.exact(hpObj), true); if (paths.size() != 1) { - return hostAndPort; + return new TServerInstance(ServerIdUtil.tserver(hpObj.getHost(), hpObj.getPort()), + Long.toHexString(0)); } - long sessionId = ServiceLock.getSessionId(zooCache, paths.iterator().next()); + ServiceLockPath slp = paths.iterator().next(); + long sessionId = ServiceLock.getSessionId(zooCache, slp); if (sessionId == 0) { - return hostAndPort; + return new TServerInstance(ServerIdUtil.tserver(hpObj.getHost(), hpObj.getPort()), + Long.toHexString(0)); } - return hostAndPort + "[" + Long.toHexString(sessionId) + "]"; + return new TServerInstance(ServerIdUtil.tserver(hpObj.getHost(), hpObj.getPort()), + Long.toHexString(sessionId)); } private static final String ACCUMULO_SITE_BACKUP_FILE = "accumulo.properties.bak"; @@ -1142,7 +1152,8 @@ private ServiceLock createAdminLock(ServerContext context) throws InterruptedExc AdminLockWatcher lw = new AdminLockWatcher(); ServiceLockData.ServiceDescriptors descriptors = new ServiceLockData.ServiceDescriptors(); descriptors.addService(new ServiceLockData.ServiceDescriptor(uuid, - ServiceLockData.ThriftService.NONE, "fake_admin_util_host", ResourceGroupId.DEFAULT)); + ServiceLockData.ThriftService.NONE, + ServerIdUtil.dynamic(ServerId.Type.MANAGER, ResourceGroupId.DEFAULT, "admin_utility", 0))); ServiceLockData sld = new ServiceLockData(descriptors); String lockPath = slp.toString(); String parentLockPath = lockPath.substring(0, lockPath.lastIndexOf("/")); diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java b/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java index 870e8652a92..e956022ece1 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java @@ -49,7 +49,6 @@ import com.beust.jcommander.Parameter; import com.beust.jcommander.Parameters; import com.google.auto.service.AutoService; -import com.google.common.net.HostAndPort; import com.google.gson.Gson; import com.google.gson.GsonBuilder; @@ -63,7 +62,7 @@ public class ECAdmin implements KeywordExecutable { public static class RunningCompactionSummary { private final String ecid; - private final String addr; + private final ServerId addr; private final TCompactionKind kind; private final ResourceGroupId groupName; private final String ke; @@ -78,9 +77,9 @@ public RunningCompactionSummary(RunningCompaction runningCompaction, RunningCompactionInfo runningCompactionInfo) { super(); ecid = runningCompaction.getJob().getExternalCompactionId(); - addr = runningCompaction.getCompactorAddress(); + addr = runningCompaction.getCompactor(); kind = runningCompaction.getJob().kind; - groupName = runningCompaction.getGroup(); + groupName = addr.getResourceGroup(); KeyExtent extent = KeyExtent.fromThrift(runningCompaction.getJob().extent); ke = extent.obscured(); tableId = extent.tableId().canonical(); @@ -138,7 +137,7 @@ public String getEcid() { return ecid; } - public String getAddr() { + public ServerId getAddr() { return addr; } @@ -325,7 +324,7 @@ private CompactionCoordinatorService.Client getCoordinatorClient(ServerContext c if (coordinatorHost.isEmpty()) { throw new IllegalStateException("Unable to find coordinator. Check that it is running."); } - HostAndPort address = coordinatorHost.orElseThrow(); + ServerId address = coordinatorHost.orElseThrow(); CompactionCoordinatorService.Client coordinatorClient; try { coordinatorClient = ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, address, context); diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java b/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java index 6be707062bf..0b392a3821b 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java @@ -174,7 +174,7 @@ private static String getManager(ZooSession zs, InstanceId iid, boolean printErr if (sld.isEmpty()) { return null; } - return sld.orElseThrow().getAddressString(ThriftService.MANAGER); + return sld.orElseThrow().getServer(ThriftService.MANAGER).toHostPortString(); } catch (Exception e) { handleException(e, printErrors); return null; diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ServiceStatusCmd.java b/server/base/src/main/java/org/apache/accumulo/server/util/ServiceStatusCmd.java index c43492d339f..17b525575e5 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/ServiceStatusCmd.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/ServiceStatusCmd.java @@ -185,7 +185,7 @@ private StatusSummary getStatusSummary(ServiceStatusReport.ReportKey displayName var services = sld.getServices(); services.forEach(sd -> { byGroup.computeIfAbsent(sd.getGroup().canonical(), set -> new TreeSet<>()) - .add(sd.getAddress()); + .add(sd.getServer().toHostPortString()); }); }); byGroup.forEach((group, hosts) -> resourceGroups.put(group, hosts.size())); diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java b/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java index 545be6f88a5..65be2b089e6 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java @@ -53,7 +53,7 @@ public static void execute(final ServerContext context, final String lock, final Optional lockData = ServiceLock.getLockData(cache, tabletServer, null); final String holder; if (lockData.isPresent()) { - holder = lockData.orElseThrow().getAddressString(ThriftService.TSERV); + holder = lockData.orElseThrow().getServer(ThriftService.TSERV).toString(); } else { holder = ""; } diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java b/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java index 1cdfaf6b513..def8cfb88d8 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java @@ -33,6 +33,7 @@ import org.apache.accumulo.core.client.Accumulo; import org.apache.accumulo.core.client.AccumuloClient; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.clientImpl.thrift.TInfo; import org.apache.accumulo.core.data.Range; @@ -50,7 +51,6 @@ import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService; import org.apache.accumulo.core.trace.TraceUtil; -import org.apache.accumulo.core.util.HostAndPortComparator; import org.apache.accumulo.core.util.threads.ThreadPools; import org.apache.hadoop.io.Text; import org.apache.thrift.TException; @@ -58,8 +58,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.net.HostAndPort; - import io.opentelemetry.api.trace.Span; import io.opentelemetry.context.Scope; @@ -92,7 +90,7 @@ private static void checkTable(final ClientContext context, final boolean verbos final HashSet failures = new HashSet<>(); - Map> extentsPerServer = new TreeMap<>(new HostAndPortComparator()); + Map> extentsPerServer = new TreeMap<>(); try (var tabletsMeta = context.getAmple().readTablets().forTable(tableId) .fetch(TabletMetadata.ColumnType.LOCATION).checkConsistency().build()) { @@ -106,9 +104,8 @@ private static void checkTable(final ClientContext context, final boolean verbos } if (loc != null) { - final HostAndPort parsedLoc = loc.getHostAndPort(); - List extentList = - extentsPerServer.computeIfAbsent(parsedLoc, k -> new ArrayList<>()); + List extentList = extentsPerServer + .computeIfAbsent(loc.getServerInstance().getServer(), k -> new ArrayList<>()); if (check == null || check.contains(keyExtent)) { extentList.add(keyExtent); @@ -120,7 +117,7 @@ private static void checkTable(final ClientContext context, final boolean verbos ExecutorService tp = ThreadPools.getServerThreadPools() .getPoolBuilder(UTILITY_VERIFY_TABLET_ASSIGNMENTS).numCoreThreads(20).build(); - for (final Entry> entry : extentsPerServer.entrySet()) { + for (final Entry> entry : extentsPerServer.entrySet()) { Runnable r = () -> { try { checkTabletServer(context, entry, failures); @@ -142,7 +139,7 @@ private static void checkTable(final ClientContext context, final boolean verbos } } - private static void checkFailures(HostAndPort server, HashSet failures, + private static void checkFailures(ServerId server, HashSet failures, MultiScanResult scanResult) { for (TKeyExtent tke : scanResult.failures.keySet()) { KeyExtent ke = KeyExtent.fromThrift(tke); @@ -152,7 +149,7 @@ private static void checkFailures(HostAndPort server, HashSet failure } private static void checkTabletServer(ClientContext context, - Entry> entry, HashSet failures) throws TException { + Entry> entry, HashSet failures) throws TException { TabletScanClientService.Iface client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SCAN, entry.getKey(), context); diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java b/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java index 8b84b2099ed..0308a3e5eb6 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java @@ -242,9 +242,9 @@ static Optional filterSingleton(ServerContext context, ServiceL Optional sld = context.getZooCache().getLockData(path); return sld.filter(lockData -> { for (var service : ServiceLockData.ThriftService.values()) { - var address = lockData.getAddress(service); + var address = lockData.getServer(service); if (address != null) { - return addressSelector.getPredicate().test(address.toString()); + return addressSelector.getPredicate().test(address.toHostPortString()); } } diff --git a/server/base/src/test/java/org/apache/accumulo/server/constraints/MetadataConstraintsTest.java b/server/base/src/test/java/org/apache/accumulo/server/constraints/MetadataConstraintsTest.java index 2ed44cbf4b1..492c3f40a78 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/constraints/MetadataConstraintsTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/constraints/MetadataConstraintsTest.java @@ -25,6 +25,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import java.lang.reflect.Field; import java.lang.reflect.Method; import java.time.Duration; import java.util.Base64; @@ -34,6 +35,7 @@ import java.util.concurrent.TimeUnit; import org.apache.accumulo.core.client.admin.TabletAvailability; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Range; @@ -67,8 +69,6 @@ import org.easymock.EasyMock; import org.junit.jupiter.api.Test; -import com.google.common.net.HostAndPort; - public class MetadataConstraintsTest { private SystemEnvironment createEnv() { @@ -155,25 +155,31 @@ public void testCheck() { } @Test - public void testSuspensionCheck() { + public void testSuspensionCheck() throws Exception { Mutation m = new Mutation(new Text("0;foo")); MetadataConstraints mc = new MetadataConstraints(); - TServerInstance ser1 = new TServerInstance(HostAndPort.fromParts("server1", 8555), "s001"); + TServerInstance ser1 = new TServerInstance(ServerIdUtil.tserver("server1", 8555), "s001"); - SuspendLocationColumn.SUSPEND_COLUMN.put(m, SuspendingTServer.toValue(ser1, - SteadyTime.from(System.currentTimeMillis(), TimeUnit.MILLISECONDS))); + SuspendLocationColumn.SUSPEND_COLUMN.put(m, new SuspendingTServer(ser1, + SteadyTime.from(System.currentTimeMillis(), TimeUnit.MILLISECONDS)).toValue()); List violations = mc.check(createEnv(), m); assertTrue(violations.isEmpty()); m = new Mutation(new Text("0;foo")); SuspendLocationColumn.SUSPEND_COLUMN.put(m, - SuspendingTServer.toValue(ser1, SteadyTime.from(0, TimeUnit.MILLISECONDS))); + new SuspendingTServer(ser1, SteadyTime.from(0, TimeUnit.MILLISECONDS)).toValue()); violations = mc.check(createEnv(), m); assertTrue(violations.isEmpty()); m = new Mutation(new Text("0;foo")); // We must encode manually since SteadyTime won't allow a negative - SuspendLocationColumn.SUSPEND_COLUMN.put(m, new Value(ser1.getHostPort() + "|" + -1L)); + var sts = new SuspendingTServer(ser1, + SteadyTime.from(System.currentTimeMillis(), TimeUnit.MILLISECONDS)); + Field timeField = SteadyTime.class.getDeclaredField("time"); + assertNotNull(timeField); + timeField.setAccessible(true); + timeField.set(sts.suspensionTime, Duration.ofMillis(-1)); + SuspendLocationColumn.SUSPEND_COLUMN.put(m, new Value(sts.toValue())); violations = mc.check(createEnv(), m); assertFalse(violations.isEmpty()); assertEquals(1, violations.size()); diff --git a/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeUtilTest.java b/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeUtilTest.java index 033cf172125..d9870d3b9db 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeUtilTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeUtilTest.java @@ -169,19 +169,19 @@ public void testWalVolumeReplacment() { replacements.put(new Path("hdfs://nn2/accumulo"), new Path("viewfs:/b/accumulo")); String walUUID = UUID.randomUUID().toString(); - String fileName = "hdfs://nn1/accumulo/wal/localhost+9997/" + walUUID; + String fileName = "hdfs://nn1/accumulo/wal/default+localhost+9997/" + walUUID; LogEntry le = LogEntry.fromPath(fileName); LogEntry fixedVolume = VolumeUtil.switchVolume(le, replacements); - assertEquals("viewfs:/a/accumulo/wal/localhost+9997/" + walUUID, fixedVolume.getPath()); + assertEquals("viewfs:/a/accumulo/wal/default+localhost+9997/" + walUUID, fixedVolume.getPath()); - fileName = "hdfs://nn1:9000/accumulo/wal/localhost+9997/" + walUUID; + fileName = "hdfs://nn1:9000/accumulo/wal/default+localhost+9997/" + walUUID; le = LogEntry.fromPath(fileName); fixedVolume = VolumeUtil.switchVolume(le, replacements); - assertEquals("viewfs:/a/accumulo/wal/localhost+9997/" + walUUID, fixedVolume.getPath()); + assertEquals("viewfs:/a/accumulo/wal/default+localhost+9997/" + walUUID, fixedVolume.getPath()); - fileName = "hdfs://nn2/accumulo/wal/localhost+9997/" + walUUID; + fileName = "hdfs://nn2/accumulo/wal/default+localhost+9997/" + walUUID; le = LogEntry.fromPath(fileName); fixedVolume = VolumeUtil.switchVolume(le, replacements); - assertEquals("viewfs:/b/accumulo/wal/localhost+9997/" + walUUID, fixedVolume.getPath()); + assertEquals("viewfs:/b/accumulo/wal/default+localhost+9997/" + walUUID, fixedVolume.getPath()); } } diff --git a/server/base/src/test/java/org/apache/accumulo/server/manager/LiveTServerSetTest.java b/server/base/src/test/java/org/apache/accumulo/server/manager/LiveTServerSetTest.java index 15e9d5e3ac5..65968576075 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/manager/LiveTServerSetTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/manager/LiveTServerSetTest.java @@ -24,15 +24,13 @@ import java.util.HashMap; import java.util.Map; -import org.apache.accumulo.core.data.ResourceGroupId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection; import org.apache.accumulo.server.manager.LiveTServerSet.TServerInfo; import org.easymock.EasyMock; import org.junit.jupiter.api.Test; -import com.google.common.net.HostAndPort; - public class LiveTServerSetTest { @Test @@ -40,15 +38,14 @@ public void testSessionIds() { Map servers = new HashMap<>(); TServerConnection mockConn = EasyMock.createMock(TServerConnection.class); - TServerInfo server1 = - new TServerInfo(new TServerInstance(HostAndPort.fromParts("localhost", 1234), "5555"), - mockConn, ResourceGroupId.DEFAULT); + TServerInfo server1 = new TServerInfo( + new TServerInstance(ServerIdUtil.tserver("localhost", 1234), "5555"), mockConn); servers.put("server1", server1); - assertEquals(server1.instance, LiveTServerSet.find(servers, "localhost:1234")); - assertNull(LiveTServerSet.find(servers, "localhost:4321")); - assertEquals(server1.instance, LiveTServerSet.find(servers, "localhost:1234[5555]")); - assertNull(LiveTServerSet.find(servers, "localhost:1234[55755]")); + assertEquals(server1.instance, LiveTServerSet.find(servers, "default+localhost:1234+0")); + assertNull(LiveTServerSet.find(servers, "default+localhost:4321+0")); + assertEquals(server1.instance, LiveTServerSet.find(servers, "default+localhost:1234+5555")); + assertNull(LiveTServerSet.find(servers, "default+localhost:1234+55755")); } } diff --git a/server/base/src/test/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStoreTest.java b/server/base/src/test/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStoreTest.java index b9d8ddcbddd..eea918d98f6 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStoreTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStoreTest.java @@ -20,6 +20,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; 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,9 +31,11 @@ public class AbstractTabletStateStoreTest { private Ample.TabletMutator tabletMutator; - private final TServerInstance server1 = new TServerInstance("127.0.0.1:10000", 0); + private final TServerInstance server1 = + new TServerInstance(ServerIdUtil.tserver("127.0.0.1", 10000), 0); private final Location last1 = Location.last(server1); - private final TServerInstance server2 = new TServerInstance("127.0.0.2:10000", 1); + private final TServerInstance server2 = + new TServerInstance(ServerIdUtil.tserver("127.0.0.2", 10000), 1); private final Location last2 = Location.last(server2); @BeforeEach diff --git a/server/base/src/test/java/org/apache/accumulo/server/manager/state/TabletManagementParametersTest.java b/server/base/src/test/java/org/apache/accumulo/server/manager/state/TabletManagementParametersTest.java index 5a2e5e5a287..326d31060c7 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/manager/state/TabletManagementParametersTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/manager/state/TabletManagementParametersTest.java @@ -24,6 +24,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.ResourceGroupId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.FateId; @@ -44,8 +45,9 @@ public void testDeSer() { final Map parentUpgradeMap = Map.of(Ample.DataLevel.ROOT, true, Ample.DataLevel.USER, true, Ample.DataLevel.METADATA, true); final Set onlineTables = Set.of(TableId.of("1"), TableId.of("2"), TableId.of("3")); - final Set tservers = Set.of(new TServerInstance("127.0.0.1:10000", 0), - new TServerInstance("127.0.0.1:10001", 1)); + final Set tservers = + Set.of(new TServerInstance(ServerIdUtil.tserver("127.0.0.1", 10000), 0), + new TServerInstance(ServerIdUtil.tserver("127.0.0.1", 10001), 1)); final LiveTServerSet.LiveTServersSnapshot serverSnapshot = new LiveTServerSet.LiveTServersSnapshot(tservers, Map.of(ResourceGroupId.DEFAULT, tservers)); diff --git 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 index 0bab62dfbc2..e7e091f0e28 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 @@ -33,6 +33,7 @@ import java.util.UUID; import java.util.stream.Collectors; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.TableId; @@ -45,6 +46,7 @@ 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.TServerInstance; 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; @@ -104,13 +106,15 @@ private SortedMap createMetadataEntryKV(KeyExtent extent) { 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(CurrentLocationColumnFamily.NAME).qualifier("s001") + .put(new TServerInstance(ServerIdUtil.tserver("server1", 8555), "s001").serialize()); - mutation.at().family(LastLocationColumnFamily.NAME).qualifier("s000").put("server2:8555"); + mutation.at().family(LastLocationColumnFamily.NAME).qualifier("s000") + .put(new TServerInstance(ServerIdUtil.tserver("server2", 8555), "s000").serialize()); - LogEntry le1 = LogEntry.fromPath("localhost+8020/" + UUID.randomUUID()); + LogEntry le1 = LogEntry.fromPath("default+localhost+8020/" + UUID.randomUUID()); le1.addToMutation(mutation); - LogEntry le2 = LogEntry.fromPath("localhost+8020/" + UUID.randomUUID()); + LogEntry le2 = LogEntry.fromPath("default+localhost+8020/" + UUID.randomUUID()); le2.addToMutation(mutation); StoredTabletFile sf1 = diff --git a/server/base/src/test/java/org/apache/accumulo/server/manager/state/ZooTabletStateStoreTest.java b/server/base/src/test/java/org/apache/accumulo/server/manager/state/ZooTabletStateStoreTest.java index 7b52eaac19b..11cec785435 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/manager/state/ZooTabletStateStoreTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/manager/state/ZooTabletStateStoreTest.java @@ -23,6 +23,7 @@ import java.util.List; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.lock.ServiceLock; @@ -35,8 +36,6 @@ import org.easymock.EasyMock; import org.junit.jupiter.api.Test; -import com.google.common.net.HostAndPort; - public class ZooTabletStateStoreTest { @Test @@ -52,7 +51,7 @@ public void testZooTabletStateStore() throws DistributedStoreException { String sessionId = "this is my unique session data"; TServerInstance server = - new TServerInstance(HostAndPort.fromParts("127.0.0.1", 10000), sessionId); + new TServerInstance(ServerIdUtil.tserver("127.0.0.1", 10000), sessionId); KeyExtent notRoot = new KeyExtent(TableId.of("0"), null, null); final var assignmentList = List.of(new Assignment(notRoot, server, null)); diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/AdminTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/AdminTest.java index cc2ffaeeeb4..3aaac08af57 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/util/AdminTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/util/AdminTest.java @@ -45,7 +45,7 @@ import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.clientImpl.ClientContext; -import org.apache.accumulo.core.data.ResourceGroupId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.fate.FateId; @@ -55,6 +55,7 @@ import org.apache.accumulo.core.lock.ServiceLockPaths; 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.SelectedFiles; import org.apache.accumulo.core.metadata.schema.TabletMetadata; import org.apache.accumulo.core.metadata.schema.TabletOperationId; @@ -67,6 +68,8 @@ import org.easymock.EasyMock; import org.junit.jupiter.api.Test; +import com.google.common.net.HostAndPort; + public class AdminTest { @Test @@ -77,9 +80,10 @@ public void testQualifySessionId() throws KeeperException, InterruptedException String type = Constants.ZTSERVERS; String group = type + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME; String server = "localhost:12345"; + var hp = HostAndPort.fromString(server); final long session = 123456789L; - ServiceLockData sld1 = new ServiceLockData(UUID.randomUUID(), server, ThriftService.TABLET_SCAN, - ResourceGroupId.DEFAULT); + ServiceLockData sld1 = new ServiceLockData(UUID.randomUUID(), + ServerIdUtil.tserver(hp.getHost(), hp.getPort()), ThriftService.TABLET_SCAN); String serverPath = group + "/" + server; String validZLockEphemeralNode = "zlock#" + UUID.randomUUID() + "#0000000000"; @@ -99,7 +103,7 @@ public void testQualifySessionId() throws KeeperException, InterruptedException expect(ctx.getServerPaths()).andReturn(new ServiceLockPaths(zc)).anyTimes(); replay(ctx, zc); - assertEquals(server + "[" + Long.toHexString(session) + "]", + assertEquals(new TServerInstance(ServerIdUtil.tserver("localhost", 12345), session), Admin.qualifyWithZooKeeperSessionId(ctx, zc, server)); verify(ctx, zc); @@ -121,7 +125,8 @@ public void testCannotQualifySessionId() throws KeeperException, InterruptedExce replay(ctx, zc); // A server that isn't in ZooKeeper. Can't qualify it, should return the original - assertEquals(server, Admin.qualifyWithZooKeeperSessionId(ctx, zc, server)); + assertEquals(new TServerInstance(ServerIdUtil.tserver("localhost", 12345), "0"), + Admin.qualifyWithZooKeeperSessionId(ctx, zc, server)); verify(ctx, zc); } diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java index 70c738ff7a5..431a6ac4c28 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java @@ -81,18 +81,65 @@ public void validateMocks() { @Test public void testManagerHosts() throws Exception { replay(zooCache); - String lock1Name = "zlock#" + UUID.randomUUID() + "#0000000001"; - String lock2Name = "zlock#" + UUID.randomUUID() + "#0000000002"; - String lock3Name = "zlock#" + UUID.randomUUID() + "#0000000003"; - - String lock1data = - "{\"descriptors\":[{\"uuid\":\"6effb690-c29c-4e0b-92ff-f6b308385a42\",\"service\":\"MANAGER\",\"address\":\"localhost:9991\",\"group\":\"default\"}]}"; - String lock2Data = - "{\"descriptors\":[{\"uuid\":\"6effb690-c29c-4e0b-92ff-f6b308385a42\",\"service\":\"MANAGER\",\"address\":\"localhost:9992\",\"group\":\"default\"}]}"; - String lock3Data = - "{\"descriptors\":[{\"uuid\":\"6effb690-c29c-4e0b-92ff-f6b308385a42\",\"service\":\"MANAGER\",\"address\":\"hostA:9999\",\"group\":\"manager1\"}]}"; - - String lockPath = Constants.ZMANAGER_LOCK; + final String lock1Name = "zlock#" + UUID.randomUUID() + "#0000000001"; + final String lock2Name = "zlock#" + UUID.randomUUID() + "#0000000002"; + final String lock3Name = "zlock#" + UUID.randomUUID() + "#0000000003"; + + final String lock1data = """ + { + 'descriptors':[ + { + 'uuid':'6effb690-c29c-4e0b-92ff-f6b308385a42', + 'service':'MANAGER', + 'address': + { + 'type': 'MANAGER', + 'resourceGroup': 'default', + 'host': 'localhost', + 'port': 9991 + } + } + ] + } + """.replaceAll("'", "\""); + + final String lock2Data = """ + { + 'descriptors':[ + { + 'uuid':'6effb690-c29c-4e0b-92ff-f6b308385a42', + 'service':'MANAGER', + 'address': + { + 'type': 'MANAGER', + 'resourceGroup': 'default', + 'host': 'localhost', + 'port': 9992 + } + } + ] + } + """.replaceAll("'", "\""); + + final String lock3Data = """ + { + 'descriptors':[ + { + 'uuid':'6effb690-c29c-4e0b-92ff-f6b308385a42', + 'service':'MANAGER', + 'address': + { + 'type': 'MANAGER', + 'resourceGroup': 'manager1', + 'host': 'hostA', + 'port': 9999 + } + } + ] + } + """.replaceAll("'", "\""); + + final String lockPath = Constants.ZMANAGER_LOCK; expect(zooReader.getChildren(lockPath, null)) .andReturn(List.of(lock1Name, lock2Name, lock3Name)); expect(zooReader.getData(lockPath + "/" + lock1Name, null, null)) @@ -129,15 +176,46 @@ public void testManagerHosts() throws Exception { @Test public void testMonitorHosts() throws Exception { replay(zooCache); - String lock1Name = "zlock#" + UUID.randomUUID() + "#0000000001"; - String lock2Name = "zlock#" + UUID.randomUUID() + "#0000000002"; - - String host1 = - "{\"descriptors\":[{\"uuid\":\"87465459-9c8f-4f95-b4c6-ef3029030d05\",\"service\":\"NONE\",\"address\":\"hostA\",\"group\":\"default\"}]}"; - String host2 = - "{\"descriptors\":[{\"uuid\":\"87465459-9c8f-4f95-b4c6-ef3029030d05\",\"service\":\"NONE\",\"address\":\"hostB\",\"group\":\"default\"}]}"; - - String lockPath = Constants.ZMONITOR_LOCK; + final String lock1Name = "zlock#" + UUID.randomUUID() + "#0000000001"; + final String lock2Name = "zlock#" + UUID.randomUUID() + "#0000000002"; + + final String host1 = """ + { + 'descriptors':[ + { + 'uuid':'87465459-9c8f-4f95-b4c6-ef3029030d05', + 'service':'NONE', + 'address': + { + 'type': 'MONITOR', + 'resourceGroup': 'default', + 'host': 'hostA', + 'port': 9999 + } + } + ] + } + """.replaceAll("'", "\""); + + final String host2 = """ + { + 'descriptors':[ + { + 'uuid':'87465459-9c8f-4f95-b4c6-ef3029030d05', + 'service':'NONE', + 'address': + { + 'type': 'MONITOR', + 'resourceGroup': 'default', + 'host': 'hostB', + 'port': 9999 + } + } + ] + } + """.replaceAll("'", "\""); + + final String lockPath = Constants.ZMONITOR_LOCK; expect(zooReader.getChildren(lockPath, null)).andReturn(List.of(lock1Name, lock2Name)); expect(zooReader.getData(lockPath + "/" + lock1Name, null, null)) .andReturn(host1.getBytes(UTF_8)); @@ -154,12 +232,11 @@ public void testMonitorHosts() throws Exception { // expect sorted by name Map> hostByGroup = new TreeMap<>(); - hostByGroup.put("default", new TreeSet<>(List.of("hostA", "hostB"))); + hostByGroup.put("default", new TreeSet<>(List.of("hostA:9999", "hostB:9999"))); StatusSummary expected = new StatusSummary(ServiceStatusReport.ReportKey.MONITOR, Map.of("default", 2), hostByGroup, 0); - assertEquals(expected.hashCode(), status.hashCode()); assertEquals(expected.getDisplayName(), status.getDisplayName()); assertEquals(expected.getResourceGroups(), status.getResourceGroups()); assertEquals(expected.getServiceByGroups(), status.getServiceByGroups()); @@ -179,39 +256,191 @@ public void testTServerHosts() throws Exception { String host2 = "localhost:10000"; String host3 = "hostZ:9999"; - String lockData1 = - "{\"descriptors\":[{\"uuid\":\"e0a717f2-43a1-466c-aa91-8b33e20e17e5\",\"service\":\"TABLET_SCAN\",\"address\":\"" - + host1 - + "\",\"group\":\"default\"},{\"uuid\":\"e0a717f2-43a1-466c-aa91-8b33e20e17e5\",\"service\":\"CLIENT\",\"address\":\"" - + host1 - + "\",\"group\":\"default\"},{\"uuid\":\"e0a717f2-43a1-466c-aa91-8b33e20e17e5\",\"service\":\"TABLET_INGEST\",\"address\":\"" - + host1 - + "\",\"group\":\"default\"},{\"uuid\":\"e0a717f2-43a1-466c-aa91-8b33e20e17e5\",\"service\":\"TABLET_MANAGEMENT\",\"address\":\"" - + host1 - + "\",\"group\":\"default\"},{\"uuid\":\"e0a717f2-43a1-466c-aa91-8b33e20e17e5\",\"service\":\"TSERV\",\"address\":\"" - + host1 + "\",\"group\":\"default\"}]}\n"; - String lockData2 = - "{\"descriptors\":[{\"uuid\":\"d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32\",\"service\":\"TABLET_SCAN\",\"address\":\"" - + host2 - + "\",\"group\":\"default\"},{\"uuid\":\"d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32\",\"service\":\"TABLET_MANAGEMENT\",\"address\":\"" - + host2 - + "\",\"group\":\"default\"},{\"uuid\":\"d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32\",\"service\":\"CLIENT\",\"address\":\"" - + host2 - + "\",\"group\":\"default\"},{\"uuid\":\"d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32\",\"service\":\"TSERV\",\"address\":\"" - + host2 - + "\",\"group\":\"default\"},{\"uuid\":\"d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32\",\"service\":\"TABLET_INGEST\",\"address\":\"" - + host2 + "\",\"group\":\"default\"}]}"; - String lockData3 = - "{\"descriptors\":[{\"uuid\":\"d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32\",\"service\":\"TABLET_SCAN\",\"address\":\"" - + host3 - + "\",\"group\":\"default\"},{\"uuid\":\"d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32\",\"service\":\"TABLET_MANAGEMENT\",\"address\":\"" - + host3 - + "\",\"group\":\"default\"},{\"uuid\":\"d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32\",\"service\":\"CLIENT\",\"address\":\"" - + host3 - + "\",\"group\":\"default\"},{\"uuid\":\"d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32\",\"service\":\"TSERV\",\"address\":\"" - + host3 - + "\",\"group\":\"default\"},{\"uuid\":\"d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32\",\"service\":\"TABLET_INGEST\",\"address\":\"" - + host3 + "\",\"group\":\"default\"}]}"; + final String lockData1 = """ + { + 'descriptors':[ + { + 'uuid':'e0a717f2-43a1-466c-aa91-8b33e20e17e5', + 'service':'TABLET_SCAN', + 'address': + { + 'type': 'TABLET_SERVER', + 'resourceGroup': 'default', + 'host': 'localhost', + 'port': 9997 + } + }, + { + 'uuid':'e0a717f2-43a1-466c-aa91-8b33e20e17e5', + 'service':'CLIENT', + 'address': + { + 'type': 'TABLET_SERVER', + 'resourceGroup': 'default', + 'host': 'localhost', + 'port': 9997 + } + }, + { + 'uuid':'e0a717f2-43a1-466c-aa91-8b33e20e17e5', + 'service':'TABLET_INGEST', + 'address': + { + 'type': 'TABLET_SERVER', + 'resourceGroup': 'default', + 'host': 'localhost', + 'port': 9997 + } + }, + { + 'uuid':'e0a717f2-43a1-466c-aa91-8b33e20e17e5', + 'service':'TABLET_MANAGEMENT', + 'address': + { + 'type': 'TABLET_SERVER', + 'resourceGroup': 'default', + 'host': 'localhost', + 'port': 9997 + } + }, + { + 'uuid':'e0a717f2-43a1-466c-aa91-8b33e20e17e5', + 'service':'TSERV', + 'address': + { + 'type': 'TABLET_SERVER', + 'resourceGroup': 'default', + 'host': 'localhost', + 'port': 9997 + } + } + ] + } + """.replaceAll("'", "\""); + + final String lockData2 = """ + { + 'descriptors':[ + { + 'uuid':'d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32', + 'service':'TABLET_SCAN', + 'address': + { + 'type': 'TABLET_SERVER', + 'resourceGroup': 'default', + 'host': 'localhost', + 'port': 10000 + } + }, + { + 'uuid':'d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32', + 'service':'CLIENT', + 'address': + { + 'type': 'TABLET_SERVER', + 'resourceGroup': 'default', + 'host': 'localhost', + 'port': 10000 + } + }, + { + 'uuid':'d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32', + 'service':'TABLET_INGEST', + 'address': + { + 'type': 'TABLET_SERVER', + 'resourceGroup': 'default', + 'host': 'localhost', + 'port': 10000 + } + }, + { + 'uuid':'d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32', + 'service':'TABLET_MANAGEMENT', + 'address': + { + 'type': 'TABLET_SERVER', + 'resourceGroup': 'default', + 'host': 'localhost', + 'port': 10000 + } + }, + { + 'uuid':'d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32', + 'service':'TSERV', + 'address': + { + 'type': 'TABLET_SERVER', + 'resourceGroup': 'default', + 'host': 'localhost', + 'port': 10000 + } + } + ] + } + """.replaceAll("'", "\""); + + final String lockData3 = """ + { + 'descriptors':[ + { + 'uuid':'d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32', + 'service':'TABLET_SCAN', + 'address': + { + 'type': 'TABLET_SERVER', + 'resourceGroup': 'default', + 'host': 'hostZ', + 'port': 9999 + } + }, + { + 'uuid':'d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32', + 'service':'CLIENT', + 'address': + { + 'type': 'TABLET_SERVER', + 'resourceGroup': 'default', + 'host': 'hostZ', + 'port': 9999 + } + }, + { + 'uuid':'d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32', + 'service':'TABLET_INGEST', + 'address': + { + 'type': 'TABLET_SERVER', + 'resourceGroup': 'default', + 'host': 'hostZ', + 'port': 9999 + } + }, + { + 'uuid':'d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32', + 'service':'TABLET_MANAGEMENT', + 'address': + { + 'type': 'TABLET_SERVER', + 'resourceGroup': 'default', + 'host': 'hostZ', + 'port': 9999 + } + }, + { + 'uuid':'d0e29f70-1eb5-4dc5-9ad6-2466ab56ea32', + 'service':'TSERV', + 'address': + { + 'type': 'TABLET_SERVER', + 'resourceGroup': 'default', + 'host': 'hostZ', + 'port': 9999 + } + } + ] + } + """.replaceAll("'", "\""); String basePath = Constants.ZTSERVERS; expect(zooCache.getChildren(basePath)) @@ -271,40 +500,135 @@ public void testTServerHosts() throws Exception { public void testScanServerHosts() throws Exception { replay(zooReader); UUID uuid1 = UUID.randomUUID(); - String lock1Name = "zlock#" + uuid1 + "#0000000001"; + final String lock1Name = "zlock#" + uuid1 + "#0000000001"; UUID uuid2 = UUID.randomUUID(); - String lock2Name = "zlock#" + uuid2 + "#0000000022"; + final String lock2Name = "zlock#" + uuid2 + "#0000000022"; UUID uuid3 = UUID.randomUUID(); - String lock3Name = "zlock#" + uuid3 + "#0000000033"; - String lock4Name = "zlock#" + uuid3 + "#0000000044"; - - String host1 = "host1:8080"; - String host2 = "host2:9090"; - String host3 = "host3:9091"; - String host4 = "host4:9091"; - - String lockData1 = - "{\"descriptors\":[{\"uuid\":\"f408fed7-ce93-40d2-8e60-63e8a3daf416\",\"service\":\"TABLET_SCAN\",\"address\":\"" - + host1 - + "\",\"group\":\"sg1\"},{\"uuid\":\"f408fed7-ce93-40d2-8e60-63e8a3daf416\",\"service\":\"CLIENT\",\"address\":\"" - + host1 + "\",\"group\":\"sg1\"}]}"; - String lockData2 = - "{\"descriptors\":[{\"uuid\":\"f408fed7-ce93-40d2-8e60-63e8a3daf416\",\"service\":\"TABLET_SCAN\",\"address\":\"" - + host2 - + "\",\"group\":\"default\"},{\"uuid\":\"f408fed7-ce93-40d2-8e60-63e8a3daf416\",\"service\":\"CLIENT\",\"address\":\"" - + host2 + "\",\"group\":\"default\"}]}"; - String lockData3 = - "{\"descriptors\":[{\"uuid\":\"f408fed7-ce93-40d2-8e60-63e8a3daf416\",\"service\":\"TABLET_SCAN\",\"address\":\"" - + host3 - + "\",\"group\":\"sg1\"},{\"uuid\":\"f408fed7-ce93-40d2-8e60-63e8a3daf416\",\"service\":\"CLIENT\",\"address\":\"" - + host3 + "\",\"group\":\"sg1\"}]}"; - String lockData4 = - "{\"descriptors\":[{\"uuid\":\"f408fed7-ce93-40d2-8e60-63e8a3daf416\",\"service\":\"TABLET_SCAN\",\"address\":\"" - + host4 - + "\",\"group\":\"default\"},{\"uuid\":\"f408fed7-ce93-40d2-8e60-63e8a3daf416\",\"service\":\"CLIENT\",\"address\":\"" - + host4 + "\",\"group\":\"default\"}]}"; - - String lockPath = Constants.ZSSERVERS; + final String lock3Name = "zlock#" + uuid3 + "#0000000033"; + final String lock4Name = "zlock#" + uuid3 + "#0000000044"; + + final String host1 = "host1:8080"; + final String host2 = "host2:9090"; + final String host3 = "host3:9091"; + final String host4 = "host4:9091"; + + final String lockData1 = """ + { + 'descriptors':[ + { + 'uuid':'f408fed7-ce93-40d2-8e60-63e8a3daf416', + 'service':'TABLET_SCAN', + 'address': + { + 'type': 'SCAN_SERVER', + 'resourceGroup': 'sg1', + 'host': 'host1', + 'port': 8080 + } + }, + { + 'uuid':'f408fed7-ce93-40d2-8e60-63e8a3daf416', + 'service':'CLIENT', + 'address': + { + 'type': 'SCAN_SERVER', + 'resourceGroup': 'sg1', + 'host': 'host1', + 'port': 8080 + } + } + ] + } + """.replaceAll("'", "\""); + + final String lockData2 = """ + { + 'descriptors':[ + { + 'uuid':'f408fed7-ce93-40d2-8e60-63e8a3daf416', + 'service':'TABLET_SCAN', + 'address': + { + 'type': 'SCAN_SERVER', + 'resourceGroup': 'default', + 'host': 'host2', + 'port': 9090 + } + }, + { + 'uuid':'f408fed7-ce93-40d2-8e60-63e8a3daf416', + 'service':'CLIENT', + 'address': + { + 'type': 'SCAN_SERVER', + 'resourceGroup': 'default', + 'host': 'host2', + 'port': 9090 + } + } + ] + } + """.replaceAll("'", "\""); + + final String lockData3 = """ + { + 'descriptors':[ + { + 'uuid':'f408fed7-ce93-40d2-8e60-63e8a3daf416', + 'service':'TABLET_SCAN', + 'address': + { + 'type': 'SCAN_SERVER', + 'resourceGroup': 'sg1', + 'host': 'host3', + 'port': 9091 + } + }, + { + 'uuid':'f408fed7-ce93-40d2-8e60-63e8a3daf416', + 'service':'CLIENT', + 'address': + { + 'type': 'SCAN_SERVER', + 'resourceGroup': 'sg1', + 'host': 'host3', + 'port': 9091 + } + } + ] + } + """.replaceAll("'", "\""); + + final String lockData4 = """ + { + 'descriptors':[ + { + 'uuid':'f408fed7-ce93-40d2-8e60-63e8a3daf416', + 'service':'TABLET_SCAN', + 'address': + { + 'type': 'SCAN_SERVER', + 'resourceGroup': 'default', + 'host': 'host4', + 'port': 9091 + } + }, + { + 'uuid':'f408fed7-ce93-40d2-8e60-63e8a3daf416', + 'service':'CLIENT', + 'address': + { + 'type': 'SCAN_SERVER', + 'resourceGroup': 'default', + 'host': 'host4', + 'port': 9091 + } + } + ] + } + """.replaceAll("'", "\""); + + final String lockPath = Constants.ZSSERVERS; expect(zooCache.getChildren(lockPath)) .andReturn(List.of(Constants.DEFAULT_RESOURCE_GROUP_NAME, "sg1")); expect(zooCache.getChildren(lockPath + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME)) @@ -355,24 +679,87 @@ public void testCompactorStatus() throws Exception { replay(zooReader); UUID uuid1 = UUID.randomUUID(); - String lock1Name = "zlock#" + uuid1 + "#0000000001"; + final String lock1Name = "zlock#" + uuid1 + "#0000000001"; UUID uuid2 = UUID.randomUUID(); - String lock2Name = "zlock#" + uuid2 + "#0000000022"; + final String lock2Name = "zlock#" + uuid2 + "#0000000022"; UUID uuid3 = UUID.randomUUID(); - String lock3Name = "zlock#" + uuid3 + "#0000000033"; + final String lock3Name = "zlock#" + uuid3 + "#0000000033"; UUID uuid4 = UUID.randomUUID(); - String lock4Name = "zlock#" + uuid4 + "#0000000044"; - - String lock1data = - "{\"descriptors\":[{\"uuid\":\"6effb690-c29c-4e0b-92ff-f6b308385a42\",\"service\":\"COMPACTOR\",\"address\":\"hostA:8080\",\"group\":\"q1\"}]}"; - String lock2data = - "{\"descriptors\":[{\"uuid\":\"6effb690-c29c-4e0b-92ff-f6b308385a42\",\"service\":\"COMPACTOR\",\"address\":\"hostC:8081\",\"group\":\"q1\"}]}"; - String lock3data = - "{\"descriptors\":[{\"uuid\":\"6effb690-c29c-4e0b-92ff-f6b308385a42\",\"service\":\"COMPACTOR\",\"address\":\"hostB:9090\",\"group\":\"q2\"}]}"; - String lock4data = - "{\"descriptors\":[{\"uuid\":\"6effb690-c29c-4e0b-92ff-f6b308385a42\",\"service\":\"COMPACTOR\",\"address\":\"hostD:9091\",\"group\":\"q2\"}]}"; - - String lockPath = Constants.ZCOMPACTORS; + final String lock4Name = "zlock#" + uuid4 + "#0000000044"; + + final String lock1data = """ + { + 'descriptors':[ + { + 'uuid':'6effb690-c29c-4e0b-92ff-f6b308385a42', + 'service':'COMPACTOR', + 'address': + { + 'type': 'COMPACTOR', + 'resourceGroup': 'q1', + 'host': 'hostA', + 'port': 8080 + } + } + ] + } + """.replaceAll("'", "\""); + + final String lock2data = """ + { + 'descriptors':[ + { + 'uuid':'6effb690-c29c-4e0b-92ff-f6b308385a42', + 'service':'COMPACTOR', + 'address': + { + 'type': 'COMPACTOR', + 'resourceGroup': 'q1', + 'host': 'hostC', + 'port': 8081 + } + } + ] + } + """.replaceAll("'", "\""); + + final String lock3data = """ + { + 'descriptors':[ + { + 'uuid':'6effb690-c29c-4e0b-92ff-f6b308385a42', + 'service':'COMPACTOR', + 'address': + { + 'type': 'COMPACTOR', + 'resourceGroup': 'q2', + 'host': 'hostB', + 'port': 9090 + } + } + ] + } + """.replaceAll("'", "\""); + + final String lock4data = """ + { + 'descriptors':[ + { + 'uuid':'6effb690-c29c-4e0b-92ff-f6b308385a42', + 'service':'COMPACTOR', + 'address': + { + 'type': 'COMPACTOR', + 'resourceGroup': 'q2', + 'host': 'hostD', + 'port': 9091 + } + } + ] + } + """.replaceAll("'", "\""); + + final String lockPath = Constants.ZCOMPACTORS; expect(zooCache.getChildren(lockPath)).andReturn(List.of("q1", "q2", "q3")); expect(zooCache.getChildren(lockPath + "/q1")).andReturn(List.of("hostA:8080", "hostC:8081")); expect(zooCache.getChildren(lockPath + "/q2")).andReturn(List.of("hostB:9090", "hostD:9091")); @@ -409,21 +796,50 @@ public void testCompactorStatus() throws Exception { public void testGcHosts() throws Exception { replay(zooCache); - String lockPath = ZGC_LOCK; + final String lockPath = ZGC_LOCK; UUID uuid1 = UUID.randomUUID(); - String lock1Name = "zlock#" + uuid1 + "#0000000001"; + final String lock1Name = "zlock#" + uuid1 + "#0000000001"; UUID uuid2 = UUID.randomUUID(); - String lock2Name = "zlock#" + uuid2 + "#0000000022"; - - String host1 = "host1:8080"; - String host2 = "host2:9090"; - - String lockData1 = - "{\"descriptors\":[{\"uuid\":\"5c901352-b027-4f78-8ee1-05ae163fbb0e\",\"service\":\"GC\",\"address\":\"" - + host2 + "\",\"group\":\"default\"}]}"; - String lockData2 = - "{\"descriptors\":[{\"uuid\":\"5c901352-b027-4f78-8ee1-05ae163fbb0e\",\"service\":\"GC\",\"address\":\"" - + host1 + "\",\"group\":\"gc1\"}]}"; + final String lock2Name = "zlock#" + uuid2 + "#0000000022"; + + final String host1 = "host1:8080"; + final String host2 = "host2:9090"; + + final String lockData1 = """ + { + 'descriptors':[ + { + 'uuid':'5c901352-b027-4f78-8ee1-05ae163fbb0e', + 'service':'GC', + 'address': + { + 'type': 'GARBAGE_COLLECTOR', + 'resourceGroup': 'default', + 'host': 'host2', + 'port': 9090 + } + } + ] + } + """.replaceAll("'", "\""); + + final String lockData2 = """ + { + 'descriptors':[ + { + 'uuid':'5c901352-b027-4f78-8ee1-05ae163fbb0e', + 'service':'GC', + 'address': + { + 'type': 'GARBAGE_COLLECTOR', + 'resourceGroup': 'gc1', + 'host': 'host1', + 'port': 8080 + } + } + ] + } + """.replaceAll("'", "\""); expect(zooReader.getChildren(lockPath, null)).andReturn(List.of(lock1Name, lock2Name)); expect(zooReader.getData(lockPath + "/" + lock1Name, null, null)) @@ -454,19 +870,48 @@ public void testGcHosts() throws Exception { @Test public void zkNodeDeletedTest() throws Exception { replay(zooCache); - String lock1Name = "zlock#" + UUID.randomUUID() + "#0000000001"; - String lock2Name = "zlock#" + UUID.randomUUID() + "#0000000002"; - String lock3Name = "zlock#" + UUID.randomUUID() + "#0000000003"; - - String host2 = "localhost:9992"; - String host3 = "hostA:9999"; - - String lock2Data = - "{\"descriptors\":[{\"uuid\":\"6effb690-c29c-4e0b-92ff-f6b308385a42\",\"service\":\"MANAGER\",\"address\":\"" - + host2 + "\",\"group\":\"default\"}]}"; - String lock3Data = - "{\"descriptors\":[{\"uuid\":\"6effb690-c29c-4e0b-92ff-f6b308385a42\",\"service\":\"MANAGER\",\"address\":\"" - + host3 + "\",\"group\":\"manager1\"}]}"; + final String lock1Name = "zlock#" + UUID.randomUUID() + "#0000000001"; + final String lock2Name = "zlock#" + UUID.randomUUID() + "#0000000002"; + final String lock3Name = "zlock#" + UUID.randomUUID() + "#0000000003"; + + final String host2 = "localhost:9992"; + final String host3 = "hostA:9999"; + + final String lock2Data = """ + { + 'descriptors':[ + { + 'uuid':'6effb690-c29c-4e0b-92ff-f6b308385a42', + 'service':'MANAGER', + 'address': + { + 'type': 'MANAGER', + 'resourceGroup': 'default', + 'host': 'localhost', + 'port': 9992 + } + } + ] + } + """.replaceAll("'", "\""); + + final String lock3Data = """ + { + 'descriptors':[ + { + 'uuid':'6effb690-c29c-4e0b-92ff-f6b308385a42', + 'service':'MANAGER', + 'address': + { + 'type': 'MANAGER', + 'resourceGroup': 'manager1', + 'host': 'hostA', + 'port': 9999 + } + } + ] + } + """.replaceAll("'", "\""); String lockPath = Constants.ZMANAGER_LOCK; expect(zooReader.getChildren(lockPath, null)) diff --git a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java index 9772d4bd9f6..a97d775b8a2 100644 --- a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java +++ b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java @@ -365,12 +365,13 @@ protected void checkIfCanceled() { * @throws KeeperException zookeeper error * @throws InterruptedException thread interrupted */ - protected void announceExistence(HostAndPort clientAddress) + protected void announceExistence(ServerId clientAddress) throws KeeperException, InterruptedException { final ZooReaderWriter zoo = getContext().getZooSession().asReaderWriter(); final ServiceLockPath path = - getContext().getServerPaths().createCompactorPath(getResourceGroup(), clientAddress); + getContext().getServerPaths().createCompactorPath(getResourceGroup(), + HostAndPort.fromParts(clientAddress.getHost(), clientAddress.getPort())); ServiceLockSupport.createNonHaServiceLockPath(Type.COMPACTOR, zoo, path); compactorLock = new ServiceLock(getContext().getZooSession(), path, compactorId); LockWatcher lw = new ServiceLockWatcher(Type.COMPACTOR, () -> getShutdownComplete().get(), @@ -383,8 +384,8 @@ protected void announceExistence(HostAndPort clientAddress) ServiceDescriptors descriptors = new ServiceDescriptors(); for (ThriftService svc : new ThriftService[] {ThriftService.CLIENT, ThriftService.COMPACTOR}) { - descriptors.addService(new ServiceDescriptor(compactorId, svc, - ExternalCompactionUtil.getHostPortString(clientAddress), this.getResourceGroup())); + descriptors + .addService(new ServiceDescriptor(compactorId, svc, this.getAdvertiseAddress())); } if (compactorLock.tryLock(lw, new ServiceLockData(descriptors))) { @@ -540,7 +541,6 @@ protected TNextCompactionJob getNextJob(Supplier uuid) throws RetriesExcee getConfiguration().getTimeInMillis(Property.COMPACTOR_MIN_JOB_WAIT_TIME); final long maxWaitTime = getConfiguration().getTimeInMillis(Property.COMPACTOR_MAX_JOB_WAIT_TIME); - RetryableThriftCall nextJobThriftCall = new RetryableThriftCall<>(startingWaitTime, maxWaitTime, 0, () -> { Client coordinatorClient = getCoordinatorClient(); @@ -550,7 +550,7 @@ protected TNextCompactionJob getNextJob(Supplier uuid) throws RetriesExcee currentCompactionId.set(eci); return coordinatorClient.getCompactionJob(TraceUtil.traceInfo(), getContext().rpcCreds(), this.getResourceGroup().canonical(), - ExternalCompactionUtil.getHostPortString(getAdvertiseAddress()), eci.toString()); + getAdvertiseAddress().toHostPortString(), eci.toString()); } catch (Exception e) { currentCompactionId.set(null); throw e; @@ -753,9 +753,9 @@ protected long getWaitTimeBetweenCompactionChecks(int numCompactors) { return sleepTime; } - protected Collection getServiceTags(HostAndPort clientAddress) { + protected Collection getServiceTags(ServerId clientAddress) { return MetricsInfo.serviceTags(getContext().getInstanceName(), getApplicationName(), - clientAddress, getResourceGroup()); + clientAddress); } private void performFailureProcessing(ConsecutiveErrorHistory errorHistory) @@ -808,7 +808,7 @@ public void run() { } catch (UnknownHostException e1) { throw new RuntimeException("Failed to start the compactor client service", e1); } - final HostAndPort clientAddress = getAdvertiseAddress(); + final ServerId clientAddress = getAdvertiseAddress(); try { announceExistence(clientAddress); diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java index a60c955da58..02175086075 100644 --- a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java +++ b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java @@ -225,6 +225,7 @@ private long removeTabletServerMarkers(Map uidMap, try { for (Entry entry : uidMap.entrySet()) { walMarker.removeWalMarker(entry.getValue(), entry.getKey()); + result++; } } catch (Exception ex) { throw new RuntimeException(ex); diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java index bc5edfa1681..8642fa2a4b9 100644 --- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java +++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java @@ -172,6 +172,7 @@ public void run() { throw new RuntimeException("Failed to start the gc client service", e1); } + ServerId address = getAdvertiseAddress(); try { getZooLock(getAdvertiseAddress()); } catch (Exception ex) { @@ -183,8 +184,8 @@ public void run() { MetricsInfo metricsInfo = getContext().getMetricsInfo(); metricsInfo.addMetricsProducers(this, new GcMetrics(this)); - metricsInfo.init(MetricsInfo.serviceTags(getContext().getInstanceName(), getApplicationName(), - getAdvertiseAddress(), getResourceGroup())); + metricsInfo.init( + MetricsInfo.serviceTags(getContext().getInstanceName(), getApplicationName(), address)); try { long delay = getStartDelay(); log.debug("Sleeping for {} milliseconds before beginning garbage collection cycles", delay); @@ -387,7 +388,7 @@ boolean moveToTrash(Path path) throws IOException { } } - private void getZooLock(HostAndPort addr) throws KeeperException, InterruptedException { + private void getZooLock(ServerId addr) throws KeeperException, InterruptedException { var path = getContext().getServerPaths().createGarbageCollectorPath(); UUID zooLockUUID = UUID.randomUUID(); @@ -396,8 +397,7 @@ private void getZooLock(HostAndPort addr) throws KeeperException, InterruptedExc new HAServiceLockWatcher(Type.GARBAGE_COLLECTOR, () -> getShutdownComplete().get()); while (true) { - gcLock.lock(gcLockWatcher, new ServiceLockData(zooLockUUID, addr.toString(), ThriftService.GC, - this.getResourceGroup())); + gcLock.lock(gcLockWatcher, new ServiceLockData(zooLockUUID, addr, ThriftService.GC)); gcLockWatcher.waitForChange(); diff --git a/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java b/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java index 896afcc2c31..8222228bb35 100644 --- a/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java +++ b/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java @@ -31,6 +31,8 @@ import java.util.stream.Stream; import org.apache.accumulo.core.client.admin.TabletAvailability; +import org.apache.accumulo.core.client.admin.servers.ServerId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.dataImpl.KeyExtent; @@ -52,8 +54,9 @@ public class GarbageCollectWriteAheadLogsTest { - private final TServerInstance server1 = new TServerInstance("localhost:1234[SESSION]"); - private final TServerInstance server2 = new TServerInstance("localhost:1234[OTHERSESS]"); + private final ServerId tserver = ServerIdUtil.tserver("localhost", 1234); + private final TServerInstance server1 = new TServerInstance(tserver, "SESSION"); + private final TServerInstance server2 = new TServerInstance(tserver, "OTHERSESS"); private final UUID id = UUID.randomUUID(); private final Map> markers = Collections.singletonMap(server1, Collections.singletonList(id)); diff --git a/server/manager/pom.xml b/server/manager/pom.xml index 0c7e154321b..cb52ea5681f 100644 --- a/server/manager/pom.xml +++ b/server/manager/pom.xml @@ -40,6 +40,10 @@ auto-service true + + com.google.code.gson + gson + com.google.guava guava 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 3eb2d5437f9..ea59f819a7f 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 @@ -1137,7 +1137,7 @@ boolean canSuspendTablets() { producers.addAll(managerMetrics.getProducers(this)); metricsInfo.addMetricsProducers(producers.toArray(new MetricsProducer[0])); metricsInfo.init(MetricsInfo.serviceTags(getContext().getInstanceName(), getApplicationName(), - getAdvertiseAddress(), getResourceGroup())); + getAdvertiseAddress())); ThreadPools.watchCriticalScheduledTask(context.getScheduledExecutor() .scheduleWithFixedDelay(() -> ScanServerMetadataEntries.clean(context), 10, 10, MINUTES)); @@ -1177,7 +1177,7 @@ boolean canSuspendTablets() { // Now that the Manager is up, start the ThriftServer Objects.requireNonNull(getThriftServerAddress(), "Thrift Server Address should not be null"); getThriftServerAddress().startThriftServer("Manager Client Service Handler"); - log.info("Started Manager client service at {}", getAdvertiseAddress()); + log.info("Started Manager client service at {}", getAdvertiseAddress().toHostPortString()); // Replace the ServiceLockData information in the Manager lock node in ZooKeeper. // This advertises the address that clients can use to connect to the Manager @@ -1187,8 +1187,7 @@ boolean canSuspendTablets() { ServiceDescriptors descriptors = new ServiceDescriptors(); for (ThriftService svc : new ThriftService[] {ThriftService.MANAGER, ThriftService.COORDINATOR, ThriftService.FATE}) { - descriptors.addService(new ServiceDescriptor(uuid, svc, getAdvertiseAddress().toString(), - this.getResourceGroup())); + descriptors.addService(new ServiceDescriptor(uuid, svc, getAdvertiseAddress())); } sld = new ServiceLockData(descriptors); @@ -1383,8 +1382,8 @@ private ServiceLockData getManagerLock(final ServiceLockPath zManagerLoc) // and the address set to 0.0.0.0. When the lock is acquired (could be // waiting to due an HA-pair), then the Manager startup process begins // and the lock service descriptors are updated with the advertise address - descriptors.addService(new ServiceDescriptor(zooLockUUID, ThriftService.NONE, - ConfigOpts.BIND_ALL_ADDRESSES, this.getResourceGroup())); + descriptors + .addService(new ServiceDescriptor(zooLockUUID, ThriftService.NONE, getAdvertiseAddress())); ServiceLockData sld = new ServiceLockData(descriptors); managerLock = new ServiceLock(zooKeeper, zManagerLoc, zooLockUUID); HAServiceLockWatcher managerLockWatcher = @@ -1424,7 +1423,7 @@ public void update(LiveTServerSet current, Set deleted, if (!added.isEmpty()) { log.info("New servers: {}", added); for (TServerInstance up : added) { - obit.delete(up.getHostPort()); + obit.delete(up.getServer().toHostPortString()); } } for (TServerInstance dead : deleted) { @@ -1433,7 +1432,7 @@ public void update(LiveTServerSet current, Set deleted, cause = "clean shutdown"; // maybe an incorrect assumption } if (!getManagerGoalState().equals(ManagerGoalState.CLEAN_STOP)) { - obit.post(dead.getHostPort(), cause); + obit.post(dead.getServer().toHostPortString(), cause); } } @@ -1467,13 +1466,13 @@ private static void cleanListByHostAndPort(Collection badServer while (badIter.hasNext()) { TServerInstance bad = badIter.next(); for (TServerInstance add : added) { - if (bad.getHostPort().equals(add.getHostPort())) { + if (bad.getServer().toHostPortString().equals(add.getServer().toHostPortString())) { badIter.remove(); break; } } for (TServerInstance del : deleted) { - if (bad.getHostPort().equals(del.getHostPort())) { + if (bad.getServer().toHostPortString().equals(del.getServer().toHostPortString())) { badIter.remove(); break; } @@ -1556,7 +1555,8 @@ public ManagerMonitorInfo getManagerMonitorInfo() { result.badTServers = new HashMap<>(); synchronized (badServers) { for (TServerInstance bad : badServers.keySet()) { - result.badTServers.put(bad.getHostPort(), TabletServerState.UNRESPONSIVE.getId()); + result.badTServers.put(bad.getServer().toHostPortString(), + TabletServerState.UNRESPONSIVE.getId()); } } result.state = getManagerState(); @@ -1565,7 +1565,7 @@ public ManagerMonitorInfo getManagerMonitorInfo() { result.serversShuttingDown = new HashSet<>(); synchronized (serversToShutdown) { for (TServerInstance server : serversToShutdown) { - result.serversShuttingDown.add(server.getHostPort()); + result.serversShuttingDown.add(server.getServer().toHostPortString()); } } DeadServerList obit = new DeadServerList(getContext()); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java index bab48d06588..b53924ca6d3 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java @@ -332,9 +332,7 @@ public void shutdownTabletServer(TInfo info, TCredentials c, String tabletServer String msg = "Shutdown tserver " + tabletServer; fate.seedTransaction(Fate.FateOperation.SHUTDOWN_TSERVER, fateId, - new TraceRepo<>( - new ShutdownTServer(doomed, manager.tserverSet.getResourceGroup(doomed), force)), - false, msg); + new TraceRepo<>(new ShutdownTServer(doomed, force)), false, msg); fate.waitForCompletion(fateId); fate.delete(fateId); @@ -350,7 +348,7 @@ public void tabletServerStopping(TInfo tinfo, TCredentials credentials, String t SecurityErrorCode.PERMISSION_DENIED); } log.info("Tablet Server {} has reported it's shutting down", tabletServer); - var tserver = new TServerInstance(tabletServer); + var tserver = TServerInstance.deserialize(tabletServer); if (manager.shutdownTServer(tserver)) { // If there is an exception seeding the fate tx this should cause the RPC to fail which should // cause the tserver to halt. Because of that not making an attempt to handle failure here. @@ -359,8 +357,7 @@ public void tabletServerStopping(TInfo tinfo, TCredentials credentials, String t String msg = "Shutdown tserver " + tabletServer; fate.seedTransaction(Fate.FateOperation.SHUTDOWN_TSERVER, tid, - new TraceRepo<>(new ShutdownTServer(tserver, ResourceGroupId.of(resourceGroup), false)), - true, msg); + new TraceRepo<>(new ShutdownTServer(tserver, false)), true, msg); } } @@ -684,7 +681,7 @@ public List getActiveTservers(TInfo tinfo, TCredentials credentials) Set tserverInstances = manager.onlineTabletServers(); List servers = new ArrayList<>(); for (TServerInstance tserverInstance : tserverInstances) { - servers.add(tserverInstance.getHostPort()); + servers.add(tserverInstance.getServer().toHostPortString()); } return servers; 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 29eb475c8e4..2a6ced75e81 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 @@ -50,6 +50,7 @@ import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.admin.TabletAvailability; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Range; @@ -107,7 +108,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSortedSet; import com.google.common.collect.Sets; -import com.google.common.net.HostAndPort; abstract class TabletGroupWatcher extends AccumuloDaemonThread { @@ -855,11 +855,11 @@ private void hostUnassignedTablet(TabletLists tLists, KeyExtent tablet, } static TServerInstance findServerIgnoringSession(SortedMap servers, - HostAndPort server) { + ServerId server) { var tail = servers.tailMap(new TServerInstance(server, 0L)).keySet().iterator(); if (tail.hasNext()) { TServerInstance found = tail.next(); - if (found.getHostAndPort().equals(server)) { + if (found.getServer().equals(server)) { return found; } } @@ -873,7 +873,7 @@ private void hostSuspendedTablet(TabletLists tLists, TabletMetadata tm, Location < tableConf.getTimeInMillis(Property.TABLE_SUSPEND_DURATION)) { // Tablet is suspended. See if its tablet server is back. TServerInstance returnInstance = - findServerIgnoringSession(tLists.destinations, tm.getSuspend().server); + findServerIgnoringSession(tLists.destinations, tm.getSuspend().server.getServer()); // Old tablet server is back. Return this tablet to its previous owner. if (returnInstance != null) { @@ -934,7 +934,7 @@ private void logIncorrectTabletLocations(TabletMetadata tabletMetadata) { tabletMetadata.getExtent()); } else { for (Map.Entry entry : locations.entrySet()) { - TServerInstance alive = manager.tserverSet.find(entry.getValue().toString()); + TServerInstance alive = TServerInstance.deserialize(entry.getValue().toString()); Manager.log.debug("Saw duplicate location key:{} value:{} alive:{} ", entry.getKey(), entry.getValue(), alive != null); } diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java index 5e033273d5d..65e22c0b5aa 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java @@ -69,6 +69,7 @@ import org.apache.accumulo.core.client.admin.CompactionConfig; import org.apache.accumulo.core.client.admin.compaction.CompactableFile; import org.apache.accumulo.core.client.admin.servers.ServerId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode; import org.apache.accumulo.core.clientImpl.thrift.TInfo; import org.apache.accumulo.core.clientImpl.thrift.TableOperation; @@ -242,7 +243,7 @@ static FailureCounts incrementSuccess(Object key, FailureCounts counts) { private final ConcurrentHashMap failingQueues = new ConcurrentHashMap<>(); - private final ConcurrentHashMap failingCompactors = + private final ConcurrentHashMap failingCompactors = new ConcurrentHashMap<>(); private final ConcurrentHashMap failingTables = new ConcurrentHashMap<>(); @@ -419,7 +420,8 @@ public void run() { rc.setStartTime(this.coordinatorStartTime); RUNNING_CACHE.put(ExternalCompactionId.of(rc.getJob().getExternalCompactionId()), rc); LONG_RUNNING_COMPACTIONS_BY_RG - .computeIfAbsent(rc.getGroup().canonical(), k -> new TimeOrderedRunningCompactionSet()) + .computeIfAbsent(rc.getCompactor().getResourceGroup().canonical(), + k -> new TimeOrderedRunningCompactionSet()) .add(rc); }); } @@ -437,22 +439,21 @@ public void run() { LOG.info("Shutting down"); } - private Map> getIdleCompactors(Set runningCompactors) { + private Map> getIdleCompactors(Set runningCompactors) { - final Map> allCompactors = new HashMap<>(); + final Map> allCompactors = new HashMap<>(); runningCompactors.forEach((csi) -> allCompactors - .computeIfAbsent(csi.getResourceGroup().canonical(), (k) -> new HashSet<>()) - .add(HostAndPort.fromParts(csi.getHost(), csi.getPort()))); + .computeIfAbsent(csi.getResourceGroup().canonical(), (k) -> new HashSet<>()).add(csi)); final Set emptyQueues = new HashSet<>(); // Remove all of the compactors that are running a compaction RUNNING_CACHE.values().forEach(rc -> { - Set busyCompactors = allCompactors.get(rc.getGroup().canonical()); - if (busyCompactors != null - && busyCompactors.remove(HostAndPort.fromString(rc.getCompactorAddress()))) { + Set busyCompactors = + allCompactors.get(rc.getCompactor().getResourceGroup().canonical()); + if (busyCompactors != null && busyCompactors.remove(rc.getCompactor())) { if (busyCompactors.isEmpty()) { - emptyQueues.add(rc.getGroup().canonical()); + emptyQueues.add(rc.getCompactor().getResourceGroup().canonical()); } } }); @@ -521,8 +522,10 @@ public TNextCompactionJob getCompactionJob(TInfo tinfo, TCredentials credentials result = createThriftJob(externalCompactionId, ecm, rcJob, compactionConfig); // It is possible that by the time this added that the the compactor that made this request // is dead. In this cases the compaction is not actually running. + var hp = HostAndPort.fromString(compactorAddress); RUNNING_CACHE.put(ExternalCompactionId.of(result.getExternalCompactionId()), - new RunningCompaction(result, compactorAddress, groupId)); + new RunningCompaction(result, + ServerIdUtil.compactor(groupId, hp.getHost(), hp.getPort()))); TabletLogger.compacting(rcJob.getExtent(), rcJob.getSelectedFateId(), cid, compactorAddress, rcJob); break; @@ -852,9 +855,8 @@ public void compactionFailed(TInfo tinfo, TCredentials credentials, String exter private void captureFailure(ExternalCompactionId ecid, KeyExtent extent) { var rc = RUNNING_CACHE.get(ecid); if (rc != null) { - failingQueues.compute(rc.getGroup(), FailureCounts::incrementFailure); - final String compactor = rc.getCompactorAddress(); - failingCompactors.compute(compactor, FailureCounts::incrementFailure); + failingQueues.compute(rc.getCompactor().getResourceGroup(), FailureCounts::incrementFailure); + failingCompactors.compute(rc.getCompactor(), FailureCounts::incrementFailure); } failingTables.compute(extent.tableId(), FailureCounts::incrementFailure); } @@ -892,9 +894,9 @@ private void printStats(String logPrefix, ConcurrentHashMap private void printStats() { // Remove down compactors from failing list - Map> allCompactors = ExternalCompactionUtil.getCompactorAddrs(ctx); - Set allCompactorAddrs = new HashSet<>(); - allCompactors.values().forEach(l -> l.forEach(c -> allCompactorAddrs.add(c.toString()))); + Map> allCompactors = ExternalCompactionUtil.getCompactorAddrs(ctx); + Set allCompactorAddrs = new HashSet<>(); + allCompactors.values().forEach(l -> l.forEach(c -> allCompactorAddrs.add(c))); failingCompactors.keySet().retainAll(allCompactorAddrs); printStats("Queue", failingQueues, false); printStats("Table", failingTables, false); @@ -904,9 +906,8 @@ private void printStats() { private void captureSuccess(ExternalCompactionId ecid, KeyExtent extent) { var rc = RUNNING_CACHE.get(ecid); if (rc != null) { - failingQueues.compute(rc.getGroup(), FailureCounts::incrementSuccess); - final String compactor = rc.getCompactorAddress(); - failingCompactors.compute(compactor, FailureCounts::incrementSuccess); + failingQueues.compute(rc.getCompactor().getResourceGroup(), FailureCounts::incrementSuccess); + failingCompactors.compute(rc.getCompactor(), FailureCounts::incrementSuccess); } failingTables.compute(extent.tableId(), FailureCounts::incrementSuccess); } @@ -1047,13 +1048,16 @@ public void updateCompactionStatus(TInfo tinfo, TCredentials credentials, rc.addUpdate(timestamp, update); switch (update.state) { case STARTED: - LONG_RUNNING_COMPACTIONS_BY_RG.computeIfAbsent(rc.getGroup().canonical(), - k -> new TimeOrderedRunningCompactionSet()).add(rc); + LONG_RUNNING_COMPACTIONS_BY_RG + .computeIfAbsent(rc.getCompactor().getResourceGroup().canonical(), + k -> new TimeOrderedRunningCompactionSet()) + .add(rc); break; case CANCELLED: case FAILED: case SUCCEEDED: - var compactionSet = LONG_RUNNING_COMPACTIONS_BY_RG.get(rc.getGroup().canonical()); + var compactionSet = + LONG_RUNNING_COMPACTIONS_BY_RG.get(rc.getCompactor().getResourceGroup().canonical()); if (compactionSet != null) { compactionSet.remove(rc); } @@ -1072,7 +1076,8 @@ public void recordCompletion(ExternalCompactionId ecid) { var rc = RUNNING_CACHE.remove(ecid); if (rc != null) { completed.put(ecid, rc); - var compactionSet = LONG_RUNNING_COMPACTIONS_BY_RG.get(rc.getGroup().canonical()); + var compactionSet = + LONG_RUNNING_COMPACTIONS_BY_RG.get(rc.getCompactor().getResourceGroup().canonical()); if (compactionSet != null) { compactionSet.remove(rc); } @@ -1108,8 +1113,8 @@ public TExternalCompactionMap getRunningCompactions(TInfo tinfo, TCredentials cr final TExternalCompactionMap result = new TExternalCompactionMap(); RUNNING_CACHE.forEach((ecid, rc) -> { TExternalCompaction trc = new TExternalCompaction(); - trc.setGroupName(rc.getGroup().canonical()); - trc.setCompactor(rc.getCompactorAddress()); + trc.setGroupName(rc.getCompactor().getResourceGroup().canonical()); + trc.setCompactor(rc.getCompactor().toHostPortString()); trc.setUpdates(rc.getUpdates()); trc.setJob(rc.getJob()); result.putToCompactions(ecid.canonical(), trc); @@ -1144,8 +1149,8 @@ public Map getLongRunningCompactions(TInfo tinfo while (iter.hasNext()) { RunningCompaction rc = iter.next(); TExternalCompaction trc = new TExternalCompaction(); - trc.setGroupName(rc.getGroup().canonical()); - trc.setCompactor(rc.getCompactorAddress()); + trc.setGroupName(rc.getCompactor().getResourceGroup().canonical()); + trc.setCompactor(rc.getCompactor().toHostPortString()); trc.setUpdates(rc.getUpdates()); trc.setJob(rc.getJob()); compactions.addToCompactions(trc); @@ -1174,8 +1179,8 @@ public TExternalCompactionMap getCompletedCompactions(TInfo tinfo, TCredentials final TExternalCompactionMap result = new TExternalCompactionMap(); completed.asMap().forEach((ecid, rc) -> { TExternalCompaction trc = new TExternalCompaction(); - trc.setGroupName(rc.getGroup().canonical()); - trc.setCompactor(rc.getCompactorAddress()); + trc.setGroupName(rc.getCompactor().getResourceGroup().canonical()); + trc.setCompactor(rc.getCompactor().toHostPortString()); trc.setJob(rc.getJob()); trc.setUpdates(rc.getUpdates()); result.putToCompactions(ecid.canonical(), trc); @@ -1199,7 +1204,7 @@ public void cancel(TInfo tinfo, TCredentials credentials, String externalCompact TableOperation.COMPACT_CANCEL, TableOperationExceptionType.NOTFOUND, e.getMessage()); } - cancelCompactionOnCompactor(runningCompaction.getCompactorAddress(), externalCompactionId); + cancelCompactionOnCompactor(runningCompaction.getCompactor(), externalCompactionId); } /* Method exists to be called from test */ @@ -1218,9 +1223,8 @@ protected Set getRunningCompactors() { } /* Method exists to be overridden in test to hide static method */ - protected void cancelCompactionOnCompactor(String address, String externalCompactionId) { - HostAndPort hostPort = HostAndPort.fromString(address); - ExternalCompactionUtil.cancelCompaction(this.ctx, hostPort, externalCompactionId); + protected void cancelCompactionOnCompactor(ServerId address, String externalCompactionId) { + ExternalCompactionUtil.cancelCompaction(this.ctx, address, externalCompactionId); } private void deleteEmpty(ZooReaderWriter zoorw, String path) @@ -1360,13 +1364,13 @@ public void cleanUpInternalState() { if (jobGroupsNotInConfiguration != null && !jobGroupsNotInConfiguration.isEmpty()) { RUNNING_CACHE.values().forEach(rc -> { - if (jobGroupsNotInConfiguration.contains(ResourceGroupId.of(rc.getGroup().canonical()))) { + var rgid = ResourceGroupId.of(rc.getCompactor().getResourceGroup().canonical()); + if (jobGroupsNotInConfiguration.contains(rgid)) { LOG.warn( "External compaction {} running in group {} on compactor {}," + " but group not found in current configuration. Failing compaction...", - rc.getJob().getExternalCompactionId(), rc.getGroup(), rc.getCompactorAddress()); - cancelCompactionOnCompactor(rc.getCompactorAddress(), - rc.getJob().getExternalCompactionId()); + rc.getJob().getExternalCompactionId(), rgid, rc.getCompactor()); + cancelCompactionOnCompactor(rc.getCompactor(), rc.getJob().getExternalCompactionId()); } }); @@ -1404,7 +1408,7 @@ public void cleanUpInternalState() { final long now = System.currentTimeMillis(); final long warningTime = getMissingCompactorWarningTime(); - Map> idleCompactors = getIdleCompactors(runningCompactors); + Map> idleCompactors = getIdleCompactors(runningCompactors); for (ResourceGroupId groupName : groupsInConfiguration) { long lastCheckTime = TIME_COMPACTOR_LAST_CHECKED.getOrDefault(groupName, coordinatorStartTime); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/CommitCompaction.java b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/CommitCompaction.java index 29e626d9540..a47d429e907 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/CommitCompaction.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/CommitCompaction.java @@ -38,6 +38,7 @@ import org.apache.accumulo.core.metadata.AbstractTabletFile; 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.CompactionMetadata; import org.apache.accumulo.core.metadata.schema.DataFileValue; @@ -81,9 +82,9 @@ public Repo call(FateId fateId, Manager manager) throws Exception { // fact that a commit may not have happened in the current call and continues for this reason. TabletMetadata tabletMetadata = commitCompaction(manager.getContext(), ecid, newFile); - String loc = null; + TServerInstance loc = null; if (tabletMetadata != null && tabletMetadata.getLocation() != null) { - loc = tabletMetadata.getLocation().getHostPortSession(); + loc = tabletMetadata.getLocation().getServerInstance(); } // This will causes the tablet to be reexamined to see if it needs any more compactions. diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/PutGcCandidates.java b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/PutGcCandidates.java index 3e5b62caa93..82b91dc247f 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/PutGcCandidates.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/PutGcCandidates.java @@ -20,6 +20,7 @@ import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.Repo; +import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; import org.apache.accumulo.manager.Manager; import org.apache.accumulo.manager.tableOps.ManagerRepo; @@ -27,9 +28,9 @@ public class PutGcCandidates extends ManagerRepo { private static final long serialVersionUID = 1L; private final CompactionCommitData commitData; - private final String refreshLocation; + private final TServerInstance refreshLocation; - public PutGcCandidates(CompactionCommitData commitData, String refreshLocation) { + public PutGcCandidates(CompactionCommitData commitData, TServerInstance refreshLocation) { this.commitData = commitData; this.refreshLocation = refreshLocation; } diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/RefreshTablet.java b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/RefreshTablet.java index bf5900ad4a2..8f44815a533 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/RefreshTablet.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/RefreshTablet.java @@ -38,10 +38,10 @@ public class RefreshTablet extends ManagerRepo { private static final long serialVersionUID = 1L; private final TKeyExtent extent; - private final String tserverInstance; + private final TServerInstance tserverInstance; private final String compactionId; - public RefreshTablet(String ecid, TKeyExtent extent, String tserverInstance) { + public RefreshTablet(String ecid, TKeyExtent extent, TServerInstance tserverInstance) { this.compactionId = ecid; this.extent = extent; this.tserverInstance = tserverInstance; @@ -50,15 +50,13 @@ public RefreshTablet(String ecid, TKeyExtent extent, String tserverInstance) { @Override public Repo call(FateId fateId, Manager manager) throws Exception { - TServerInstance tsi = new TServerInstance(tserverInstance); - // there is a single tserver and single tablet, do not need a thread pool. The direct executor // will run everything in the current thread ExecutorService executorService = MoreExecutors.newDirectExecutorService(); try { TabletRefresher.refreshTablets(executorService, "compaction:" + KeyExtent.fromThrift(extent), manager.getContext(), manager::onlineTabletServers, - Map.of(TabletMetadata.Location.current(tsi), List.of(extent))); + Map.of(TabletMetadata.Location.current(tserverInstance), List.of(extent))); } finally { executorService.shutdownNow(); } diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java index 4bb58b1bfdb..993d1862673 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java @@ -39,6 +39,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.bulk.Bulk; import org.apache.accumulo.core.clientImpl.bulk.Bulk.Files; import org.apache.accumulo.core.clientImpl.bulk.BulkSerialize; @@ -53,6 +54,7 @@ import org.apache.accumulo.core.manager.thrift.BulkImportState; 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.TabletFile; import org.apache.accumulo.core.metadata.schema.Ample; import org.apache.accumulo.core.metadata.schema.Ample.ConditionalResult.Status; @@ -77,7 +79,6 @@ import org.slf4j.LoggerFactory; import com.google.common.base.Preconditions; -import com.google.common.net.HostAndPort; /** * Make asynchronous load calls to each overlapping Tablet. This RepO does its work on the isReady @@ -288,23 +289,24 @@ void load(List tablets, Files files) { private Map allocateTimestamps(List tablets, int numStamps) { - Map> serversToAsk = new HashMap<>(); + Map> serversToAsk = new HashMap<>(); Map allTimestamps = new HashMap<>(); for (var tablet : tablets) { if (tablet.getLocation() != null && tablet.getLocation().getType() == CURRENT) { - var location = tablet.getLocation().getHostAndPort(); + var location = tablet.getLocation().getServerInstance(); serversToAsk.computeIfAbsent(location, l -> new ArrayList<>()) .add(tablet.getExtent().toThrift()); } } for (var entry : serversToAsk.entrySet()) { - HostAndPort server = entry.getKey(); + TServerInstance server = entry.getKey(); List extents = entry.getValue(); - Map serversTimestamps = allocateTimestamps(server, extents, numStamps); + Map serversTimestamps = + allocateTimestamps(server.getServer(), extents, numStamps); allTimestamps.putAll(serversTimestamps); } @@ -312,7 +314,7 @@ private Map allocateTimestamps(List tablets, int return allTimestamps; } - private Map allocateTimestamps(HostAndPort server, List extents, + private Map allocateTimestamps(ServerId server, List extents, int numStamps) { TabletServerClientService.Client client = null; var context = manager.getContext(); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/TabletRefresher.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/TabletRefresher.java index 339857ec20a..bfc51cf2801 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/TabletRefresher.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/TabletRefresher.java @@ -203,8 +203,8 @@ private static List sendSyncRefreshRequest(ServerContext context, St log.trace("{} sending refresh request to {} for {} extents", logId, location, refreshes.size()); var timeInMillis = context.getConfiguration().getTimeInMillis(Property.MANAGER_BULK_TIMEOUT); - client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, location.getHostAndPort(), - context, timeInMillis); + client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, + location.getServerInstance().getServer(), context, timeInMillis); var unrefreshed = client.refreshTablets(TraceUtil.traceInfo(), context.rpcCreds(), refreshes); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tserverOps/ShutdownTServer.java b/server/manager/src/main/java/org/apache/accumulo/manager/tserverOps/ShutdownTServer.java index 3ef18695c40..0f73463dc82 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tserverOps/ShutdownTServer.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tserverOps/ShutdownTServer.java @@ -20,7 +20,6 @@ import static java.nio.charset.StandardCharsets.UTF_8; -import org.apache.accumulo.core.data.ResourceGroupId; import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; @@ -41,21 +40,16 @@ public class ShutdownTServer extends ManagerRepo { private static final long serialVersionUID = 2L; private static final Logger log = LoggerFactory.getLogger(ShutdownTServer.class); - private final ResourceGroupId resourceGroup; - private final HostAndPort hostAndPort; - private final String serverSession; + private final TServerInstance server; private final boolean force; - public ShutdownTServer(TServerInstance server, ResourceGroupId resourceGroup, boolean force) { - this.hostAndPort = server.getHostAndPort(); - this.resourceGroup = resourceGroup; - this.serverSession = server.getSession(); + public ShutdownTServer(TServerInstance server, boolean force) { + this.server = server; this.force = force; } @Override public long isReady(FateId fateId, Manager manager) { - TServerInstance server = new TServerInstance(hostAndPort, serverSession); // suppress assignment of tablets to the server if (force) { return 0; @@ -98,12 +92,13 @@ public long isReady(FateId fateId, Manager manager) { public Repo call(FateId fateId, Manager manager) throws Exception { // suppress assignment of tablets to the server if (force) { + var hp = HostAndPort.fromParts(server.getServer().getHost(), server.getServer().getPort()); ZooReaderWriter zoo = manager.getContext().getZooSession().asReaderWriter(); - var path = - manager.getContext().getServerPaths().createTabletServerPath(resourceGroup, hostAndPort); + var path = manager.getContext().getServerPaths() + .createTabletServerPath(server.getServer().getResourceGroup(), hp); ServiceLock.deleteLock(zoo, path); - path = manager.getContext().getServerPaths().createDeadTabletServerPath(resourceGroup, - hostAndPort); + path = manager.getContext().getServerPaths() + .createDeadTabletServerPath(server.getServer().getResourceGroup(), hp); zoo.putPersistentData(path.toString(), "forced down".getBytes(UTF_8), NodeExistsPolicy.OVERWRITE); } diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java index 89a92beda73..4695271c07f 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java @@ -50,6 +50,7 @@ import org.apache.accumulo.core.client.admin.TabletAvailability; import org.apache.accumulo.core.clientImpl.Namespace; import org.apache.accumulo.core.clientImpl.NamespaceMapping; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; @@ -65,14 +66,18 @@ import org.apache.accumulo.core.manager.state.tables.TableState; import org.apache.accumulo.core.metadata.StoredTabletFile; import org.apache.accumulo.core.metadata.SystemTables; +import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.core.metadata.schema.Ample; import org.apache.accumulo.core.metadata.schema.Ample.DataLevel; import org.apache.accumulo.core.metadata.schema.Ample.TabletsMutator; import org.apache.accumulo.core.metadata.schema.MetadataSchema; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ChoppedColumnFamily; +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.ExternalCompactionColumnFamily; +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily; +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LastLocationColumnFamily; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily; import org.apache.accumulo.core.metadata.schema.RootTabletMetadata; import org.apache.accumulo.core.metadata.schema.TabletMergeabilityMetadata; @@ -104,6 +109,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.net.HostAndPort; +import com.google.gson.JsonSyntaxException; //TODO when removing this class, also remove MetadataSchema.Upgrader11to12 public class Upgrader11to12 implements Upgrader { @@ -214,8 +221,10 @@ public static String getRowPrefix() { static final String ZTABLE_NAME = "/name"; @VisibleForTesting - static final Set UPGRADE_FAMILIES = Set.of(DataFileColumnFamily.NAME, CHOPPED, - ExternalCompactionColumnFamily.NAME, ScanFileColumnFamily.NAME); + static final Set UPGRADE_FAMILIES = + Set.of(DataFileColumnFamily.NAME, CHOPPED, ExternalCompactionColumnFamily.NAME, + ScanFileColumnFamily.NAME, CurrentLocationColumnFamily.NAME, + FutureLocationColumnFamily.NAME, LastLocationColumnFamily.NAME); @VisibleForTesting static final String ZNAMESPACE_NAME = "/name"; @@ -767,6 +776,33 @@ static void upgradeDataFileCF(final Key key, final Value value, final Mutation m } } + static void upgradeLocationCF(final Key key, final Value value, final Mutation m) { + // Before version 4.0 the locations in the RootTabletMetadata were stored + // in the format: {"session": "host:port"}. Attempt to convert the location + // using the current format and if that fails attempt to read the old format. + try { + TServerInstance.deserialize(value.toString()); + } catch (JsonSyntaxException e) { + final String session = key.getColumnQualifier().toString(); + final HostAndPort hp = HostAndPort.fromString(value.toString()); + final var tsi = + new TServerInstance(ServerIdUtil.tserver(hp.getHost(), hp.getPort()), session); + switch (key.getColumnFamily().toString()) { + case (CurrentLocationColumnFamily.STR_NAME): + m.at().family(CurrentLocationColumnFamily.NAME).qualifier(session).put(tsi.serialize()); + break; + case (FutureLocationColumnFamily.STR_NAME): + m.at().family(FutureLocationColumnFamily.NAME).qualifier(session).put(tsi.serialize()); + break; + case (LastLocationColumnFamily.STR_NAME): + m.at().family(LastLocationColumnFamily.NAME).qualifier(session).put(tsi.serialize()); + break; + default: + throw new IllegalArgumentException("Unhandled location colf: " + key.getColumnFamily()); + } + } + } + void processReferences(MutationWriter batchWriter, Iterable> scanner, String tableName) { try { @@ -793,6 +829,10 @@ void processReferences(MutationWriter batchWriter, Iterable var family = key.getColumnFamily(); if (family.equals(DataFileColumnFamily.NAME)) { upgradeDataFileCF(key, value, update); + } else if (family.equals(CurrentLocationColumnFamily.NAME) + || family.equals(FutureLocationColumnFamily.NAME) + || family.equals(LastLocationColumnFamily.NAME)) { + upgradeLocationCF(key, value, update); } else if (family.equals(ScanFileColumnFamily.NAME)) { LOG.debug("Deleting scan reference from:{}. Ref: {}", tableName, key.getRow()); update.at().family(ScanFileColumnFamily.NAME).qualifier(key.getColumnQualifier()) diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/TabletGroupWatcherTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/TabletGroupWatcherTest.java index dd4aef9e51f..81d1e720c8b 100644 --- a/server/manager/src/test/java/org/apache/accumulo/manager/TabletGroupWatcherTest.java +++ b/server/manager/src/test/java/org/apache/accumulo/manager/TabletGroupWatcherTest.java @@ -24,32 +24,31 @@ import java.util.TreeMap; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.metadata.TServerInstance; import org.junit.jupiter.api.Test; -import com.google.common.net.HostAndPort; - public class TabletGroupWatcherTest { @Test public void testFindingServer() { TreeMap servers = new TreeMap<>(); - servers.put(new TServerInstance("192.168.1.2:9997", 50L), "tserver1"); + servers.put(new TServerInstance(ServerIdUtil.tserver("192.168.1.2", 9997), 50L), "tserver1"); // add an entry where only the session differs. For this case the code does not really care // which one is found, the impl happens to find the first one. This situation could happen // temporarily, and it should not cause any problems. - servers.put(new TServerInstance("192.168.1.2:9997", 70L), "tserver2"); - servers.put(new TServerInstance("192.168.1.4:9997", -90L), "tserver3"); - - assertNull(findServerIgnoringSession(servers, HostAndPort.fromString("192.168.1.1:9997"))); - assertNull(findServerIgnoringSession(servers, HostAndPort.fromString("192.168.1.2:9996"))); - assertNull(findServerIgnoringSession(servers, HostAndPort.fromString("192.168.1.2:9998"))); - assertNull(findServerIgnoringSession(servers, HostAndPort.fromString("192.168.1.3:9997"))); - assertNull(findServerIgnoringSession(servers, HostAndPort.fromString("192.168.1.5:9997"))); - - assertEquals(new TServerInstance("192.168.1.2:9997", 50L), - findServerIgnoringSession(servers, HostAndPort.fromString("192.168.1.2:9997"))); - assertEquals(new TServerInstance("192.168.1.4:9997", -90L), - findServerIgnoringSession(servers, HostAndPort.fromString("192.168.1.4:9997"))); + servers.put(new TServerInstance(ServerIdUtil.tserver("192.168.1.2", 9997), 70L), "tserver2"); + servers.put(new TServerInstance(ServerIdUtil.tserver("192.168.1.4", 9997), -90L), "tserver3"); + + assertNull(findServerIgnoringSession(servers, ServerIdUtil.tserver("192.168.1.1", 9997))); + assertNull(findServerIgnoringSession(servers, ServerIdUtil.tserver("192.168.1.2", 9996))); + assertNull(findServerIgnoringSession(servers, ServerIdUtil.tserver("192.168.1.2", 9998))); + assertNull(findServerIgnoringSession(servers, ServerIdUtil.tserver("192.168.1.3", 9997))); + assertNull(findServerIgnoringSession(servers, ServerIdUtil.tserver("192.168.1.5", 9997))); + + assertEquals(new TServerInstance(ServerIdUtil.tserver("192.168.1.2", 9997), 50L), + findServerIgnoringSession(servers, ServerIdUtil.tserver("192.168.1.2", 9997))); + assertEquals(new TServerInstance(ServerIdUtil.tserver("192.168.1.4", 9997), -90L), + findServerIgnoringSession(servers, ServerIdUtil.tserver("192.168.1.4", 9997))); } } diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/compaction/CompactionCoordinatorTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/compaction/CompactionCoordinatorTest.java index 3cb474fa408..00022083e78 100644 --- a/server/manager/src/test/java/org/apache/accumulo/manager/compaction/CompactionCoordinatorTest.java +++ b/server/manager/src/test/java/org/apache/accumulo/manager/compaction/CompactionCoordinatorTest.java @@ -44,6 +44,7 @@ import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.admin.CompactionConfig; import org.apache.accumulo.core.client.admin.servers.ServerId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.clientImpl.thrift.TInfo; import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; import org.apache.accumulo.core.compaction.thrift.TExternalCompaction; @@ -88,8 +89,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInfo; -import com.google.common.net.HostAndPort; - public class CompactionCoordinatorTest { // Need a non-null fateInstances reference for CompactionCoordinator.compactionCompleted @@ -98,7 +97,7 @@ public class CompactionCoordinatorTest { private static final ResourceGroupId GROUP_ID = ResourceGroupId.of("R2DQ"); - private final HostAndPort tserverAddr = HostAndPort.fromParts("192.168.1.1", 9090); + private final ServerId tserverAddr = ServerIdUtil.tserver(GROUP_ID, "192.168.1.1", 9090); public MetricsInfo getMockMetrics() { MetricsInfo metricsInfo = createMock(MetricsInfo.class); @@ -219,7 +218,7 @@ protected TExternalCompactionJob createThriftJob(String externalCompactionId, } @Override - protected void cancelCompactionOnCompactor(String address, String externalCompactionId) {} + protected void cancelCompactionOnCompactor(ServerId address, String externalCompactionId) {} } @@ -289,7 +288,7 @@ public void testCoordinatorRestartOneRunningCompaction() throws Exception { expect(job.getExternalCompactionId()).andReturn(eci.toString()).atLeastOnce(); TKeyExtent extent = new TKeyExtent(); extent.setTable("1".getBytes(UTF_8)); - runningCompactions.add(new RunningCompaction(job, tserverAddr.toString(), GROUP_ID)); + runningCompactions.add(new RunningCompaction(job, tserverAddr)); replay(job); var coordinator = new TestCoordinator(manager, runningCompactions); @@ -307,14 +306,12 @@ public void testCoordinatorRestartOneRunningCompaction() throws Exception { Entry ecomp = running.entrySet().iterator().next(); assertEquals(eci, ecomp.getKey()); RunningCompaction rc = ecomp.getValue(); - assertEquals(GROUP_ID, rc.getGroup()); - assertEquals(tserverAddr.toString(), rc.getCompactorAddress()); + assertEquals(tserverAddr, rc.getCompactor()); assertTrue(coordinator.getLongRunningByGroup().containsKey(GROUP_ID.toString())); assertTrue(coordinator.getLongRunningByGroup().get(GROUP_ID.toString()).size() == 1); rc = coordinator.getLongRunningByGroup().get(GROUP_ID.toString()).iterator().next(); - assertEquals(GROUP_ID, rc.getGroup()); - assertEquals(tserverAddr.toString(), rc.getCompactorAddress()); + assertEquals(tserverAddr, rc.getCompactor()); verify(job); } @@ -362,7 +359,7 @@ public void testGetCompactionJob() throws Exception { Entry entry = coordinator.getRunning().entrySet().iterator().next(); assertEquals(eci.toString(), entry.getKey().toString()); - assertEquals("localhost:10241", entry.getValue().getCompactorAddress()); + assertEquals("localhost:10241", entry.getValue().getCompactor().toHostPortString()); assertEquals(eci.toString(), entry.getValue().getJob().getExternalCompactionId()); verify(tm); diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/ShutdownTServerTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/ShutdownTServerTest.java index ee5304219a8..c55cd67cef1 100644 --- a/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/ShutdownTServerTest.java +++ b/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/ShutdownTServerTest.java @@ -26,7 +26,7 @@ import java.util.HashMap; import java.util.UUID; -import org.apache.accumulo.core.data.ResourceGroupId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.FateInstanceType; import org.apache.accumulo.core.fate.Repo; @@ -39,17 +39,15 @@ import org.easymock.EasyMock; import org.junit.jupiter.api.Test; -import com.google.common.net.HostAndPort; - public class ShutdownTServerTest { @Test public void testSingleShutdown() throws Exception { - HostAndPort hap = HostAndPort.fromParts("localhost", 1234); - final TServerInstance tserver = new TServerInstance(hap, "fake"); + final TServerInstance tserver = + new TServerInstance(ServerIdUtil.tserver("localhost", 1234), "fake"); final boolean force = false; - final ShutdownTServer op = new ShutdownTServer(tserver, ResourceGroupId.DEFAULT, force); + final ShutdownTServer op = new ShutdownTServer(tserver, force); final Manager manager = EasyMock.createMock(Manager.class); final FateId fateId = FateId.from(FateInstanceType.USER, UUID.randomUUID()); diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/merge/MergeTabletsTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/merge/MergeTabletsTest.java index 9e3037dbe01..04ef9f59633 100644 --- a/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/merge/MergeTabletsTest.java +++ b/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/merge/MergeTabletsTest.java @@ -47,6 +47,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import java.time.Duration; import java.util.ArrayList; import java.util.EnumSet; import java.util.List; @@ -58,9 +59,9 @@ import java.util.function.Consumer; import org.apache.accumulo.core.client.admin.TabletAvailability; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.NamespaceId; 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.FateId; import org.apache.accumulo.core.fate.FateInstanceType; @@ -151,10 +152,13 @@ public void testManyColumns() throws Exception { var tabletTime = MetadataTime.parse("L30"); var flushID = OptionalLong.of(40); var availability = TabletAvailability.HOSTED; - var lastLocation = TabletMetadata.Location.last("1.2.3.4:1234", "123456789"); - var suspendingTServer = SuspendingTServer.fromValue(new Value("1.2.3.4:5|56")); + var lastLocation = TabletMetadata.Location + .last(new TServerInstance(ServerIdUtil.tserver("1.2.3.4", 1234), "123456789")); + var suspendingTServer = + new SuspendingTServer(new TServerInstance(ServerIdUtil.tserver("1.2.3.4", 1025), ""), + SteadyTime.from(Duration.ofMillis(56))); var mergeability = TabletMergeabilityMetadata.always(SteadyTime.from(1, TimeUnit.SECONDS)); - var migration = new TServerInstance("localhost:1234", 56L); + var migration = new TServerInstance(ServerIdUtil.tserver("localhost", 1234), 56L); var tablet1 = TabletMetadata.builder(ke1).putOperation(opid).putDirName("td1").putFile(file3, dfv3) @@ -306,7 +310,7 @@ public void testMisplacedMerge() throws Exception { @Test public void testUnexpectedColumns() { - var tserver = new TServerInstance("1.2.3.4:1234", 123456789L); + var tserver = new TServerInstance(ServerIdUtil.tserver("1.2.3.4", 1234), 123456789L); var futureLoc = TabletMetadata.Location.future(tserver); testUnexpectedColumn(tmb -> tmb.putLocation(futureLoc), "had location", futureLoc.toString()); @@ -318,7 +322,7 @@ public void testUnexpectedColumns() { testUnexpectedColumn(tmb -> tmb.putOperation(otherOpid), "had unexpected opid", otherOpid.toString()); - var walog = LogEntry.fromPath("localhost+8020/" + UUID.randomUUID()); + var walog = LogEntry.fromPath("default+localhost+8020/" + UUID.randomUUID()); testUnexpectedColumn(tmb -> tmb.putWal(walog), "has unexpected walogs 1"); FateId ucfid1 = otherFateId; diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/split/UpdateTabletsTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/split/UpdateTabletsTest.java index 0e242f75e04..b7992bf6661 100644 --- a/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/split/UpdateTabletsTest.java +++ b/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/split/UpdateTabletsTest.java @@ -23,6 +23,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import java.time.Duration; import java.util.EnumSet; import java.util.List; import java.util.Map; @@ -39,9 +40,9 @@ import org.apache.accumulo.core.client.admin.TabletAvailability; import org.apache.accumulo.core.client.admin.TabletMergeability; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.clientImpl.TabletMergeabilityUtil; 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.FateId; import org.apache.accumulo.core.fate.FateInstanceType; @@ -225,9 +226,12 @@ public void testManyColumns() throws Exception { var tabletTime = MetadataTime.parse("L30"); var flushID = OptionalLong.of(40); var availability = TabletAvailability.HOSTED; - var lastLocation = TabletMetadata.Location.last("1.2.3.4:1234", "123456789"); - var suspendingTServer = SuspendingTServer.fromValue(new Value("1.2.3.4:5|56")); - var migration = new TServerInstance("localhost:1234", 56L); + var lastLocation = TabletMetadata.Location + .last(new TServerInstance(ServerIdUtil.tserver("1.2.3.4", 1234), "123456789")); + var suspendingTServer = + new SuspendingTServer(new TServerInstance(ServerIdUtil.tserver("1.2.3.4", 1025), ""), + SteadyTime.from(Duration.ofMillis(56))); + var migration = new TServerInstance(ServerIdUtil.tserver("localhost", 1234), 56L); String dir1 = "dir1"; String dir2 = "dir2"; @@ -412,7 +416,8 @@ public void testErrors() throws Exception { var opid = TabletOperationId.from(TabletOperationType.SPLITTING, fateId); // Test splitting a tablet with a location - var location = TabletMetadata.Location.future(new TServerInstance("1.2.3.4:1234", 123456789L)); + var location = TabletMetadata.Location + .future(new TServerInstance(ServerIdUtil.tserver("1.2.3.4", 1234), 123456789L)); var tablet1 = TabletMetadata.builder(origExtent).putOperation(opid).putLocation(location).build(); var e = assertThrows(IllegalStateException.class, () -> testError(origExtent, tablet1, fateId)); @@ -433,7 +438,7 @@ public void testErrors() throws Exception { assertTrue(e.getMessage().contains(opid2.toString())); // Test splitting a tablet with walogs - var walog = LogEntry.fromPath("localhost+8020/" + UUID.randomUUID()); + var walog = LogEntry.fromPath("default+localhost+8020/" + UUID.randomUUID()); var tablet4 = TabletMetadata.builder(origExtent).putOperation(opid).putWal(walog) .build(ColumnType.LOCATION); e = assertThrows(IllegalStateException.class, () -> testError(origExtent, tablet4, fateId)); diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader11to12Test.java b/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader11to12Test.java index e1c35e1e3f6..53ea3f66a89 100644 --- a/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader11to12Test.java +++ b/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader11to12Test.java @@ -371,7 +371,41 @@ public void upgradeZooKeeperTest() throws Exception { "{\"version\":1,\"columnValues\":{\"file\":{\"hdfs://localhost:8020/accumulo/tables/+r/root_tablet/A0000030.rf\":\"856,15\",\"hdfs://localhost:8020/accumulo/tables/+r/root_tablet/F000000r.rf\":\"308,2\"},\"last\":{\"100017f46240004\":\"localhost:9997\"},\"loc\":{\"100017f46240004\":\"localhost:9997\"},\"srv\":{\"dir\":\"root_tablet\",\"flush\":\"16\",\"lock\":\"tservers/localhost:9997/zlock#f6a582b9-9583-4553-b179-a7a3852c8332#0000000000$100017f46240004\",\"time\":\"L42\"},\"~tab\":{\"~pr\":\"\\u0000\"}}}\n" .getBytes(UTF_8); final String zKRootV2 = - "{\"version\":1,\"columnValues\":{\"file\":{\"{\\\"path\\\":\\\"hdfs://localhost:8020/accumulo/tables/+r/root_tablet/A0000030.rf\\\",\\\"startRow\\\":\\\"\\\",\\\"endRow\\\":\\\"\\\"}\":\"856,15\",\"{\\\"path\\\":\\\"hdfs://localhost:8020/accumulo/tables/+r/root_tablet/F000000r.rf\\\",\\\"startRow\\\":\\\"\\\",\\\"endRow\\\":\\\"\\\"}\":\"308,2\"},\"last\":{\"100017f46240004\":\"localhost:9997\"},\"loc\":{\"100017f46240004\":\"localhost:9997\"},\"srv\":{\"dir\":\"root_tablet\",\"flush\":\"16\",\"lock\":\"tservers/localhost:9997/zlock#f6a582b9-9583-4553-b179-a7a3852c8332#0000000000$100017f46240004\",\"time\":\"L42\"},\"~tab\":{\"~pr\":\"\\u0000\"}}}"; + """ + { \ + 'version': 1, \ + 'columnValues': \ + { \ + 'file': \ + { \ + '{\\\"path\\\":\\\"hdfs://localhost:8020/accumulo/tables/+r/root_tablet/A0000030.rf\\\",\\\"startRow\\\":\\\"\\\",\\\"endRow\\\":\\\"\\\"}': '856,15', \ + '{\\\"path\\\":\\\"hdfs://localhost:8020/accumulo/tables/+r/root_tablet/F000000r.rf\\\",\\\"startRow\\\":\\\"\\\",\\\"endRow\\\":\\\"\\\"}': '308,2' \ + }, \ + 'last': \ + { \ + '100017f46240004': \ + '{ \ + \\\"server\\\": {\\\"type\\\":\\\"TABLET_SERVER\\\",\\\"resourceGroup\\\":\\\"default\\\",\\\"host\\\":\\\"localhost\\\",\\\"port\\\":9997}, \ + \\\"session\\\":\\\"100017f46240004\\\" \ + }' \ + }, \ + 'loc': \ + { \ + '100017f46240004': \ + '{ \ + \\\"server\\\": {\\\"type\\\":\\\"TABLET_SERVER\\\",\\\"resourceGroup\\\":\\\"default\\\",\\\"host\\\":\\\"localhost\\\",\\\"port\\\":9997}, \ + \\\"session\\\":\\\"100017f46240004\\\" \ + }' \ + }, \ + 'srv':{'dir':'root_tablet','flush':'16','lock':'tservers/localhost:9997/zlock#f6a582b9-9583-4553-b179-a7a3852c8332#0000000000$100017f46240004','time':'L42'}, \ + '~tab': \ + { \ + '~pr':'\\u0000' \ + } \ + } \ + } \ + """ + .replaceAll(" ", "").replaceAll("'", "\""); InstanceId iid = InstanceId.of(UUID.randomUUID()); Upgrader11to12 upgrader = new Upgrader11to12(); diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java index 070b93266d5..0f50b20de0c 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java @@ -49,6 +49,7 @@ import org.apache.accumulo.core.cli.ConfigOpts; import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.client.admin.servers.ServerId.Type; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService; import org.apache.accumulo.core.compaction.thrift.TExternalCompaction; import org.apache.accumulo.core.compaction.thrift.TExternalCompactionMap; @@ -104,7 +105,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Suppliers; -import com.google.common.net.HostAndPort; /** * Serve manager statistics with an embedded web server. @@ -148,7 +148,7 @@ public static void main(String[] args) throws Exception { private final AtomicBoolean fetching = new AtomicBoolean(false); private ManagerMonitorInfo mmi; private GCStatus gcStatus; - private volatile Optional coordinatorHost = Optional.empty(); + private volatile Optional coordinatorHost = Optional.empty(); private final String coordinatorMissingMsg = "Error getting the compaction coordinator client. Check that the Manager is running."; @@ -240,8 +240,7 @@ public void fetchData() { "io.getServers returned nothing for Manager, but it's up."); } ServerId manager = managers.iterator().next(); - Optional nextCoordinatorHost = - Optional.of(HostAndPort.fromString(manager.toHostPortString())); + Optional nextCoordinatorHost = Optional.of(manager); if (coordinatorHost.isEmpty() || !coordinatorHost.orElseThrow().equals(nextCoordinatorHost.orElseThrow())) { coordinatorHost = nextCoordinatorHost; @@ -326,7 +325,7 @@ public void fetchData() { private GCStatus fetchGcStatus() { ServerContext context = getContext(); GCStatus result = null; - HostAndPort address = null; + ServerId address = null; try { // Read the gc location from its lock ZooReaderWriter zk = context.getZooSession().asReaderWriter(); @@ -334,7 +333,7 @@ private GCStatus fetchGcStatus() { List locks = ServiceLock.validateAndSort(path, zk.getChildren(path.toString())); if (locks != null && !locks.isEmpty()) { address = ServiceLockData.parse(zk.getData(path + "/" + locks.get(0))) - .map(sld -> sld.getAddress(ThriftService.GC)).orElse(null); + .map(sld -> sld.getServer(ThriftService.GC)).orElse(null); if (address == null) { log.warn("Unable to contact the garbage collector (no address)"); return null; @@ -383,7 +382,7 @@ public void run() { log.debug("Monitor listening on {}:{}", server.getHostName(), livePort); } - HostAndPort advertiseAddress = getAdvertiseAddress(); + ServerId advertiseAddress = getAdvertiseAddress(); if (advertiseAddress == null) { // use the bind address from the connector, unless it's null or 0.0.0.0 String advertiseHost = server.getHostName(); @@ -394,11 +393,11 @@ public void run() { throw new RuntimeException("Unable to get hostname for advertise address", e); } } - updateAdvertiseAddress(HostAndPort.fromParts(advertiseHost, livePort)); + updateAdvertiseAddress(ServerIdUtil.monitor(advertiseHost, livePort)); } else { - updateAdvertiseAddress(HostAndPort.fromParts(advertiseAddress.getHost(), livePort)); + updateAdvertiseAddress(ServerIdUtil.monitor(advertiseAddress.getHost(), livePort)); } - HostAndPort monitorHostAndPort = getAdvertiseAddress(); + ServerId monitorHostAndPort = getAdvertiseAddress(); log.debug("Using {} to advertise monitor location in ZooKeeper", monitorHostAndPort); try { @@ -412,7 +411,7 @@ public void run() { MetricsInfo metricsInfo = getContext().getMetricsInfo(); metricsInfo.addMetricsProducers(this); metricsInfo.init(MetricsInfo.serviceTags(getContext().getInstanceName(), getApplicationName(), - monitorHostAndPort, getResourceGroup())); + monitorHostAndPort)); // Needed to support the existing zk monitor address format if (!rootContext.endsWith("/")) { @@ -557,13 +556,13 @@ public static class CompactionStats { // Use Suppliers.memoizeWithExpiration() to cache the results of expensive fetch operations. This // avoids unnecessary repeated fetches within the expiration period and ensures that multiple // requests around the same time use the same cached data. - private final Supplier> tserverScansSupplier = + private final Supplier> tserverScansSupplier = Suppliers.memoizeWithExpiration(this::fetchTServerScans, expirationTimeMinutes, MINUTES); - private final Supplier> sserverScansSupplier = + private final Supplier> sserverScansSupplier = Suppliers.memoizeWithExpiration(this::fetchSServerScans, expirationTimeMinutes, MINUTES); - private final Supplier> compactionsSupplier = + private final Supplier> compactionsSupplier = Suppliers.memoizeWithExpiration(this::fetchCompactions, expirationTimeMinutes, MINUTES); private final Supplier compactorInfoSupplier = @@ -577,7 +576,7 @@ public static class CompactionStats { * @return active tablet server scans. Values are cached and refresh after * {@link #expirationTimeMinutes}. */ - public Map getScans() { + public Map getScans() { return tserverScansSupplier.get(); } @@ -585,14 +584,14 @@ public Map getScans() { * @return active scan server scans. Values are cached and refresh after * {@link #expirationTimeMinutes}. */ - public Map getScanServerScans() { + public Map getScanServerScans() { return sserverScansSupplier.get(); } /** * @return active compactions. Values are cached and refresh after {@link #expirationTimeMinutes}. */ - public Map getCompactions() { + public Map getCompactions() { return compactionsSupplier.get(); } @@ -628,16 +627,15 @@ public RunningCompactorDetails getRunningCompactorDetails(ExternalCompactionId e return new RunningCompactorDetails(extCompaction); } - private Map fetchScans(Collection servers) { + private Map fetchScans(Collection servers) { ServerContext context = getContext(); - Map scans = new HashMap<>(); + Map scans = new HashMap<>(); for (ServerId server : servers) { - final HostAndPort parsedServer = HostAndPort.fromString(server.toHostPortString()); TabletScanClientService.Client client = null; try { - client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SCAN, parsedServer, context); + client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SCAN, server, context); List activeScans = client.getActiveScans(null, context.rpcCreds()); - scans.put(parsedServer, new ScanStats(activeScans)); + scans.put(server.toString(), new ScanStats(activeScans)); } catch (Exception ex) { log.error("Failed to get active scans from {}", server, ex); } finally { @@ -647,24 +645,23 @@ private Map fetchScans(Collection servers) { return Collections.unmodifiableMap(scans); } - private Map fetchTServerScans() { + private Map fetchTServerScans() { return fetchScans(getContext().instanceOperations().getServers(TABLET_SERVER)); } - private Map fetchSServerScans() { + private Map fetchSServerScans() { return fetchScans(getContext().instanceOperations().getServers(SCAN_SERVER)); } - private Map fetchCompactions() { + private Map fetchCompactions() { ServerContext context = getContext(); - Map allCompactions = new HashMap<>(); + Map allCompactions = new HashMap<>(); for (ServerId server : context.instanceOperations().getServers(TABLET_SERVER)) { - final HostAndPort parsedServer = HostAndPort.fromString(server.toHostPortString()); Client tserver = null; try { - tserver = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context); + tserver = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, server, context); var compacts = tserver.getActiveCompactions(null, context.rpcCreds()); - allCompactions.put(parsedServer, new CompactionStats(compacts)); + allCompactions.put(server, new CompactionStats(compacts)); } catch (Exception ex) { log.debug("Failed to get active compactions from {}", server, ex); } finally { @@ -725,7 +722,7 @@ private ExternalCompactionsSnapshot computeExternalCompactionsSnapshot() { /** * Get the monitor lock in ZooKeeper */ - private void getMonitorLock(HostAndPort monitorLocation) + private void getMonitorLock(ServerId monitorLocation) throws KeeperException, InterruptedException { ServerContext context = getContext(); final var monitorLockPath = context.getServerPaths().createMonitorPath(); @@ -764,9 +761,7 @@ private void getMonitorLock(HostAndPort monitorLocation) while (true) { monitorLock.lock(monitorLockWatcher, - new ServiceLockData(zooLockUUID, - monitorLocation.getHost() + ":" + monitorLocation.getPort(), ThriftService.NONE, - this.getResourceGroup())); + new ServiceLockData(zooLockUUID, monitorLocation, ThriftService.NONE)); monitorLockWatcher.waitForChange(); @@ -836,7 +831,7 @@ public double getLookupRate() { return lookupRateTracker.calculateRate(); } - public Optional getCoordinatorHost() { + public Optional getCoordinatorHost() { return coordinatorHost; } diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/next/InformationFetcher.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/next/InformationFetcher.java index af832ee3e1c..34a542b5180 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/next/InformationFetcher.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/next/InformationFetcher.java @@ -66,7 +66,6 @@ import com.github.benmanes.caffeine.cache.RemovalCause; import com.github.benmanes.caffeine.cache.RemovalListener; import com.github.benmanes.caffeine.cache.Scheduler; -import com.google.common.net.HostAndPort; public class InformationFetcher implements RemovalListener, Runnable { @@ -125,8 +124,7 @@ private MetricFetcher(ServerContext ctx, ServerId server, SystemInformation summ @Override public void run() { try { - Client metricsClient = ThriftUtil.getClient(ThriftClientTypes.SERVER_PROCESS, - HostAndPort.fromParts(server.getHost(), server.getPort()), ctx); + Client metricsClient = ThriftUtil.getClient(ThriftClientTypes.SERVER_PROCESS, server, ctx); try { MetricResponse response = metricsClient.getMetrics(TraceUtil.traceInfo(), ctx.rpcCreds()); summary.processResponse(server, response); @@ -186,22 +184,21 @@ private Map getLongRunningCompactions() { if (managers.isEmpty()) { throw new IllegalStateException(coordinatorMissingMsg); } - ServerId manager = managers.iterator().next(); - HostAndPort hp = HostAndPort.fromParts(manager.getHost(), manager.getPort()); + final ServerId manager = managers.iterator().next(); try { CompactionCoordinatorService.Client client = - ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, hp, ctx); + ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, manager, ctx); try { return client.getLongRunningCompactions(TraceUtil.traceInfo(), ctx.rpcCreds()); } catch (Exception e) { - throw new IllegalStateException("Unable to get running compactions from " + hp, e); + throw new IllegalStateException("Unable to get running compactions from " + manager, e); } finally { if (client != null) { ThriftUtil.returnClient(client, ctx); } } } catch (TTransportException e) { - LOG.error("Unable to get Compaction coordinator at {}", hp, e); + LOG.error("Unable to get Compaction coordinator at {}", manager, e); throw new IllegalStateException(coordinatorMissingMsg, e); } } diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/next/SystemInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/next/SystemInformation.java index 89297e98bfc..743d18397c6 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/next/SystemInformation.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/next/SystemInformation.java @@ -273,7 +273,7 @@ public void addResponded() { } public void addNotResponded(ServerId server) { - notResponded.add(server.getHost() + ":" + server.getPort()); + notResponded.add(server.toHostPortString()); } public long getConfigured() { diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/CompactionInfo.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/CompactionInfo.java index f222bc42ed1..1be8b2207a7 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/CompactionInfo.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/CompactionInfo.java @@ -18,10 +18,9 @@ */ package org.apache.accumulo.monitor.rest.compactions; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.monitor.Monitor; -import com.google.common.net.HostAndPort; - /** * Generates a compaction info JSON object * @@ -41,8 +40,8 @@ public CompactionInfo() {} /** * Stores new compaction information */ - public CompactionInfo(HostAndPort address, Monitor.CompactionStats stats) { - this.server = address.toString(); + public CompactionInfo(ServerId address, Monitor.CompactionStats stats) { + this.server = address.toHostPortString(); this.fetched = stats.fetched; this.count = stats.count; this.oldest = stats.oldest; diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/CompactionsResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/CompactionsResource.java index b96f9f774d7..42cc8d2829d 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/CompactionsResource.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/CompactionsResource.java @@ -26,11 +26,10 @@ import jakarta.ws.rs.Produces; import jakarta.ws.rs.core.MediaType; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo; import org.apache.accumulo.monitor.Monitor; -import com.google.common.net.HostAndPort; - /** * Generate a new Compaction list JSON object * @@ -56,7 +55,7 @@ public Compactions getActiveCompactions() { return compactions; } - Map entry = monitor.getCompactions(); + Map entry = monitor.getCompactions(); entry.forEach((k, v) -> compactions.addCompaction(new CompactionInfo(k, v))); diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/CoordinatorInfo.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/CoordinatorInfo.java index 059347e3f12..c3935a79d9f 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/CoordinatorInfo.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/CoordinatorInfo.java @@ -23,8 +23,6 @@ import org.apache.accumulo.core.client.admin.servers.ServerId; -import com.google.common.net.HostAndPort; - public class CoordinatorInfo { // Variable names become JSON keys @@ -33,8 +31,8 @@ public class CoordinatorInfo { public long numQueues; public int numCompactors; - public CoordinatorInfo(Optional serverOpt, ExternalCompactionInfo ecInfo) { - server = serverOpt.map(HostAndPort::toString).orElse("none"); + public CoordinatorInfo(Optional serverOpt, ExternalCompactionInfo ecInfo) { + server = serverOpt.orElseThrow().toHostPortString(); Set compactors = ecInfo.getCompactors(); numQueues = compactors.stream().map(csi -> csi.getResourceGroup()).distinct().count(); numCompactors = compactors.size(); diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/ExternalCompactionInfo.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/ExternalCompactionInfo.java index 6d0e9f798a4..e2550efa1f0 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/ExternalCompactionInfo.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/ExternalCompactionInfo.java @@ -24,22 +24,20 @@ import org.apache.accumulo.core.client.admin.servers.ServerId; -import com.google.common.net.HostAndPort; - /** * Bag of everything going on with external compactions. */ public class ExternalCompactionInfo { - private Optional coordinatorHost; + private Optional coordinatorHost; private Set compactors = new HashSet<>(); private long fetchedTimeMillis; - public void setCoordinatorHost(Optional coordinatorHost) { + public void setCoordinatorHost(Optional coordinatorHost) { this.coordinatorHost = coordinatorHost; } - public Optional getCoordinatorHost() { + public Optional getCoordinatorHost() { return coordinatorHost; } diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/scans/ScansResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/scans/ScansResource.java index 5ff0f456a25..c54c45fcae3 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/scans/ScansResource.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/scans/ScansResource.java @@ -31,8 +31,6 @@ import org.apache.accumulo.monitor.Monitor; import org.apache.accumulo.monitor.Monitor.ScanStats; -import com.google.common.net.HostAndPort; - /** * Generate a new Scan list JSON object * @@ -58,11 +56,11 @@ public Scans getActiveScans() throws Exception { return scans; } - Map entry = monitor.getScans(); + Map entry = monitor.getScans(); // Adds new scans to the array for tservers known to the Manager for (TabletServerStatus tserverInfo : mmi.getTServerInfo()) { - ScanStats stats = entry.get(HostAndPort.fromString(tserverInfo.name)); + ScanStats stats = entry.get(tserverInfo.name); if (stats != null) { scans.addScan(new ScanInformation(tserverInfo.name, stats)); } diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java index 3121b65715d..d892f85df10 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java @@ -147,7 +147,7 @@ public static TableInformationList getTables(Monitor monitor) { if (SystemTables.ROOT.tableId().equals(tableId)) { var rootLoc = monitor.getContext().getAmple().readTablet(RootTable.EXTENT).getLocation(); if (rootLoc != null && rootLoc.getType() == TabletMetadata.LocationType.CURRENT) { - locs.add(rootLoc.getHostPort()); + locs.add(rootLoc.getServerInstance().getServer().toHostPortString()); } } else { var level = Ample.DataLevel.of(tableId); @@ -156,7 +156,7 @@ public static TableInformationList getTables(Monitor monitor) { for (TabletMetadata tm : tablets) { try { - locs.add(tm.getLocation().getHostPort()); + locs.add(tm.getLocation().getServerInstance().getServer().toHostPortString()); } catch (Exception ex) { return tabletServers; } diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java index b21407049c7..1c9794e64b9 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java @@ -37,6 +37,7 @@ import jakarta.ws.rs.core.MediaType; import org.apache.accumulo.core.clientImpl.ClientContext; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo; @@ -48,7 +49,6 @@ import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService; import org.apache.accumulo.core.tabletserver.thrift.TabletStats; import org.apache.accumulo.core.trace.TraceUtil; -import org.apache.accumulo.core.util.AddressUtil; import org.apache.accumulo.monitor.Monitor; import org.apache.accumulo.monitor.rest.manager.ManagerResource; import org.apache.accumulo.server.manager.state.DeadServerList; @@ -125,7 +125,7 @@ public TabletServersRecovery getTserverRecovery() { for (TabletServerStatus server : mmi.tServerInfo) { if (server.logSorts != null) { for (RecoveryStatus recovery : server.logSorts) { - String serv = AddressUtil.parseAddress(server.name).getHost(); + String serv = HostAndPort.fromString(server.name).getHost(); String log = recovery.name; int time = recovery.runtime; double progress = recovery.progress; @@ -178,7 +178,8 @@ public TabletServerSummary getTserverDetails( try { ClientContext context = monitor.getContext(); TabletServerClientService.Client client = - ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context); + ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, + ServerIdUtil.tserver(address.getHost(), address.getPort()), context); try { for (String tableId : mmi.tableMap.keySet()) { tsStats.addAll(client.getTabletStats(TraceUtil.traceInfo(), context.rpcCreds(), tableId)); diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java index 172656000cc..9256dc2eacb 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java @@ -101,7 +101,7 @@ public void run() { try { tabletMetadata = server.getContext().getAmple().readTablet(extent); - canLoad = checkTabletMetadata(extent, server.getTabletSession(), tabletMetadata); + canLoad = checkTabletMetadata(extent, server.getTServerInstance(), tabletMetadata); } catch (Exception e) { synchronized (server.openingTablets) { server.openingTablets.remove(extent); @@ -150,7 +150,7 @@ public void run() { } Assignment assignment = - new Assignment(extent, server.getTabletSession(), tabletMetadata.getLast()); + new Assignment(extent, server.getTServerInstance(), tabletMetadata.getLast()); TabletStateStore.setLocation(server.getContext(), assignment); // refresh the tablet metadata after setting the location (See #3358) diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java index 71303627227..258cbcf94f9 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java @@ -130,6 +130,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Sets; +import com.google.common.net.HostAndPort; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; @@ -315,8 +316,9 @@ private ServiceLock announceExistence() { final ZooReaderWriter zoo = getContext().getZooSession().asReaderWriter(); try { - final ServiceLockPath zLockPath = - context.getServerPaths().createScanServerPath(getResourceGroup(), getAdvertiseAddress()); + final ServiceLockPath zLockPath = context.getServerPaths().createScanServerPath( + getResourceGroup(), + HostAndPort.fromParts(getAdvertiseAddress().getHost(), getAdvertiseAddress().getPort())); ServiceLockSupport.createNonHaServiceLockPath(Type.SCAN_SERVER, zoo, zLockPath); serverLockUUID = UUID.randomUUID(); scanServerLock = new ServiceLock(getContext().getZooSession(), zLockPath, serverLockUUID); @@ -329,8 +331,7 @@ private ServiceLock announceExistence() { ServiceDescriptors descriptors = new ServiceDescriptors(); for (ThriftService svc : new ThriftService[] {ThriftService.CLIENT, ThriftService.TABLET_SCAN}) { - descriptors.addService(new ServiceDescriptor(serverLockUUID, svc, - getAdvertiseAddress().toString(), this.getResourceGroup())); + descriptors.addService(new ServiceDescriptor(serverLockUUID, svc, getAdvertiseAddress())); } if (scanServerLock.tryLock(lw, new ServiceLockData(descriptors))) { @@ -378,7 +379,7 @@ public void run() { metricsInfo.addMetricsProducers(this, scanMetrics, scanServerMetrics, blockCacheMetrics); metricsInfo.init(MetricsInfo.serviceTags(getContext().getInstanceName(), getApplicationName(), - getAdvertiseAddress(), getResourceGroup())); + getAdvertiseAddress())); // We need to set the compaction manager so that we don't get an NPE in CompactableImpl.close ServiceLock lock = announceExistence(); diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java index 5de50cfe935..d00f8e26ad4 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java @@ -1039,7 +1039,7 @@ public void loadTablet(TInfo tinfo, TCredentials credentials, String lock, } } - TabletLogger.loading(extent, server.getTabletSession()); + TabletLogger.loading(extent, server.getTServerInstance()); final AssignmentHandler ah = new AssignmentHandler(server, extent); // Root tablet assignment must take place immediately diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java index 43be28270d6..806babd9831 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java @@ -42,6 +42,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.SortedMap; @@ -203,7 +204,7 @@ public PausedCompactionMetrics getPausedCompactionMetrics() { private ServiceLock tabletServerLock; private volatile ZooUtil.LockID lockID; - private volatile long lockSessionId = -1; + private volatile TServerInstance instance; public static final AtomicLong seekCount = new AtomicLong(0); @@ -417,13 +418,13 @@ private void startServer(String address, TProcessor processor) throws UnknownHos }, true); } - private HostAndPort getManagerAddress() { + private ServerId getManagerAddress() { try { Set managers = getContext().instanceOperations().getServers(ServerId.Type.MANAGER); if (managers == null || managers.isEmpty()) { return null; } - return HostAndPort.fromString(managers.iterator().next().toHostPortString()); + return managers.iterator().next(); } catch (Exception e) { log.warn("Failed to obtain manager host " + e); } @@ -432,7 +433,7 @@ private HostAndPort getManagerAddress() { } // Connect to the manager for posting asynchronous results - private ManagerClientService.Client managerConnection(HostAndPort address) { + private ManagerClientService.Client managerConnection(ServerId address) { try { if (address == null) { return null; @@ -480,12 +481,13 @@ public ServiceLock getLock() { return tabletServerLock; } - private void announceExistence() { + private long announceExistence() { final ZooReaderWriter zoo = getContext().getZooSession().asReaderWriter(); try { - final ServiceLockPath zLockPath = context.getServerPaths() - .createTabletServerPath(getResourceGroup(), getAdvertiseAddress()); + final ServiceLockPath zLockPath = context.getServerPaths().createTabletServerPath( + getResourceGroup(), + HostAndPort.fromParts(getAdvertiseAddress().getHost(), getAdvertiseAddress().getPort())); ServiceLockSupport.createNonHaServiceLockPath(Type.TABLET_SERVER, zoo, zLockPath); UUID tabletServerUUID = UUID.randomUUID(); tabletServerLock = new ServiceLock(getContext().getZooSession(), zLockPath, tabletServerUUID); @@ -500,17 +502,17 @@ private void announceExistence() { for (ThriftService svc : new ThriftService[] {ThriftService.CLIENT, ThriftService.TABLET_INGEST, ThriftService.TABLET_MANAGEMENT, ThriftService.TABLET_SCAN, ThriftService.TSERV}) { - descriptors.addService(new ServiceDescriptor(tabletServerUUID, svc, - getAdvertiseAddress().toString(), this.getResourceGroup())); + descriptors + .addService(new ServiceDescriptor(tabletServerUUID, svc, getAdvertiseAddress())); } if (tabletServerLock.tryLock(lw, new ServiceLockData(descriptors))) { lockID = tabletServerLock.getLockID(); - lockSessionId = tabletServerLock.getSessionId(); + final long lockSessionId = tabletServerLock.getSessionId(); log.debug("Obtained tablet server lock {} {}", tabletServerLock.getLockPath(), - getTabletSession()); + lockSessionId); startServiceLockVerificationThread(); - return; + return lockSessionId; } log.info("Waiting for tablet server lock"); sleepUninterruptibly(5, TimeUnit.SECONDS); @@ -562,13 +564,14 @@ public void run() { metricsInfo.addMetricsProducers(this, metrics, updateMetrics, scanMetrics, mincMetrics, pausedMetrics, blockCacheMetrics); metricsInfo.init(MetricsInfo.serviceTags(context.getInstanceName(), getApplicationName(), - getAdvertiseAddress(), getResourceGroup())); + getAdvertiseAddress())); - announceExistence(); + final long lockSessionId = announceExistence(); getContext().setServiceLock(tabletServerLock); + setTServerInstance(lockSessionId); try { - walMarker.initWalMarker(getTabletSession()); + walMarker.initWalMarker(getTServerInstance()); } catch (Exception e) { log.error("Unable to create WAL marker node in zookeeper", e); throw new RuntimeException(e); @@ -598,7 +601,7 @@ public void run() { evaluateOnDemandTabletsForUnload(); }); - HostAndPort managerHost; + ServerId managerHost; final String advertiseAddressString = getAdvertiseAddress().toString(); while (!isShutdownRequested()) { if (Thread.currentThread().isInterrupted()) { @@ -676,7 +679,7 @@ public void run() { Halt.halt(1, "Error informing Manager that we are shutting down, exiting!"); } else { iface.tabletServerStopping(TraceUtil.traceInfo(), getContext().rpcCreds(), - getTabletSession().getHostPortSession(), getResourceGroup().canonical()); + getTServerInstance().serialize(), getResourceGroup().canonical()); } boolean managerDown = false; @@ -726,20 +729,13 @@ private boolean sendManagerMessages(boolean managerDown, ManagerClientService.Cl return managerDown; } - public TServerInstance getTabletSession() { - if (getAdvertiseAddress() == null) { - return null; - } - if (lockSessionId == -1) { - return null; - } + public void setTServerInstance(final long lockSessionId) { + this.instance = new TServerInstance(getAdvertiseAddress(), lockSessionId); + } - try { - return new TServerInstance(getAdvertiseAddress().toString(), lockSessionId); - } catch (Exception ex) { - log.warn("Unable to read session from tablet server lock" + ex); - return null; - } + public TServerInstance getTServerInstance() { + Objects.requireNonNull(this.instance, "TServerInstance not set yet"); + return this.instance; } private static void checkWalCanSync(ServerContext context) { @@ -845,7 +841,7 @@ public TabletServerStatus getStats(Map> scanCou result.lastContact = RelativeTime.currentTimeMillis(); result.tableMap = tables; result.osLoad = ManagementFactory.getOperatingSystemMXBean().getSystemLoadAverage(); - result.name = String.valueOf(getAdvertiseAddress()); + result.name = getAdvertiseAddress().toString(); result.holdTime = resourceManager.holdTime(); result.lookups = seekCount.get(); result.indexCacheHits = resourceManager.getIndexCache().getStats().hitCount(); @@ -1020,7 +1016,7 @@ private void markUnusedWALs() { Set eligible = findOldestUnreferencedWals(closedCopy, refRemover); try { - TServerInstance session = this.getTabletSession(); + TServerInstance session = this.getTServerInstance(); for (DfsLogger candidate : eligible) { log.info("Marking " + candidate.getPath() + " as unreferenced"); walMarker.walUnreferenced(session, candidate.getPath()); @@ -1035,7 +1031,7 @@ private void markUnusedWALs() { public void addNewLogMarker(DfsLogger copy) throws WalMarkerException { log.info("Writing log marker for " + copy.getPath()); - walMarker.addNewWalMarker(getTabletSession(), copy.getPath()); + walMarker.addNewWalMarker(getTServerInstance(), copy.getPath()); } public void walogClosed(DfsLogger currentLog) throws WalMarkerException { @@ -1048,7 +1044,7 @@ public void walogClosed(DfsLogger currentLog) throws WalMarkerException { clSize = closedLogs.size(); } log.info("Marking " + currentLog.getPath() + " as closed. Total closed logs " + clSize); - walMarker.closeWal(getTabletSession(), currentLog.getPath()); + walMarker.closeWal(getTServerInstance(), currentLog.getPath()); // whenever a new log is added to the set of closed logs, go through all of the tablets and // see if any need to minor compact @@ -1067,7 +1063,7 @@ public void walogClosed(DfsLogger currentLog) throws WalMarkerException { } else { log.info( "Marking " + currentLog.getPath() + " as unreferenced (skipping closed writes == 0)"); - walMarker.walUnreferenced(getTabletSession(), currentLog.getPath()); + walMarker.walUnreferenced(getTServerInstance(), currentLog.getPath()); } } @@ -1183,7 +1179,7 @@ public void evaluateOnDemandTabletsForUnload() { } private void removeHostingRequests(Collection extents) { - var myLocation = TabletMetadata.Location.current(getTabletSession()); + var myLocation = TabletMetadata.Location.current(getTServerInstance()); try (var tabletsMutator = getContext().getAmple().conditionallyMutateTablets()) { extents.forEach(ke -> { diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java index 559cc8a487e..ead78031f02 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java @@ -39,6 +39,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.accumulo.core.client.Durability; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.file.blockfile.impl.BasicCacheProvider; @@ -281,7 +282,7 @@ private synchronized void startLogMaker() { try { alog = DfsLogger.createNew(tserver.getContext(), syncCounter, flushCounter, - tserver.getAdvertiseAddress().toString()); + ServerIdUtil.toWalFileName(tserver.getAdvertiseAddress())); } catch (Exception t) { log.error("Failed to open WAL", t); // the log is not advertised in ZK yet, so we can just delete it if it exists diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanfileManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanfileManager.java index 78019e59cae..7fe17910741 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanfileManager.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanfileManager.java @@ -71,7 +71,7 @@ public void removeBatchedScanRefs() { synchronized (tablet) { snapshot = new HashSet<>(filesToDeleteAfterScan); filesToDeleteAfterScan.clear(); - currLoc = Location.current(tablet.getTabletServer().getTabletSession()); + currLoc = Location.current(tablet.getTabletServer().getTServerInstance()); } removeFilesAfterScan(snapshot, currLoc); } 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 1c96bddbe70..583c60ab7bd 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 @@ -300,7 +300,7 @@ public Tablet(final TabletServer tabletServer, final KeyExtent extent, if (entriesUsedOnTablet.get() == 0) { log.debug("No replayed mutations applied, removing unused walog entries for {}", extent); - final Location expectedLocation = Location.future(this.tabletServer.getTabletSession()); + final Location expectedLocation = Location.future(this.tabletServer.getTServerInstance()); try (ConditionalTabletsMutator mutator = getContext().getAmple().conditionallyMutateTablets()) { ConditionalTabletMutator mut = mutator.mutateTablet(extent).requireAbsentOperation() @@ -347,7 +347,7 @@ public Tablet(final TabletServer tabletServer, final KeyExtent extent, computeNumEntries(); getScanfileManager().removeFilesAfterScan(metadata.getScans(), - Location.future(tabletServer.getTabletSession())); + Location.future(tabletServer.getTServerInstance())); } public TabletMetadata getMetadata() { @@ -496,7 +496,7 @@ public void flush(long tableFlushID) { if (lastTabletMetadata.getFlushId().orElse(-1) < tableFlushID) { try (var tabletsMutator = getContext().getAmple().conditionallyMutateTablets()) { var tablet = tabletsMutator.mutateTablet(extent) - .requireLocation(Location.current(tabletServer.getTabletSession())) + .requireLocation(Location.current(tabletServer.getTServerInstance())) .requireSame(lastTabletMetadata, ColumnType.FLUSH_ID); tablet.putFlushId(tableFlushID); @@ -507,7 +507,7 @@ public void flush(long tableFlushID) { if (result.getStatus() != Ample.ConditionalResult.Status.ACCEPTED) { throw new IllegalStateException("Failed to update flush id " + extent + " " - + tabletServer.getTabletSession() + " " + tableFlushID); + + tabletServer.getTServerInstance() + " " + tableFlushID); } } @@ -1397,7 +1397,7 @@ private Optional updateTabletDataFile(long maxCommittedTime, var lastTabletMetadata = getMetadata(); return updateTabletDataFile(getContext().getAmple(), maxCommittedTime, newDatafile, dfv, - unusedWalLogs, flushId, mincReason, tabletServer.getTabletSession(), extent, + unusedWalLogs, flushId, mincReason, tabletServer.getTServerInstance(), extent, lastTabletMetadata, tabletTime, RANDOM.get().nextLong()); } @@ -1709,7 +1709,7 @@ private boolean refreshMetadata(RefreshPurpose refreshPurpose, Long observedRefr Preconditions.checkArgument(tabletMetadata != null); } - if (tabletMetadata.getLocation() == null || !tabletServer.getTabletSession() + if (tabletMetadata.getLocation() == null || !tabletServer.getTServerInstance() .equals(tabletMetadata.getLocation().getServerInstance())) { log.debug("Unable to refresh tablet {} for {} because it has a different location {}", extent, refreshPurpose, tabletMetadata.getLocation()); @@ -1776,7 +1776,7 @@ private boolean refreshMetadata(RefreshPurpose refreshPurpose, Long observedRefr if (refreshPurpose == RefreshPurpose.REFRESH_RPC) { scanfileManager.removeFilesAfterScan(getMetadata().getScans(), - Location.current(tabletServer.getTabletSession())); + Location.current(tabletServer.getTServerInstance())); } return true; diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/CheckTabletMetadataTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/CheckTabletMetadataTest.java index 65003ab4f92..38c02bfb2b7 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/CheckTabletMetadataTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/CheckTabletMetadataTest.java @@ -27,6 +27,8 @@ import java.util.EnumSet; import java.util.TreeMap; +import org.apache.accumulo.core.client.admin.servers.ServerId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.data.Value; @@ -42,6 +44,8 @@ import org.apache.hadoop.io.Text; import org.junit.jupiter.api.Test; +import com.google.common.net.HostAndPort; + public class CheckTabletMetadataTest { private static Key newKey(String row, ColumnFQ cfq) { @@ -58,9 +62,9 @@ private static void put(TreeMap tabletMeta, String row, ColumnFQ cfq, } private static void put(TreeMap tabletMeta, String row, Text cf, String cq, - String val) { + ServerId server) { Key k = new Key(new Text(row), cf, new Text(cq)); - tabletMeta.put(k, new Value(val)); + tabletMeta.put(k, new Value(new TServerInstance(server, cq).serialize())); } private static void assertFail(TreeMap tabletMeta, KeyExtent ke, TServerInstance tsi) { @@ -86,9 +90,19 @@ private static void assertFail(TreeMap tabletMeta, KeyExtent ke, TSer } } + private static ServerId csi(String location) { + var hp = HostAndPort.fromString(location); + return ServerIdUtil.tserver(hp.getHost(), hp.getPort()); + } + @Test public void testBadTabletMetadata() throws Exception { + final var server1 = csi("127.0.0.1:9997"); + final var server2 = csi("127.0.0.1:9998"); + final var server3 = csi("127.0.0.2:9997"); + final var server4 = csi("127.0.0.2:9999"); + KeyExtent ke = new KeyExtent(TableId.of("1"), null, null); TreeMap tabletMeta = new TreeMap<>(); @@ -97,19 +111,19 @@ public void testBadTabletMetadata() throws Exception { TabletColumnFamily.encodePrevEndRow(null).get()); put(tabletMeta, "1<", ServerColumnFamily.DIRECTORY_COLUMN, "t1".getBytes(UTF_8)); put(tabletMeta, "1<", ServerColumnFamily.TIME_COLUMN, "M0".getBytes(UTF_8)); - put(tabletMeta, "1<", FutureLocationColumnFamily.NAME, "4", "127.0.0.1:9997"); + put(tabletMeta, "1<", FutureLocationColumnFamily.NAME, "4", server1); - TServerInstance tsi = new TServerInstance("127.0.0.1:9997", 4); + TServerInstance tsi = new TServerInstance(server1, 4); TabletMetadata tm = TabletMetadata.convertRow(tabletMeta.entrySet().iterator(), EnumSet.allOf(ColumnType.class), true, false); assertTrue(checkTabletMetadata(ke, tsi, tm)); - assertFail(tabletMeta, ke, new TServerInstance("127.0.0.1:9998", 4)); - assertFail(tabletMeta, ke, new TServerInstance("127.0.0.1:9998", 5)); - assertFail(tabletMeta, ke, new TServerInstance("127.0.0.1:9997", 5)); - assertFail(tabletMeta, ke, new TServerInstance("127.0.0.2:9997", 4)); - assertFail(tabletMeta, ke, new TServerInstance("127.0.0.2:9997", 5)); + assertFail(tabletMeta, ke, new TServerInstance(server2, 4)); + assertFail(tabletMeta, ke, new TServerInstance(server2, 5)); + assertFail(tabletMeta, ke, new TServerInstance(server1, 5)); + assertFail(tabletMeta, ke, new TServerInstance(server3, 4)); + assertFail(tabletMeta, ke, new TServerInstance(server3, 5)); assertFail(tabletMeta, new KeyExtent(TableId.of("1"), null, new Text("m")), tsi); @@ -124,18 +138,18 @@ public void testBadTabletMetadata() throws Exception { assertFail(tabletMeta, ke, tsi, newKey("1<", FutureLocationColumnFamily.NAME, "4")); TreeMap copy = new TreeMap<>(tabletMeta); - put(copy, "1<", CurrentLocationColumnFamily.NAME, "4", "127.0.0.1:9997"); + put(copy, "1<", CurrentLocationColumnFamily.NAME, "4", server1); assertFail(copy, ke, tsi); assertFail(copy, ke, tsi, newKey("1<", FutureLocationColumnFamily.NAME, "4")); copy = new TreeMap<>(tabletMeta); - put(copy, "1<", CurrentLocationColumnFamily.NAME, "5", "127.0.0.1:9998"); + put(copy, "1<", CurrentLocationColumnFamily.NAME, "5", server2); assertFail(copy, ke, tsi); - put(copy, "1<", CurrentLocationColumnFamily.NAME, "6", "127.0.0.1:9999"); + put(copy, "1<", CurrentLocationColumnFamily.NAME, "6", server4); assertFail(copy, ke, tsi); copy = new TreeMap<>(tabletMeta); - put(copy, "1<", FutureLocationColumnFamily.NAME, "5", "127.0.0.1:9998"); + put(copy, "1<", FutureLocationColumnFamily.NAME, "5", server2); assertFail(copy, ke, tsi); assertFail(new TreeMap<>(), ke, tsi); diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/WalRemovalOrderTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/WalRemovalOrderTest.java index 2e682b4231a..967d0bc15f7 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/WalRemovalOrderTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/WalRemovalOrderTest.java @@ -34,7 +34,8 @@ public class WalRemovalOrderTest { private DfsLogger mockLogger(String filename) { - var mockLogEntry = LogEntry.fromPath(filename + "+1234/11111111-1111-1111-1111-111111111111"); + var mockLogEntry = + LogEntry.fromPath("default+" + filename + "+1234/11111111-1111-1111-1111-111111111111"); return DfsLogger.fromLogEntry(mockLogEntry); } diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/RecoveryLogsIteratorTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/RecoveryLogsIteratorTest.java index dbb884d0915..eda171e1095 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/RecoveryLogsIteratorTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/RecoveryLogsIteratorTest.java @@ -226,7 +226,7 @@ private void createRecoveryDir(Map logs, ArrayList entry : logs.entrySet()) { var uuid = UUID.randomUUID(); String origPath = "file://" + workDir + "/" + entry.getKey() + "/" - + VolumeManager.FileType.WAL.getDirectory() + "/localhost+9997/" + uuid; + + VolumeManager.FileType.WAL.getDirectory() + "/default+localhost+9997/" + uuid; String destPath = "file://" + workDir + "/" + entry.getKey() + "/" + VolumeManager.FileType.RECOVERY.getDirectory() + "/" + uuid; diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/SortedLogRecoveryTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/SortedLogRecoveryTest.java index 56b1da116f3..8dd3b24be37 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/SortedLogRecoveryTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/SortedLogRecoveryTest.java @@ -203,7 +203,7 @@ private List recover(Map logs, Set files, K for (Entry entry : logs.entrySet()) { var uuid = UUID.randomUUID(); String origPath = "file://" + workdir + "/" + entry.getKey() + "/" - + VolumeManager.FileType.WAL.getDirectory() + "/localhost+9997/" + uuid; + + VolumeManager.FileType.WAL.getDirectory() + "/default+localhost+9997/" + uuid; String destPath = "file://" + workdir + "/" + entry.getKey() + "/" + VolumeManager.FileType.RECOVERY.getDirectory() + "/" + uuid; FileSystem ns = fs.getFileSystemByPath(new Path(destPath)); diff --git a/shell/src/test/java/org/apache/accumulo/shell/commands/ListTabletsCommandTest.java b/shell/src/test/java/org/apache/accumulo/shell/commands/ListTabletsCommandTest.java index b7bd5f1f66e..fa796e942f2 100644 --- a/shell/src/test/java/org/apache/accumulo/shell/commands/ListTabletsCommandTest.java +++ b/shell/src/test/java/org/apache/accumulo/shell/commands/ListTabletsCommandTest.java @@ -38,6 +38,7 @@ import org.apache.accumulo.core.client.admin.TableOperations; import org.apache.accumulo.core.client.admin.TabletAvailability; import org.apache.accumulo.core.clientImpl.ClientContext; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.clientImpl.TabletInformationImpl; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.TableId; @@ -58,8 +59,6 @@ import org.easymock.EasyMock; import org.junit.jupiter.api.Test; -import com.google.common.net.HostAndPort; - public class ListTabletsCommandTest { final static String tableName = ListTabletsCommandTest.class.getName() + "-aTable"; @@ -132,8 +131,8 @@ public void mockTest() throws Exception { TableId tableId = TableId.of("123"); - TServerInstance ser1 = new TServerInstance(HostAndPort.fromParts("server1", 8555), "s001"); - TServerInstance ser2 = new TServerInstance(HostAndPort.fromParts("server2", 2354), "s002"); + TServerInstance ser1 = new TServerInstance(ServerIdUtil.tserver("server1", 8555), "s001"); + TServerInstance ser2 = new TServerInstance(ServerIdUtil.tserver("server2", 2354), "s002"); StoredTabletFile sf11 = new ReferencedTabletFile(new Path("hdfs://nn1/acc/tables/1/t-dir1/sf11.rf")).insert(); @@ -153,8 +152,8 @@ public void mockTest() throws Exception { KeyExtent extent = new KeyExtent(tableId, new Text("d"), null); - LogEntry le1 = LogEntry.fromPath("localhost+8020/" + UUID.randomUUID()); - LogEntry le2 = LogEntry.fromPath("localhost+8020/" + UUID.randomUUID()); + LogEntry le1 = LogEntry.fromPath("default+localhost+8020/" + UUID.randomUUID()); + LogEntry le2 = LogEntry.fromPath("default+localhost+8020/" + UUID.randomUUID()); TabletMetadata tm1 = TabletMetadata.builder(extent).putTabletAvailability(TabletAvailability.ONDEMAND) diff --git a/test/src/main/java/org/apache/accumulo/test/ComprehensiveTableOperationsIT_SimpleSuite.java b/test/src/main/java/org/apache/accumulo/test/ComprehensiveTableOperationsIT_SimpleSuite.java index 8605352c8ba..2fae021cb7e 100644 --- a/test/src/main/java/org/apache/accumulo/test/ComprehensiveTableOperationsIT_SimpleSuite.java +++ b/test/src/main/java/org/apache/accumulo/test/ComprehensiveTableOperationsIT_SimpleSuite.java @@ -302,7 +302,7 @@ public void test_locate() throws Exception { fateGroupByTablet.forEach(tid -> { var tabletLoc = fateLocations.getTabletLocation(tid); assertNotNull(tabletLoc); - assertTrue(tabletLoc.contains(":")); + assertTrue(tabletLoc.toHostPortString().contains(":")); }); var scanRefLocations = ops.locate(SystemTables.SCAN_REF.tableName(), Collections.singletonList(new Range())); @@ -311,7 +311,7 @@ public void test_locate() throws Exception { scanRefGroupByTablet.forEach(tid -> { var tabletLoc = scanRefLocations.getTabletLocation(tid); assertNotNull(tabletLoc); - assertTrue(tabletLoc.contains(":")); + assertTrue(tabletLoc.toHostPortString().contains(":")); }); } diff --git a/test/src/main/java/org/apache/accumulo/test/CorruptMutationIT.java b/test/src/main/java/org/apache/accumulo/test/CorruptMutationIT.java index fdab6e21425..e381a2413bd 100644 --- a/test/src/main/java/org/apache/accumulo/test/CorruptMutationIT.java +++ b/test/src/main/java/org/apache/accumulo/test/CorruptMutationIT.java @@ -77,8 +77,8 @@ public void testCorruptMutation() throws Exception { assertNotNull(location); assertEquals(TabletMetadata.LocationType.CURRENT, location.getType()); - TabletIngestClientService.Iface client = - ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, location.getHostAndPort(), ctx); + TabletIngestClientService.Iface client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, + location.getServerInstance().getServer(), ctx); // Make the same RPC calls made by the BatchWriter, but pass a corrupt serialized mutation in // this try block. try { diff --git a/test/src/main/java/org/apache/accumulo/test/ECAdminIT.java b/test/src/main/java/org/apache/accumulo/test/ECAdminIT.java index b858316a782..2b06735a44b 100644 --- a/test/src/main/java/org/apache/accumulo/test/ECAdminIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ECAdminIT.java @@ -36,6 +36,7 @@ import org.apache.accumulo.core.client.Accumulo; import org.apache.accumulo.core.client.AccumuloClient; import org.apache.accumulo.core.client.IteratorSetting; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.compaction.thrift.TExternalCompactionMap; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil; @@ -52,7 +53,6 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import com.google.common.net.HostAndPort; import com.google.gson.Gson; import com.google.gson.GsonBuilder; import com.google.gson.reflect.TypeToken; @@ -117,7 +117,7 @@ public void testListRunningCompactions() throws Exception { writeData(client, tableName); compact(client, tableName, 2, GROUP7, false); - Optional coordinatorHost = + Optional coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(getCluster().getServerContext()); // wait for the compaction to start @@ -140,7 +140,7 @@ public void testListRunningCompactions() throws Exception { assertNotNull(rcs); assertEquals(tec.getJob().getExternalCompactionId(), rcs.getEcid()); assertEquals(tec.groupName, rcs.getGroup().canonical()); - assertEquals(tec.getCompactor(), rcs.getAddr()); + assertEquals(tec.getCompactor(), rcs.getAddr().toHostPortString()); }); // Confirm JSON output works diff --git a/test/src/main/java/org/apache/accumulo/test/InstanceOperationsIT.java b/test/src/main/java/org/apache/accumulo/test/InstanceOperationsIT.java index 0b4dadc9913..e013632641c 100644 --- a/test/src/main/java/org/apache/accumulo/test/InstanceOperationsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/InstanceOperationsIT.java @@ -34,8 +34,8 @@ import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.admin.InstanceOperations; import org.apache.accumulo.core.client.admin.servers.ServerId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.conf.Property; -import org.apache.accumulo.core.data.ResourceGroupId; import org.apache.accumulo.harness.AccumuloClusterHarness; import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; import org.apache.accumulo.test.util.Wait; @@ -199,8 +199,7 @@ public void testPing() throws Exception { io.ping(sid); } - ServerId fake = - new ServerId(ServerId.Type.COMPACTOR, ResourceGroupId.DEFAULT, "localhost", 1024); + ServerId fake = ServerIdUtil.compactor("localhost", 1024); assertThrows(AccumuloException.class, () -> io.ping(fake)); } diff --git a/test/src/main/java/org/apache/accumulo/test/LocatorIT.java b/test/src/main/java/org/apache/accumulo/test/LocatorIT.java index ff89e81b922..021b2bfdbdf 100644 --- a/test/src/main/java/org/apache/accumulo/test/LocatorIT.java +++ b/test/src/main/java/org/apache/accumulo/test/LocatorIT.java @@ -62,7 +62,7 @@ protected Duration defaultTimeout() { return Duration.ofMinutes(2); } - private void assertContains(Locations locations, HashSet tservers, + private void assertContains(Locations locations, HashSet tservers, Map> expected1, Map> expected2) { Map> gbr = new HashMap<>(); @@ -77,10 +77,11 @@ private void assertContains(Locations locations, HashSet tservers, gbt.put(entry.getKey(), new HashSet<>(entry.getValue())); TabletId tid = entry.getKey(); - String location = locations.getTabletLocation(tid); + ServerId location = locations.getTabletLocation(tid); assertNotNull(location, "Location for " + tid + " was null"); assertTrue(tservers.contains(location), "Unknown location " + location); - assertEquals(2, location.split(":").length, "Expected : " + location); + assertEquals(2, location.toHostPortString().split(":").length, + "Expected : " + location); } @@ -97,7 +98,7 @@ public void testBasic() throws Exception { final Predicate hostedAndCurrentNotNull = t -> t.getTabletAvailability() == TabletAvailability.HOSTED && t.hasCurrent() - && t.getLocation().getHostAndPort() != null; + && t.getLocation().getServerInstance().getServer().getHost() != null; try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) { String tableName = getUniqueNames(1)[0]; @@ -116,9 +117,9 @@ public void testBasic() throws Exception { ArrayList ranges = new ArrayList<>(); - HashSet tservers = new HashSet<>(); + HashSet tservers = new HashSet<>(); client.instanceOperations().getServers(ServerId.Type.TABLET_SERVER) - .forEach((s) -> tservers.add(s.toHostPortString())); + .forEach((s) -> tservers.add(s)); // locate won't find any locations, tablets are not hosted ranges.add(r1); diff --git a/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java index e33abf148bf..a972295ab06 100644 --- a/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java +++ b/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java @@ -114,12 +114,12 @@ public void testEmptyWalRecoveryCompletes() throws Exception { MiniAccumuloClusterImpl cluster = getCluster(); FileSystem fs = cluster.getFileSystem(); - // Fake out something that looks like host:port, it's irrelevant - String fakeServer = "127.0.0.1:12345"; + // Fake out something that looks like group+host+port, it's irrelevant + String fakeServer = "default+127.0.0.1+12345"; java.nio.file.Path walogs = cluster.getConfig().getAccumuloDir().toPath().resolve(Constants.WAL_DIR); - java.nio.file.Path walogServerDir = walogs.resolve(fakeServer.replace(':', '+')); + java.nio.file.Path walogServerDir = walogs.resolve(fakeServer); java.nio.file.Path emptyWalog = walogServerDir.resolve(UUID.randomUUID().toString()); log.info("Created empty WAL at {}", emptyWalog.toUri()); @@ -170,12 +170,12 @@ public void testPartialHeaderWalRecoveryCompletes() throws Exception { MiniAccumuloClusterImpl cluster = getCluster(); FileSystem fs = getCluster().getFileSystem(); - // Fake out something that looks like host:port, it's irrelevant - String fakeServer = "127.0.0.1:12345"; + // Fake out something that looks like group+host+port, it's irrelevant + String fakeServer = "default+127.0.0.1+12345"; java.nio.file.Path walogs = cluster.getConfig().getAccumuloDir().toPath().resolve(Constants.WAL_DIR); - java.nio.file.Path walogServerDir = walogs.resolve(fakeServer.replace(':', '+')); + java.nio.file.Path walogServerDir = walogs.resolve(fakeServer); java.nio.file.Path partialHeaderWalog = walogServerDir.resolve(UUID.randomUUID().toString()); log.info("Created WAL with malformed header at {}", partialHeaderWalog.toUri()); diff --git a/test/src/main/java/org/apache/accumulo/test/TestDualAssignment.java b/test/src/main/java/org/apache/accumulo/test/TestDualAssignment.java index d04ff0d68f0..49b6eaf8b05 100644 --- a/test/src/main/java/org/apache/accumulo/test/TestDualAssignment.java +++ b/test/src/main/java/org/apache/accumulo/test/TestDualAssignment.java @@ -30,9 +30,11 @@ import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.admin.NewTableConfiguration; import org.apache.accumulo.core.client.admin.TabletAvailability; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.core.metadata.schema.TabletMetadata; import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; import org.apache.accumulo.server.ServerContext; @@ -75,8 +77,10 @@ public void test() throws Exception { var tableId = serverContext.getTableId(table); var extent1 = new KeyExtent(tableId, new Text("m"), new Text("l")); - var loc1 = TabletMetadata.Location.current("192.168.1.1:9997", "56"); - var loc2 = TabletMetadata.Location.future("192.168.1.2:9997", "67"); + var loc1 = TabletMetadata.Location + .current(new TServerInstance(ServerIdUtil.tserver("192.168.1.1", 9997), "56")); + var loc2 = TabletMetadata.Location + .future(new TServerInstance(ServerIdUtil.tserver("192.168.1.2", 9997), "67")); // set multiple locations for a tablet serverContext.getAmple().mutateTablet(extent1).putLocation(loc1).putLocation(loc2).mutate(); diff --git a/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java b/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java index 7f85448fa3e..07c2c6c70f1 100644 --- a/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java +++ b/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java @@ -42,8 +42,6 @@ import org.apache.hadoop.conf.Configuration; import org.junit.jupiter.api.Test; -import com.google.common.net.HostAndPort; - public class TotalQueuedIT extends ConfigurableMacBase { @Override @@ -139,8 +137,7 @@ private long getSyncs(AccumuloClient c) throws Exception { ServerContext context = getServerContext(); for (ServerId tserver : c.instanceOperations().getServers(ServerId.Type.TABLET_SERVER)) { TabletServerClientService.Client client = - ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, - HostAndPort.fromParts(tserver.getHost(), tserver.getPort()), context); + ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, tserver, context); TabletServerStatus status = client.getTabletServerStatus(null, context.rpcCreds()); return status.syncs; } diff --git a/test/src/main/java/org/apache/accumulo/test/TransportCachingIT.java b/test/src/main/java/org/apache/accumulo/test/TransportCachingIT.java index d7293c42e5f..eb3bc2b21e1 100644 --- a/test/src/main/java/org/apache/accumulo/test/TransportCachingIT.java +++ b/test/src/main/java/org/apache/accumulo/test/TransportCachingIT.java @@ -47,8 +47,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.net.HostAndPort; - /** * Test that {@link ThriftTransportPool} actually adheres to the cachedConnection argument */ @@ -68,8 +66,7 @@ public void testCachedTransport() throws InterruptedException { List servers = client.instanceOperations().getServers(ServerId.Type.TABLET_SERVER).stream().map(tsi -> { - return new ThriftTransportKey(ThriftClientTypes.CLIENT, - HostAndPort.fromParts(tsi.getHost(), tsi.getPort()), rpcTimeout, context); + return new ThriftTransportKey(ThriftClientTypes.CLIENT, tsi, rpcTimeout, context); }).collect(Collectors.toList()); // only want to use one server for all subsequent test @@ -119,7 +116,7 @@ public void testCachedTransport() throws InterruptedException { pool.returnTransport(sixth); pool.returnTransport(seventh); - Pair eigth = pool.getAnyCachedTransport(ttk.getType(), context, + Pair eigth = pool.getAnyCachedTransport(ttk.getType(), context, ThriftService.CLIENT, ResourceGroupPredicate.exact(ResourceGroupId.of("FAKE"))); assertNull(eigth); @@ -130,7 +127,7 @@ private TTransport getAnyTransport(ThriftTransportKey ttk, ThriftTransportPool p ClientContext context, ThriftService service, ResourceGroupPredicate rgp, boolean preferCached) { if (preferCached) { - Pair cached = + Pair cached = pool.getAnyCachedTransport(ttk.getType(), context, service, rgp); if (cached != null) { return cached.getSecond(); diff --git a/test/src/main/java/org/apache/accumulo/test/WaitForBalanceIT.java b/test/src/main/java/org/apache/accumulo/test/WaitForBalanceIT.java index a0837147e02..1e3a9171e9a 100644 --- a/test/src/main/java/org/apache/accumulo/test/WaitForBalanceIT.java +++ b/test/src/main/java/org/apache/accumulo/test/WaitForBalanceIT.java @@ -33,10 +33,12 @@ import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.client.admin.NewTableConfiguration; import org.apache.accumulo.core.client.admin.TabletAvailability; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.client.admin.servers.ServerId.Type; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.metadata.SystemTables; +import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily; @@ -90,9 +92,8 @@ public void test() throws Exception { } private boolean isBalanced(AccumuloClient c) throws Exception { - final Map tserverCounts = new HashMap<>(); - c.instanceOperations().getServers(Type.TABLET_SERVER) - .forEach(ts -> tserverCounts.put(ts.toHostPortString(), 0)); + final Map tserverCounts = new HashMap<>(); + c.instanceOperations().getServers(Type.TABLET_SERVER).forEach(ts -> tserverCounts.put(ts, 0)); int offline = 0; for (String tableName : new String[] {SystemTables.METADATA.tableName(), SystemTables.ROOT.tableName()}) { @@ -100,21 +101,21 @@ private boolean isBalanced(AccumuloClient c) throws Exception { s.setRange(TabletsSection.getRange()); s.fetchColumnFamily(CurrentLocationColumnFamily.NAME); TabletColumnFamily.PREV_ROW_COLUMN.fetch(s); - String location = null; + TServerInstance location = null; for (Entry entry : s) { Key key = entry.getKey(); if (key.getColumnFamily().equals(CurrentLocationColumnFamily.NAME)) { - location = entry.getValue().toString(); + location = TServerInstance.deserialize(entry.getValue().toString()); } else if (TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key)) { if (location == null) { offline++; } else { - Integer count = tserverCounts.get(location); + Integer count = tserverCounts.get(location.getServer()); if (count == null) { count = 0; } count = count + 1; - tserverCounts.put(location, count); + tserverCounts.put(location.getServer(), count); } location = null; } @@ -132,7 +133,7 @@ private boolean isBalanced(AccumuloClient c) throws Exception { average /= tserverCounts.size(); System.out.println(tserverCounts); int tablesCount = c.tableOperations().list().size(); - for (Entry hostCount : tserverCounts.entrySet()) { + for (Entry hostCount : tserverCounts.entrySet()) { if (Math.abs(average - hostCount.getValue()) > tablesCount) { System.out.println( "Average " + average + " count " + hostCount.getKey() + ": " + hostCount.getValue()); diff --git a/test/src/main/java/org/apache/accumulo/test/ample/usage/TabletFileUpdateIT_SimpleSuite.java b/test/src/main/java/org/apache/accumulo/test/ample/usage/TabletFileUpdateIT_SimpleSuite.java index 85178e1a7ed..73bba1afc78 100644 --- a/test/src/main/java/org/apache/accumulo/test/ample/usage/TabletFileUpdateIT_SimpleSuite.java +++ b/test/src/main/java/org/apache/accumulo/test/ample/usage/TabletFileUpdateIT_SimpleSuite.java @@ -32,6 +32,7 @@ import org.apache.accumulo.core.client.Accumulo; import org.apache.accumulo.core.client.admin.TimeType; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.metadata.ReferencedTabletFile; @@ -69,7 +70,7 @@ public static void teardown() { ReferencedTabletFile.of(new Path("file:///accumulo/tables/t-0/b-0/F1.rf")); private static final DataFileValue dfv1 = new DataFileValue(1000, 100); private static final TServerInstance tserverInstance = - new TServerInstance("localhost:9997", 0xabcdef123L); + new TServerInstance(ServerIdUtil.tserver("localhost", 9997), 0xabcdef123L); private static final TableId tableId = TableId.of("99"); private static final KeyExtent extent = new KeyExtent(tableId, null, null); @@ -203,8 +204,10 @@ public void testLocation() throws Exception { // try locations that differ in type, port, and session for (var location : List.of(Location.future(tserverInstance), - Location.current(new TServerInstance("localhost:9998", 0xabcdef123L)), - Location.current(new TServerInstance("localhost:9997", 0xabcdef124L)))) { + Location + .current(new TServerInstance(ServerIdUtil.tserver("localhost", 9998), 0xabcdef123L)), + Location.current( + new TServerInstance(ServerIdUtil.tserver("localhost", 9997), 0xabcdef124L)))) { // set a location on the tablet that will not match testAmple.mutateTablet(extent).putLocation(location).mutate(); // should fail to add file to tablet because tablet location is not as expected diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ClassLoaderContextCompactionIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ClassLoaderContextCompactionIT.java index 18b5c1cdf18..b5670ff937e 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ClassLoaderContextCompactionIT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ClassLoaderContextCompactionIT.java @@ -41,6 +41,7 @@ import org.apache.accumulo.core.client.AccumuloClient; import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.admin.CompactionConfig; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.ResourceGroupId; @@ -70,8 +71,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.net.HostAndPort; - public class ClassLoaderContextCompactionIT extends AccumuloClusterHarness { private static final Logger LOG = LoggerFactory.getLogger(ClassLoaderContextCompactionIT.class); @@ -174,10 +173,10 @@ public void testClassLoaderContextErrorKillsCompactor() throws Exception { try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) { Wait.waitFor(() -> ExternalCompactionUtil.countCompactors(ResourceGroupId.of(GROUP1), (ClientContext) client) == 1); - Set compactors = + Set compactors = ExternalCompactionUtil.getCompactorAddrs((ClientContext) client).get(GROUP1); assertEquals(1, compactors.size()); - final HostAndPort compactorAddr = compactors.iterator().next(); + final ServerId compactorAddr = compactors.iterator().next(); createTable(client, table1, "cs1"); client.tableOperations().setProperty(table1, TABLE_FILE_MAX.getKey(), "1001"); client.tableOperations().setProperty(table1, TABLE_MAJC_RATIO.getKey(), "1001"); diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java index 073406c1477..4d05fdd8b5d 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java @@ -75,8 +75,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.net.HostAndPort; - /** * Tests that external compactions report progress from start to finish. To prevent flaky test * failures, we only measure progress in quarter segments: STARTED, QUARTER, HALF, THREE_QUARTERS. @@ -327,7 +325,7 @@ public Thread startChecker() { */ private void checkRunning() throws TException { ServerContext ctx = getCluster().getServerContext(); - Optional coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(ctx); + Optional coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(ctx); if (coordinatorHost.isEmpty()) { throw new TTransportException("Unable to get CompactionCoordinator address from ZooKeeper"); } diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java index 18b6edae036..2808686d4e2 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java @@ -83,7 +83,6 @@ import org.apache.thrift.transport.TTransportException; import com.beust.jcommander.internal.Maps; -import com.google.common.net.HostAndPort; public class ExternalCompactionTestUtils { @@ -254,7 +253,7 @@ public static void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuratio } public static TExternalCompactionMap getRunningCompactions(ClientContext context, - Optional coordinatorHost) throws TException { + Optional coordinatorHost) throws TException { CompactionCoordinatorService.Client client = ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, coordinatorHost.orElseThrow(), context); try { @@ -267,7 +266,7 @@ public static TExternalCompactionMap getRunningCompactions(ClientContext context } private static TExternalCompactionMap getCompletedCompactions(ClientContext context, - Optional coordinatorHost) throws Exception { + Optional coordinatorHost) throws Exception { CompactionCoordinatorService.Client client = ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, coordinatorHost.orElseThrow(), context); try { @@ -327,7 +326,7 @@ public static void waitForRunningCompactions(ServerContext ctx, TableId tid, public static int confirmCompactionRunning(ServerContext ctx, Set ecids) throws Exception { int matches = 0; - Optional coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(ctx); + Optional coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(ctx); if (coordinatorHost.isEmpty()) { throw new TTransportException("Unable to get CompactionCoordinator address from ZooKeeper"); } @@ -352,7 +351,7 @@ public static int confirmCompactionRunning(ServerContext ctx, Set ecids, TCompactionState expectedState) throws Exception { - Optional coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(ctx); + Optional coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(ctx); if (coordinatorHost.isEmpty()) { throw new TTransportException("Unable to get CompactionCoordinator address from ZooKeeper"); } diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java index 259c54ea5d1..110d8e423dc 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java @@ -42,6 +42,7 @@ import org.apache.accumulo.core.client.Accumulo; import org.apache.accumulo.core.client.AccumuloClient; import org.apache.accumulo.core.client.IteratorSetting; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService; import org.apache.accumulo.core.compaction.thrift.TCompactionState; import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate; @@ -78,8 +79,6 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import com.google.common.net.HostAndPort; - public class ExternalCompaction_3_IT extends SharedMiniClusterBase { public static class ExternalCompaction3Config implements MiniClusterConfigurationCallback { @@ -249,7 +248,7 @@ private Map getRunningCompactionInfo TExternalCompactionMap running = null; while (running == null || running.getCompactions() == null) { try { - Optional coordinatorHost = + Optional coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(ctx); if (coordinatorHost.isEmpty()) { throw new TTransportException( @@ -286,8 +285,7 @@ private Map getLongRunningCompactions(ServerCont while (results.isEmpty()) { try { - Optional coordinatorHost = - ExternalCompactionUtil.findCompactionCoordinator(ctx); + Optional coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(ctx); if (coordinatorHost.isEmpty()) { throw new TTransportException( "Unable to get CompactionCoordinator address from ZooKeeper"); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/ManagerRepoIT_SimpleSuite.java b/test/src/main/java/org/apache/accumulo/test/fate/ManagerRepoIT_SimpleSuite.java index 0dc1d52e5a1..b4d099a5a0a 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/ManagerRepoIT_SimpleSuite.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/ManagerRepoIT_SimpleSuite.java @@ -160,7 +160,7 @@ public void testNoWalsMergeRepos(MergeInfo.Operation operation) throws Exception // Write a WAL to the test metadata and then re-run the repo to check for an error try (TabletsMutator tm = testAmple.mutateTablets()) { - var walFilePath = Path.of("tserver+8080", UUID.randomUUID().toString()).toString(); + var walFilePath = Path.of("default+tserver+8080", UUID.randomUUID().toString()).toString(); tm.mutateTablet(extent).putWal(LogEntry.fromPath(walFilePath)).mutate(); } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/TestLock.java b/test/src/main/java/org/apache/accumulo/test/fate/TestLock.java index 598aed5c7e7..325060f62a5 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/TestLock.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/TestLock.java @@ -21,7 +21,7 @@ import java.util.UUID; import java.util.concurrent.CountDownLatch; -import org.apache.accumulo.core.data.ResourceGroupId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.fate.user.UserFateStore; import org.apache.accumulo.core.fate.zookeeper.MetaFateStore; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; @@ -69,7 +69,7 @@ public ServiceLock createTestLock(ServerContext context) throws InterruptedExcep TestLockWatcher lw = new TestLockWatcher(); ServiceLockData.ServiceDescriptors descriptors = new ServiceLockData.ServiceDescriptors(); descriptors.addService(new ServiceLockData.ServiceDescriptor(uuid, - ServiceLockData.ThriftService.NONE, "fake_test_host", ResourceGroupId.DEFAULT)); + ServiceLockData.ThriftService.NONE, ServerIdUtil.tserver("fake_test_host", 0))); ServiceLockData sld = new ServiceLockData(descriptors); String lockPath = slp.toString(); String parentLockPath = lockPath.substring(0, lockPath.lastIndexOf("/")); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/AdvertiseAndBindIT.java b/test/src/main/java/org/apache/accumulo/test/functional/AdvertiseAndBindIT.java index 8d7f04ce6d8..2cff00e0278 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/AdvertiseAndBindIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/AdvertiseAndBindIT.java @@ -37,8 +37,6 @@ import org.apache.hadoop.conf.Configuration; import org.junit.jupiter.api.Test; -import com.google.common.net.HostAndPort; - public class AdvertiseAndBindIT extends ConfigurableMacBase { @Override @@ -69,7 +67,7 @@ public void testAdvertiseAndBindArguments() throws Exception { // advertise address. cluster.start(); try { - Map zkAddrs = getAdvertiseAddressFromZooKeeper(); + Map zkAddrs = getAdvertiseAddressFromZooKeeper(); zkAddrs.values().forEach(hp -> assertTrue(hp.getHost().equals(localHostName))); } finally { cluster.stop(); @@ -79,7 +77,7 @@ public void testAdvertiseAndBindArguments() throws Exception { // Set only the bind address restartClusterWithArguments(null, "127.0.0.1"); try { - Map zkAddrs = getAdvertiseAddressFromZooKeeper(); + Map zkAddrs = getAdvertiseAddressFromZooKeeper(); zkAddrs.values().forEach(hp -> assertTrue(hp.getHost().equals("127.0.0.1"))); } finally { cluster.stop(); @@ -89,7 +87,7 @@ public void testAdvertiseAndBindArguments() throws Exception { // Set only the advertise address restartClusterWithArguments("localhost", null); try { - Map zkAddrs = getAdvertiseAddressFromZooKeeper(); + Map zkAddrs = getAdvertiseAddressFromZooKeeper(); zkAddrs.values().forEach(hp -> assertTrue(hp.getHost().equals("localhost"))); } finally { cluster.stop(); @@ -99,7 +97,7 @@ public void testAdvertiseAndBindArguments() throws Exception { // Set advertise and bind address restartClusterWithArguments("localhost", "127.0.0.1"); try { - Map zkAddrs = getAdvertiseAddressFromZooKeeper(); + Map zkAddrs = getAdvertiseAddressFromZooKeeper(); zkAddrs.values().forEach(hp -> assertTrue(hp.getHost().equals("localhost"))); } finally { cluster.stop(); @@ -110,8 +108,9 @@ public void testAdvertiseAndBindArguments() throws Exception { // try to connect to it restartClusterWithArguments("192.168.1.2:59000", "127.0.0.1"); try { - Map zkAddrs = getAdvertiseAddressFromZooKeeper(); - zkAddrs.values().forEach(hp -> assertTrue(hp.toString().equals("192.168.1.2:59000"))); + Map zkAddrs = getAdvertiseAddressFromZooKeeper(); + zkAddrs.values() + .forEach(hp -> assertTrue(hp.getHost().equals("192.168.1.2") && hp.getPort() == 59000)); } finally { cluster.stop(); } @@ -132,7 +131,7 @@ public void testAdvertiseAndBindProperties() throws Exception { // advertise address. cluster.start(); try { - Map zkAddrs = getAdvertiseAddressFromZooKeeper(); + Map zkAddrs = getAdvertiseAddressFromZooKeeper(); zkAddrs.values().forEach(hp -> assertTrue(hp.getHost().equals(localHostName))); } finally { cluster.stop(); @@ -142,7 +141,7 @@ public void testAdvertiseAndBindProperties() throws Exception { // Set only the bind address restartClusterWithProperties(Map.of(Property.RPC_PROCESS_BIND_ADDRESS.getKey(), "127.0.0.1")); try { - Map zkAddrs = getAdvertiseAddressFromZooKeeper(); + Map zkAddrs = getAdvertiseAddressFromZooKeeper(); zkAddrs.values().forEach(hp -> assertTrue(hp.getHost().equals("127.0.0.1"))); } finally { cluster.stop(); @@ -153,7 +152,7 @@ public void testAdvertiseAndBindProperties() throws Exception { restartClusterWithProperties( Map.of(Property.RPC_PROCESS_ADVERTISE_ADDRESS.getKey(), "localhost")); try { - Map zkAddrs = getAdvertiseAddressFromZooKeeper(); + Map zkAddrs = getAdvertiseAddressFromZooKeeper(); zkAddrs.values().forEach(hp -> assertTrue(hp.getHost().equals("localhost"))); } finally { cluster.stop(); @@ -164,7 +163,7 @@ public void testAdvertiseAndBindProperties() throws Exception { restartClusterWithProperties(Map.of(Property.RPC_PROCESS_BIND_ADDRESS.getKey(), "127.0.0.1", Property.RPC_PROCESS_ADVERTISE_ADDRESS.getKey(), "localhost")); try { - Map zkAddrs = getAdvertiseAddressFromZooKeeper(); + Map zkAddrs = getAdvertiseAddressFromZooKeeper(); zkAddrs.values().forEach(hp -> assertTrue(hp.getHost().equals("localhost"))); } finally { cluster.stop(); @@ -176,8 +175,9 @@ public void testAdvertiseAndBindProperties() throws Exception { restartClusterWithProperties(Map.of(Property.RPC_PROCESS_BIND_ADDRESS.getKey(), "127.0.0.1", Property.RPC_PROCESS_ADVERTISE_ADDRESS.getKey(), "192.168.1.1:10005")); try { - Map zkAddrs = getAdvertiseAddressFromZooKeeper(); - zkAddrs.values().forEach(hp -> assertTrue(hp.toString().equals("192.168.1.1:10005"))); + Map zkAddrs = getAdvertiseAddressFromZooKeeper(); + zkAddrs.values() + .forEach(hp -> assertTrue(hp.getHost().equals("192.168.1.1") && hp.getPort() == 10005)); } finally { cluster.stop(); } @@ -228,32 +228,27 @@ private void restartClusterWithProperties(Map properties) throws cluster.start(); } - private Map getAdvertiseAddressFromZooKeeper() - throws InterruptedException { - Map addresses = new HashMap<>(); + private Map getAdvertiseAddressFromZooKeeper() throws InterruptedException { + Map addresses = new HashMap<>(); Set mgrs = getServerContext().instanceOperations().getServers(ServerId.Type.MANAGER); assertEquals(1, mgrs.size()); - addresses.put(ServerType.MANAGER, - HostAndPort.fromString(mgrs.iterator().next().toHostPortString())); + addresses.put(ServerType.MANAGER, mgrs.iterator().next()); Set tservers = getServerContext().instanceOperations().getServers(ServerId.Type.TABLET_SERVER); assertEquals(1, tservers.size()); - addresses.put(ServerType.TABLET_SERVER, - HostAndPort.fromString(tservers.iterator().next().toHostPortString())); + addresses.put(ServerType.TABLET_SERVER, tservers.iterator().next()); Set compactors = getServerContext().instanceOperations().getServers(ServerId.Type.COMPACTOR); assertEquals(1, compactors.size()); - addresses.put(ServerType.COMPACTOR, - HostAndPort.fromString(compactors.iterator().next().toHostPortString())); + addresses.put(ServerType.COMPACTOR, compactors.iterator().next()); Set sservers = getServerContext().instanceOperations().getServers(ServerId.Type.SCAN_SERVER); assertEquals(1, sservers.size()); - addresses.put(ServerType.SCAN_SERVER, - HostAndPort.fromString(sservers.iterator().next().toHostPortString())); + addresses.put(ServerType.SCAN_SERVER, sservers.iterator().next()); return addresses; } diff --git a/test/src/main/java/org/apache/accumulo/test/functional/AmpleConditionalWriterIT.java b/test/src/main/java/org/apache/accumulo/test/functional/AmpleConditionalWriterIT.java index c051fea6f5c..fcdaa5bc883 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/AmpleConditionalWriterIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/AmpleConditionalWriterIT.java @@ -67,6 +67,7 @@ import org.apache.accumulo.core.client.admin.NewTableConfiguration; import org.apache.accumulo.core.client.admin.TabletAvailability; import org.apache.accumulo.core.client.admin.TimeType; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Range; @@ -162,8 +163,8 @@ public void setupTable() throws Exception { @Test public void testLocations() { - var ts1 = new TServerInstance("localhost:9997", 5000L); - var ts2 = new TServerInstance("localhost:9997", 6000L); + var ts1 = new TServerInstance(ServerIdUtil.tserver("localhost", 9997), 5000L); + var ts2 = new TServerInstance(ServerIdUtil.tserver("localhost", 9997), 6000L); var context = getCluster().getServerContext(); @@ -321,8 +322,8 @@ public void testLocations() { @Test public void testFiles() { - var ts1 = new TServerInstance("localhost:9997", 5000L); - var ts2 = new TServerInstance("localhost:9997", 6000L); + var ts1 = new TServerInstance(ServerIdUtil.tserver("localhost", 9997), 5000L); + var ts2 = new TServerInstance(ServerIdUtil.tserver("localhost", 9997), 6000L); var context = getCluster().getServerContext(); @@ -472,7 +473,7 @@ public void testWALs() { // Test adding a WAL to a tablet and verifying its presence String walFilePath = - java.nio.file.Path.of("tserver+8080", UUID.randomUUID().toString()).toString(); + java.nio.file.Path.of("default+tserver+8080", UUID.randomUUID().toString()).toString(); final LogEntry originalLogEntry = LogEntry.fromPath(walFilePath); // create a tablet metadata with no write ahead logs var tmEmptySet = TabletMetadata.builder(e1).build(LOGS); @@ -490,7 +491,7 @@ public void testWALs() { // Test adding another WAL and verifying the update String walFilePath2 = - java.nio.file.Path.of("tserver+8080", UUID.randomUUID().toString()).toString(); + java.nio.file.Path.of("default+tserver+8080", UUID.randomUUID().toString()).toString(); LogEntry newLogEntry = LogEntry.fromPath(walFilePath2); try (var ctmi = context.getAmple().conditionallyMutateTablets()) { ctmi.mutateTablet(e1).requireAbsentOperation().putWal(newLogEntry).submit(tm -> false); @@ -503,7 +504,7 @@ public void testWALs() { assertEquals(expectedLogs, actualLogs, "Both original and new LogEntry should be present."); String walFilePath3 = - java.nio.file.Path.of("tserver+8080", UUID.randomUUID().toString()).toString(); + java.nio.file.Path.of("default+tserver+8080", UUID.randomUUID().toString()).toString(); LogEntry otherLogEntry = LogEntry.fromPath(walFilePath3); // create a powerset to ensure all possible subsets fail when using requireSame except the @@ -769,8 +770,8 @@ public static String createSelectedFilesJson(FateId fateId, boolean selAll, @Test public void testMultipleExtents() { - var ts1 = new TServerInstance("localhost:9997", 5000L); - var ts2 = new TServerInstance("localhost:9997", 6000L); + var ts1 = new TServerInstance(ServerIdUtil.tserver("localhost", 9997), 5000L); + var ts2 = new TServerInstance(ServerIdUtil.tserver("localhost", 9997), 6000L); var context = getCluster().getServerContext(); @@ -1233,7 +1234,7 @@ public void multipleFilters() { testFilterApplied(context, Set.of(), Set.of(e1, e2, e3, e4), "Initially, all tablets should be present"); - String server = "server1+8555"; + String server = "default+server1+8555"; String walFilePath = java.nio.file.Path.of(server, UUID.randomUUID().toString()).toString(); LogEntry wal = LogEntry.fromPath(walFilePath); @@ -1254,7 +1255,8 @@ public void multipleFilters() { testFilterApplied(context, Set.of(new TestTabletMetadataFilter(), new GcWalsFilter(Set.of())), tabletsWithWalCompactFlush, "Combination of filters did not return the expected tablets"); - TServerInstance serverInstance = new TServerInstance(server, 1L); + TServerInstance serverInstance = + new TServerInstance(ServerIdUtil.tserver("server1", 8555), 1L); // on a subset of the tablets, put a location final Set tabletsWithLocation = Set.of(e2, e3, e4); @@ -1339,7 +1341,7 @@ public void walFilter() { // add a wal to e2 var walFilePath = - java.nio.file.Path.of("tserver+8080", UUID.randomUUID().toString()).toString(); + java.nio.file.Path.of("default+tserver+8080", UUID.randomUUID().toString()).toString(); var wal = LogEntry.fromPath(walFilePath); try (var ctmi = context.getAmple().conditionallyMutateTablets()) { ctmi.mutateTablet(e2).requireAbsentOperation().putWal(wal).submit(tabletMetadata -> false); @@ -1350,7 +1352,8 @@ public void walFilter() { testFilterApplied(context, filter, Set.of(e2), "Only tablets with wals should be returned"); // add wal to tablet e4 - walFilePath = java.nio.file.Path.of("tserver+8080", UUID.randomUUID().toString()).toString(); + walFilePath = + java.nio.file.Path.of("default+tserver+8080", UUID.randomUUID().toString()).toString(); wal = LogEntry.fromPath(walFilePath); try (var ctmi = context.getAmple().conditionallyMutateTablets()) { ctmi.mutateTablet(e4).requireAbsentOperation().putWal(wal).submit(tabletMetadata -> false); @@ -1371,8 +1374,8 @@ public void walFilter() { // test that now only the tablet with a wal is returned when using filter() testFilterApplied(context, filter, Set.of(e2), "Only tablets with wals should be returned"); - var ts1 = new TServerInstance("localhost:9997", 5000L); - var ts2 = new TServerInstance("localhost:9997", 6000L); + var ts1 = new TServerInstance(ServerIdUtil.tserver("localhost", 9997), 5000L); + var ts2 = new TServerInstance(ServerIdUtil.tserver("localhost", 9997), 6000L); try (var ctmi = context.getAmple().conditionallyMutateTablets()) { ctmi.mutateTablet(e1).requireAbsentOperation().requireAbsentLocation() @@ -1909,8 +1912,8 @@ public void testMetadataCheck() { @Test public void testRequireMigration() { var context = getCluster().getServerContext(); - var tsi = new TServerInstance("localhost:1234", 56L); - var otherTsi = new TServerInstance("localhost:9876", 54L); + var tsi = new TServerInstance(ServerIdUtil.tserver("localhost", 1234), 56L); + var otherTsi = new TServerInstance(ServerIdUtil.tserver("localhost", 9876), 54L); try (var ctmi = context.getAmple().conditionallyMutateTablets()) { ctmi.mutateTablet(e1).requireAbsentOperation().requireMigration(tsi).deleteMigration() diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java index 20187c78d18..5a807f36907 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java @@ -118,7 +118,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.collect.Iterators; -import com.google.common.net.HostAndPort; public class CompactionIT extends CompactionITBase { @@ -946,7 +945,7 @@ private void testCancelUserCompactionTimeout(boolean timeout) throws Exception { writeRows((ClientContext) client, table2, MAX_DATA, true); var ctx = getCluster().getServerContext(); - Optional coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(ctx); + Optional coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(ctx); if (coordinatorHost.isEmpty()) { throw new TTransportException("Unable to get CompactionCoordinator address from ZooKeeper"); } diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java index e4d93597b83..8bb28eedb53 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java @@ -44,6 +44,7 @@ import org.apache.accumulo.core.client.AccumuloClient; import org.apache.accumulo.core.client.BatchWriter; import org.apache.accumulo.core.client.Scanner; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; @@ -84,7 +85,6 @@ import org.slf4j.LoggerFactory; import com.google.common.collect.Iterators; -import com.google.common.net.HostAndPort; public class GarbageCollectorIT extends ConfigurableMacBase { private static final String OUR_SECRET = "itsreallysecret"; @@ -428,7 +428,7 @@ public void testProperPortAdvertisement() throws Exception { Optional sld = ServiceLockData.parse(zk.getData(lockPath)); assertNotNull(sld.orElseThrow()); - HostAndPort hostAndPort = sld.orElseThrow().getAddress(ThriftService.GC); + ServerId hostAndPort = sld.orElseThrow().getServer(ThriftService.GC); // We shouldn't have the "bindall" address in zk assertNotEquals(ConfigOpts.BIND_ALL_ADDRESSES, hostAndPort.getHost()); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GracefulShutdownIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GracefulShutdownIT.java index c1bad36ae49..6f04d7f0c34 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/GracefulShutdownIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/GracefulShutdownIT.java @@ -38,6 +38,7 @@ import org.apache.accumulo.core.client.admin.CompactionConfig; import org.apache.accumulo.core.client.admin.NewTableConfiguration; import org.apache.accumulo.core.client.admin.servers.ServerId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.conf.ClientProperty; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Key; @@ -166,7 +167,7 @@ public void testGracefulShutdown() throws Exception { getCluster().getServerContext().getServerPaths().getGarbageCollector(true); Optional data = ServiceLock.getLockData(ctx.getZooSession(), gcLockPath); assertTrue(data.isPresent()); - final HostAndPort gcAddress = data.orElseThrow().getAddress(ThriftService.GC); + final ServerId gcAddress = data.orElseThrow().getServer(ThriftService.GC); assertTrue(!control.getProcesses(ServerType.GARBAGE_COLLECTOR).isEmpty()); // Don't call `new Admin().execute(new String[] {"signalShutdown", "-h ", host, "-p ", // Integer.toString(port)})` @@ -181,9 +182,9 @@ public void testGracefulShutdown() throws Exception { final Set tservers = getCluster().getServerContext().getServerPaths() .getTabletServer((rg) -> rg.equals(ResourceGroupId.DEFAULT), AddressSelector.all(), true); assertEquals(2, tservers.size()); - final HostAndPort tserverAddress = - HostAndPort.fromString(tservers.iterator().next().getServer()); - Admin.signalGracefulShutdown(ctx, tserverAddress); + final ServiceLockPath tserverPath = tservers.iterator().next(); + final var hp = HostAndPort.fromString(tserverPath.getServer()); + Admin.signalGracefulShutdown(ctx, ServerIdUtil.tserver(hp.getHost(), hp.getPort())); Wait.waitFor(() -> { control.refreshProcesses(ServerType.TABLET_SERVER); return control.getProcesses(ServerType.TABLET_SERVER).size() == 1; @@ -201,9 +202,7 @@ public void testGracefulShutdown() throws Exception { client.instanceOperations().getServers(ServerId.Type.MANAGER); assertNotNull(managerLocations); assertEquals(1, managerLocations.size()); - final HostAndPort managerAddress = - HostAndPort.fromString(managerLocations.iterator().next().toHostPortString()); - Admin.signalGracefulShutdown(ctx, managerAddress); + Admin.signalGracefulShutdown(ctx, managerLocations.iterator().next()); Wait.waitFor(() -> { control.refreshProcesses(ServerType.MANAGER); return control.getProcesses(ServerType.MANAGER).isEmpty(); @@ -224,8 +223,10 @@ public void testGracefulShutdown() throws Exception { final Set compactors = getCluster().getServerContext().getServerPaths() .getCompactor(ResourceGroupPredicate.exact(ResourceGroupId.of(GROUP_NAME)), AddressSelector.all(), true); - final HostAndPort compactorAddress = - HostAndPort.fromString(compactors.iterator().next().getServer()); + final ServiceLockPath compactorPath = compactors.iterator().next(); + final var compactorHP = HostAndPort.fromString(compactorPath.getServer()); + final ServerId compactorAddress = ServerIdUtil.compactor(compactorPath.getResourceGroup(), + compactorHP.getHost(), compactorHP.getPort()); final CompactionConfig cc = new CompactionConfig(); final IteratorSetting is = new IteratorSetting(100, SlowIterator.class); @@ -240,8 +241,7 @@ public void testGracefulShutdown() throws Exception { client.instanceOperations().getServers(ServerId.Type.MANAGER); assertNotNull(newManagerLocations); assertEquals(1, newManagerLocations.size()); - final HostAndPort newManagerAddress = - HostAndPort.fromString(newManagerLocations.iterator().next().toHostPortString()); + final ServerId newManagerAddress = newManagerLocations.iterator().next(); assertEquals(0, ExternalCompactionTestUtils .getRunningCompactions(ctx, Optional.of(newManagerAddress)).getCompactionsSize()); client.tableOperations().compact(tableName, cc); @@ -267,7 +267,8 @@ public void testGracefulShutdown() throws Exception { final Set sservers = getCluster().getServerContext().getServerPaths().getScanServer( (rg) -> rg.equals(ResourceGroupId.of(GROUP_NAME)), AddressSelector.all(), true); - final HostAndPort sserver = HostAndPort.fromString(sservers.iterator().next().getServer()); + final ServiceLockPath sserverPath = sservers.iterator().next(); + final var sserverHP = HostAndPort.fromString(sserverPath.getServer()); try (final Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) { scanner.setRange(new Range()); scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL); @@ -279,7 +280,8 @@ public void testGracefulShutdown() throws Exception { assertNotNull(e); count++; if (count == 2) { - Admin.signalGracefulShutdown(ctx, sserver); + Admin.signalGracefulShutdown(ctx, ServerIdUtil.sserver(sserverPath.getResourceGroup(), + sserverHP.getHost(), sserverHP.getPort())); } } assertEquals(10, count); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/LastLocationIT.java b/test/src/main/java/org/apache/accumulo/test/functional/LastLocationIT.java index 27aab35177b..f161f015b87 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/LastLocationIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/LastLocationIT.java @@ -59,7 +59,10 @@ public void test() throws Exception { newTablet = ManagerAssignmentIT.getTabletMetadata(c, tableId, null); } while (!newTablet.hasCurrent()); // this would be null if the mode was not "assign" - assertEquals(newTablet.getLocation().getHostPort(), newTablet.getLast().getHostPort()); + assertEquals(newTablet.getLocation().getServerInstance().getServer().getHost(), + newTablet.getLast().getServerInstance().getServer().getHost()); + assertEquals(newTablet.getLocation().getServerInstance().getServer().getPort(), + newTablet.getLast().getServerInstance().getServer().getPort()); // put something in it try (BatchWriter bw = c.createBatchWriter(tableName)) { @@ -70,8 +73,14 @@ public void test() throws Exception { // last location should not be set yet TabletMetadata unflushed = ManagerAssignmentIT.getTabletMetadata(c, tableId, null); - assertEquals(newTablet.getLocation().getHostPort(), unflushed.getLocation().getHostPort()); - assertEquals(newTablet.getLocation().getHostPort(), unflushed.getLast().getHostPort()); + assertEquals(newTablet.getLocation().getServerInstance().getServer().getHost(), + unflushed.getLocation().getServerInstance().getServer().getHost()); + assertEquals(newTablet.getLocation().getServerInstance().getServer().getPort(), + unflushed.getLocation().getServerInstance().getServer().getPort()); + assertEquals(newTablet.getLocation().getServerInstance().getServer().getHost(), + unflushed.getLast().getServerInstance().getServer().getHost()); + assertEquals(newTablet.getLocation().getServerInstance().getServer().getPort(), + unflushed.getLast().getServerInstance().getServer().getPort()); assertTrue(newTablet.hasCurrent()); // take the tablet offline @@ -79,14 +88,20 @@ public void test() throws Exception { TabletMetadata offline = ManagerAssignmentIT.getTabletMetadata(c, tableId, null); assertNull(offline.getLocation()); assertFalse(offline.hasCurrent()); - assertEquals(newTablet.getLocation().getHostPort(), offline.getLast().getHostPort()); + assertEquals(newTablet.getLocation().getServerInstance().getServer().getHost(), + offline.getLast().getServerInstance().getServer().getHost()); + assertEquals(newTablet.getLocation().getServerInstance().getServer().getPort(), + offline.getLast().getServerInstance().getServer().getPort()); // 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(newTablet.getLast().getHostPort(), online.getLast().getHostPort()); + assertEquals(newTablet.getLast().getServerInstance().getServer().getHost(), + online.getLast().getServerInstance().getServer().getHost()); + assertEquals(newTablet.getLast().getServerInstance().getServer().getPort(), + online.getLast().getServerInstance().getServer().getPort()); } } diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ManagerApiIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ManagerApiIT.java index 9ea4778c490..6c6a895c656 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/ManagerApiIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/ManagerApiIT.java @@ -35,12 +35,14 @@ import org.apache.accumulo.core.client.security.tokens.PasswordToken; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.clientImpl.Credentials; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.lock.ServiceLockPaths.ResourceGroupPredicate; import org.apache.accumulo.core.manager.thrift.ManagerClientService; import org.apache.accumulo.core.manager.thrift.ManagerGoalState; +import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; import org.apache.accumulo.core.security.SystemPermission; import org.apache.accumulo.core.security.TablePermission; @@ -246,10 +248,11 @@ public void testPermissions_setSystemProperty() throws Exception { public void testPermissions_shutdownTabletServer() throws Exception { // To shutdownTabletServer, user needs SystemPermission.SYSTEM // this server won't exist, so shutting it down is a NOOP on success - String fakeHostAndPort = getUniqueNames(1)[0] + ":0"; + var inst = new TServerInstance(ServerIdUtil.tserver(getUniqueNames(1)[0], 0), "1234"); + op = user -> client -> { - client.shutdownTabletServer(TraceUtil.traceInfo(), user.toThrift(instanceId), fakeHostAndPort, - false); + client.shutdownTabletServer(TraceUtil.traceInfo(), user.toThrift(instanceId), + inst.toZooKeeperPathString(), false); return null; }; expectPermissionDenied(op, regularUser); @@ -260,8 +263,9 @@ public void testPermissions_shutdownTabletServer() throws Exception { @Test public void shutdownTabletServer() throws Exception { op = user -> client -> { + var inst = new TServerInstance(ServerIdUtil.tserver("fakeTabletServer", 9997), "1234"); client.shutdownTabletServer(TraceUtil.traceInfo(), user.toThrift(instanceId), - "fakeTabletServer:9997", true); + inst.toZooKeeperPathString(), true); return null; }; try (AccumuloClient client = Accumulo.newClient().from(getClientProps()) diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java index dbeb1a99081..2fa76e3d4f6 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java @@ -67,6 +67,7 @@ import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.lock.ServiceLockPaths.ResourceGroupPredicate; import org.apache.accumulo.core.metadata.SystemTables; +import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.core.metadata.schema.Ample; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection; import org.apache.accumulo.core.metadata.schema.TabletMetadata; @@ -156,7 +157,10 @@ public void test() throws Exception { TabletMetadata flushed = getTabletMetadata(client, tableId, null); assertTrue(flushed.hasCurrent()); assertNotNull(flushed.getLocation()); - assertEquals(flushed.getLocation().getHostPort(), flushed.getLast().getHostPort()); + assertEquals(flushed.getLocation().getServerInstance().getServer().getHost(), + flushed.getLast().getServerInstance().getServer().getHost()); + assertEquals(flushed.getLocation().getServerInstance().getServer().getPort(), + flushed.getLast().getServerInstance().getServer().getPort()); assertFalse(flushed.getLocation().getType().equals(LocationType.FUTURE)); assertEquals(TabletAvailability.ONDEMAND, flushed.getTabletAvailability()); @@ -165,7 +169,10 @@ public void test() throws Exception { TabletMetadata offline = getTabletMetadata(client, tableId, null); assertFalse(offline.hasCurrent()); assertNull(offline.getLocation()); - assertEquals(flushed.getLocation().getHostPort(), offline.getLast().getHostPort()); + assertEquals(flushed.getLocation().getServerInstance().getServer().getHost(), + offline.getLast().getServerInstance().getServer().getHost()); + assertEquals(flushed.getLocation().getServerInstance().getServer().getPort(), + offline.getLast().getServerInstance().getServer().getPort()); assertEquals(TabletAvailability.ONDEMAND, offline.getTabletAvailability()); // put it back online @@ -173,7 +180,10 @@ public void test() throws Exception { TabletMetadata online = getTabletMetadata(client, tableId, null); assertTrue(online.hasCurrent()); assertNotNull(online.getLocation()); - assertEquals(online.getLocation().getHostPort(), online.getLast().getHostPort()); + assertEquals(online.getLocation().getServerInstance().getServer().getHost(), + online.getLast().getServerInstance().getServer().getHost()); + assertEquals(online.getLocation().getServerInstance().getServer().getPort(), + online.getLast().getServerInstance().getServer().getPort()); assertEquals(TabletAvailability.ONDEMAND, online.getTabletAvailability()); // set the tablet availability to HOSTED @@ -189,7 +199,10 @@ public void test() throws Exception { final TabletMetadata always = getTabletMetadata(client, tableId, null); assertTrue(hostedOrCurrentNotNull.test(always)); assertTrue(always.hasCurrent()); - assertEquals(flushed.getLocation().getHostPort(), always.getLast().getHostPort()); + assertEquals(flushed.getLocation().getServerInstance().getServer().getHost(), + always.getLast().getServerInstance().getServer().getHost()); + assertEquals(flushed.getLocation().getServerInstance().getServer().getPort(), + always.getLast().getServerInstance().getServer().getPort()); assertEquals(TabletAvailability.HOSTED, always.getTabletAvailability()); // set the hosting availability to never @@ -203,7 +216,10 @@ public void test() throws Exception { final TabletMetadata unhosted = getTabletMetadata(client, tableId, null); assertTrue(unhostedOrCurrentNull.test(unhosted)); assertNull(unhosted.getLocation()); - assertEquals(flushed.getLocation().getHostPort(), unhosted.getLast().getHostPort()); + assertEquals(flushed.getLocation().getServerInstance().getServer().getHost(), + unhosted.getLast().getServerInstance().getServer().getHost()); + assertEquals(flushed.getLocation().getServerInstance().getServer().getPort(), + unhosted.getLast().getServerInstance().getServer().getPort()); assertEquals(TabletAvailability.UNHOSTED, unhosted.getTabletAvailability()); // set the tablet availability to ONDEMAND @@ -215,7 +231,10 @@ public void test() throws Exception { final TabletMetadata ondemand = getTabletMetadata(client, tableId, null); assertTrue(ondemandHosted.test(ondemand)); assertNull(ondemand.getLocation()); - assertEquals(flushed.getLocation().getHostPort(), ondemand.getLast().getHostPort()); + assertEquals(flushed.getLocation().getServerInstance().getServer().getHost(), + ondemand.getLast().getServerInstance().getServer().getHost()); + assertEquals(flushed.getLocation().getServerInstance().getServer().getPort(), + ondemand.getLast().getServerInstance().getServer().getPort()); assertEquals(TabletAvailability.ONDEMAND, ondemand.getTabletAvailability()); } @@ -517,26 +536,21 @@ public void testShutdownOnlyTServerWithUserTable() throws Exception { // getClusterControl().stopAllServers(ServerType.TABLET_SERVER) // could potentially send a kill -9 to the process. Shut the tablet // servers down in a more graceful way. - final Map>> binnedRanges = new HashMap<>(); + final Map>> binnedRanges = new HashMap<>(); ((ClientContext) client).getTabletLocationCache(tid).binRanges((ClientContext) client, Collections.singletonList(TabletsSection.getRange()), binnedRanges); binnedRanges.keySet().forEach((location) -> { - HostAndPort address = HostAndPort.fromString(location); - String addressWithSession = address.toString(); - var zLockPath = getCluster().getServerContext().getServerPaths() - .createTabletServerPath(ResourceGroupId.DEFAULT, address); + var zLockPath = getCluster().getServerContext().getServerPaths().createTabletServerPath( + ResourceGroupId.DEFAULT, HostAndPort.fromParts(location.getHost(), location.getPort())); long sessionId = ServiceLock.getSessionId(getCluster().getServerContext().getZooCache(), zLockPath); - if (sessionId != 0) { - addressWithSession = address + "[" + Long.toHexString(sessionId) + "]"; - } + final TServerInstance inst = new TServerInstance(location, sessionId); - final String finalAddress = addressWithSession; - System.out.println("Attempting to shutdown TabletServer at: " + address); + System.out.println("Attempting to shutdown TabletServer at: " + inst); try { ThriftClientTypes.MANAGER.executeVoid((ClientContext) client, c -> c.shutdownTabletServer(TraceUtil.traceInfo(), - getCluster().getServerContext().rpcCreds(), finalAddress, false), + getCluster().getServerContext().rpcCreds(), inst.toZooKeeperPathString(), false), ResourceGroupPredicate.DEFAULT_RG_ONLY); } catch (AccumuloException | AccumuloSecurityException e) { fail("Error shutting down TabletServer", e); @@ -573,22 +587,17 @@ public void testShutdownOnlyTServerWithoutUserTable() throws Exception { Locations locs = client.tableOperations().locate(SystemTables.ROOT.tableName(), Collections.singletonList(TabletsSection.getRange())); locs.groupByTablet().keySet().stream().map(locs::getTabletLocation).forEach(location -> { - HostAndPort address = HostAndPort.fromString(location); - String addressWithSession = address.toString(); - var zLockPath = getCluster().getServerContext().getServerPaths() - .createTabletServerPath(ResourceGroupId.DEFAULT, address); + var zLockPath = getCluster().getServerContext().getServerPaths().createTabletServerPath( + ResourceGroupId.DEFAULT, HostAndPort.fromParts(location.getHost(), location.getPort())); long sessionId = ServiceLock.getSessionId(getCluster().getServerContext().getZooCache(), zLockPath); - if (sessionId != 0) { - addressWithSession = address + "[" + Long.toHexString(sessionId) + "]"; - } + final TServerInstance inst = new TServerInstance(location, sessionId); - final String finalAddress = addressWithSession; - System.out.println("Attempting to shutdown TabletServer at: " + address); + System.out.println("Attempting to shutdown TabletServer at: " + inst); try { ThriftClientTypes.MANAGER.executeVoid((ClientContext) client, c -> c.shutdownTabletServer(TraceUtil.traceInfo(), - getCluster().getServerContext().rpcCreds(), finalAddress, false), + getCluster().getServerContext().rpcCreds(), inst.toZooKeeperPathString(), false), ResourceGroupPredicate.DEFAULT_RG_ONLY); } catch (AccumuloException | AccumuloSecurityException e) { fail("Error shutting down TabletServer", e); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMajCIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMajCIT.java index 8c68a2597b2..d67497a538c 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMajCIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMajCIT.java @@ -57,8 +57,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.net.HostAndPort; - public class MemoryStarvedMajCIT extends SharedMiniClusterBase { private static final Logger LOG = LoggerFactory.getLogger(MemoryStarvedMajCIT.class); @@ -145,7 +143,6 @@ public void testMajCPauses() throws Exception { == 1, 60_000); ServerId csi = ctx.instanceOperations().getServers(ServerId.Type.COMPACTOR).iterator().next(); - HostAndPort compactorAddr = HostAndPort.fromParts(csi.getHost(), csi.getPort()); TableOperations to = client.tableOperations(); to.create(table); @@ -164,11 +161,11 @@ public void testMajCPauses() throws Exception { // Calling getRunningCompaction on the MemoryConsumingCompactor // will consume the free memory - LOG.info("Calling getRunningCompaction on {}", compactorAddr); + LOG.info("Calling getRunningCompaction on {}", csi); boolean success = false; while (!success) { try { - ExternalCompactionUtil.getRunningCompaction(compactorAddr, ctx); + ExternalCompactionUtil.getRunningCompaction(csi, ctx); success = true; } catch (Exception e) { UtilWaitThread.sleep(3000); @@ -182,8 +179,8 @@ public void testMajCPauses() throws Exception { // Calling cancel on the MemoryConsumingCompactor will free // the consumed memory - LOG.info("Calling cancel on {}", compactorAddr); - ExternalCompactionUtil.cancelCompaction(ctx, compactorAddr, "fakeECID"); + LOG.info("Calling cancel on {}", csi); + ExternalCompactionUtil.cancelCompaction(ctx, csi, "fakeECID"); compactionThread.interrupt(); compactionThread.join(); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RegexGroupBalanceIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RegexGroupBalanceIT.java index e87450211ac..8ab46f98833 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/RegexGroupBalanceIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/RegexGroupBalanceIT.java @@ -196,8 +196,8 @@ private Table getCounts(AccumuloClient client, String } else { group = group.substring(tableId.canonical().length() + 1).substring(0, 2); } - String loc = new TServerInstance(entry.getValue(), entry.getKey().getColumnQualifier()) - .getHostPortSession(); + String loc = + TServerInstance.deserialize(entry.getValue().toString()).toZooKeeperPathString(); MutableInt count = groupLocationCounts.get(group, loc); if (count == null) { diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java index c362176e688..0172747e55f 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java @@ -39,6 +39,7 @@ import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.client.admin.TimeType; import org.apache.accumulo.core.clientImpl.ScannerImpl; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.data.Value; @@ -72,6 +73,8 @@ import org.apache.hadoop.io.Text; import org.junit.jupiter.api.Test; +import com.google.common.net.HostAndPort; + public class SplitRecoveryIT extends ConfigurableMacBase { public static Map updateTabletDataFile(FateId fateId, @@ -208,7 +211,9 @@ private void splitPartiallyAndRecover(ServerContext context, KeyExtent extent, K highDatafileSizes, highDatafilesToRemove); SplitRecovery11to12.splitTablet(high, extent.prevEndRow(), splitRatio, context, Set.of()); - TServerInstance instance = new TServerInstance(location, zl.getSessionId()); + final var hp = HostAndPort.fromString(location); + TServerInstance instance = + new TServerInstance(ServerIdUtil.tserver(hp.getHost(), hp.getPort()), zl.getSessionId()); Assignment assignment = new Assignment(high, instance, null); TabletMutator tabletMutator = context.getAmple().mutateTablet(extent); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java index 1042e7b5fea..fbb13437c7c 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java @@ -59,6 +59,7 @@ import org.apache.accumulo.core.client.admin.NewTableConfiguration; import org.apache.accumulo.core.client.admin.TabletAvailability; import org.apache.accumulo.core.clientImpl.ClientContext; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Key; @@ -114,6 +115,7 @@ import org.slf4j.LoggerFactory; import com.google.common.collect.Sets; +import com.google.common.net.HostAndPort; /** * Test to ensure that the {@link TabletManagementIterator} properly skips over tablet information @@ -369,7 +371,8 @@ public void test(String compressionType) throws Exception { try (TabletsMutatorImpl mut = new TabletsMutatorImpl(getServerContext(), (dl) -> metaCopy6)) { KeyExtent nonExistantTable = new KeyExtent(badTableId, null, null); TabletMutator tm = mut.mutateTablet(nonExistantTable); - tm.putLocation(Location.current("fakeServer", "fakeSession")); + tm.putLocation(Location + .current(new TServerInstance(ServerIdUtil.tserver("fakeServer", 0), "fakeSession"))); tm.automaticallyPutServerLock(false); tm.mutate(); } @@ -389,7 +392,8 @@ public void test(String compressionType) throws Exception { try (TabletsMutatorImpl mut = new TabletsMutatorImpl(getServerContext(), (dl) -> metaCopy6)) { KeyExtent nonExistantTable = new KeyExtent(badTableId, null, null); TabletMutator tm = mut.mutateTablet(nonExistantTable); - tm.deleteLocation(Location.current("fakeServer", "fakeSession")); + tm.deleteLocation(Location + .current(new TServerInstance(ServerIdUtil.tserver("fakeServer", 0), "fakeSession"))); tm.automaticallyPutServerLock(false); tm.mutate(); } @@ -456,7 +460,8 @@ private void addDuplicateLocation(AccumuloClient client, String table, String ta TableId tableIdToModify = TableId.of(client.tableOperations().tableIdMap().get(tableNameToModify)); Mutation m = new Mutation(new KeyExtent(tableIdToModify, null, null).toMetaRow()); - m.put(CurrentLocationColumnFamily.NAME, new Text("1234567"), new Value("fake:9005")); + m.put(CurrentLocationColumnFamily.NAME, new Text("1234567"), + new Value(new TServerInstance(ServerIdUtil.tserver("fake", 9005), "1234567").serialize())); try (BatchWriter bw = client.createBatchWriter(table)) { bw.addMutation(m); } @@ -488,7 +493,8 @@ private void reassignLocation(AccumuloClient client, String table, String tableN m.putDelete(entry.getKey().getColumnFamily(), entry.getKey().getColumnQualifier(), entry.getKey().getTimestamp()); m.put(entry.getKey().getColumnFamily(), new Text("1234567"), - entry.getKey().getTimestamp() + 1, new Value("fake:9005")); + entry.getKey().getTimestamp() + 1, new Value( + new TServerInstance(ServerIdUtil.tserver("fake", 9005), "1234567").serialize())); try (BatchWriter bw = client.createBatchWriter(table)) { bw.addMutation(m); } @@ -672,7 +678,7 @@ private void createLogEntry(AccumuloClient client, String table, String tableNam TableId.of(client.tableOperations().tableIdMap().get(tableNameToModify)); KeyExtent extent = new KeyExtent(tableIdToModify, new Text("some split"), null); Mutation m = new Mutation(extent.toMetaRow()); - String fileName = "file:/accumulo/wal/localhost+9997/" + UUID.randomUUID().toString(); + String fileName = "file:/accumulo/wal/default+localhost+9997/" + UUID.randomUUID().toString(); LogEntry logEntry = LogEntry.fromPath(fileName); logEntry.addToMutation(m); try (BatchWriter bw = client.createBatchWriter(table)) { @@ -693,9 +699,12 @@ private static TabletManagementParameters createParameters(AccumuloClient client HashSet tservers = new HashSet<>(); for (ServiceLockPath tserver : context.getServerPaths() .getTabletServer(ResourceGroupPredicate.ANY, AddressSelector.all(), true)) { + final var hp = HostAndPort.fromString(tserver.getServer()); try { long sessionId = ServiceLock.getSessionId(context.getZooCache(), tserver); - tservers.add(new TServerInstance(tserver.getServer(), sessionId)); + tservers.add(new TServerInstance( + ServerIdUtil.tserver(tserver.getResourceGroup(), hp.getHost(), hp.getPort()), + sessionId)); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TabletResourceGroupBalanceIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TabletResourceGroupBalanceIT.java index 0f44b423355..940b162f1cd 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/TabletResourceGroupBalanceIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/TabletResourceGroupBalanceIT.java @@ -152,7 +152,8 @@ public void testBalancerWithResourceGroups() throws Exception { assertEquals(26, locations.size()); Location l1 = locations.get(0).getLocation(); - assertEquals(ResourceGroupId.DEFAULT, tserverGroups.get(l1.getHostAndPort().toString())); + assertEquals(ResourceGroupId.DEFAULT, + tserverGroups.get(l1.getServerInstance().getServer().toHostPortString())); locations.forEach(loc -> assertEquals(l1, loc.getLocation())); // Check table names[1] @@ -161,7 +162,8 @@ public void testBalancerWithResourceGroups() throws Exception { assertEquals(26, locations.size()); Location l2 = locations.get(0).getLocation(); - assertEquals(ResourceGroupId.of("GROUP1"), tserverGroups.get(l2.getHostAndPort().toString())); + assertEquals(ResourceGroupId.of("GROUP1"), + tserverGroups.get(l2.getServerInstance().getServer().toHostPortString())); locations.forEach(loc -> assertEquals(l2, loc.getLocation())); client.tableOperations().delete(names[0]); @@ -293,7 +295,8 @@ public void testResourceGroupPropertyChange(AccumuloClient client, String tableN List locations = getLocations(ample, tableId); assertEquals(numExpectedSplits, locations.size()); Location l1 = locations.get(0).getLocation(); - assertEquals(ResourceGroupId.DEFAULT, tserverGroups.get(l1.getHostAndPort().toString())); + assertEquals(ResourceGroupId.DEFAULT, + tserverGroups.get(l1.getServerInstance().getServer().toHostPortString())); locations.forEach(loc -> assertEquals(l1, loc.getLocation())); // change the resource group property for the table @@ -304,13 +307,13 @@ public void testResourceGroupPropertyChange(AccumuloClient client, String tableN while ((locations == null || locations.isEmpty() || locations.size() != numExpectedSplits || locations.get(0).getLocation() == null || locations.get(0).getLocation().getType() == LocationType.FUTURE) - || (locations.get(0).getLocation().getType() == LocationType.CURRENT - && !tserverGroups.get(locations.get(0).getLocation().getHostAndPort().toString()) - .equals(ResourceGroupId.of("GROUP1")))) { + || (locations.get(0).getLocation().getType() == LocationType.CURRENT && !tserverGroups + .get(locations.get(0).getLocation().getServerInstance().getServer().toHostPortString()) + .equals(ResourceGroupId.of("GROUP1")))) { locations = getLocations(ample, tableId); } Location group1Location = locations.get(0).getLocation(); - assertTrue(tserverGroups.get(group1Location.getHostAndPort().toString()) + assertTrue(tserverGroups.get(group1Location.getServerInstance().getServer().toHostPortString()) .equals(ResourceGroupId.of("GROUP1"))); client.instanceOperations().waitForBalance(); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java index 3437310e348..923f6ded35a 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java @@ -25,6 +25,8 @@ import java.util.UUID; import org.apache.accumulo.core.cli.ConfigOpts; +import org.apache.accumulo.core.client.admin.servers.ServerId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.clientImpl.thrift.ClientService; import org.apache.accumulo.core.clientImpl.thrift.TInfo; import org.apache.accumulo.core.conf.Property; @@ -131,8 +133,6 @@ public static void main(String[] args) throws Exception { HostAndPort.fromParts(ConfigOpts.BIND_ALL_ADDRESSES, port)); serverPort.startThriftServer("walking dead"); - String addressString = serverPort.address.toString(); - var zLockPath = context.getServerPaths().createTabletServerPath(ResourceGroupId.DEFAULT, serverPort.address); ZooReaderWriter zoo = context.getZooSession().asReaderWriter(); @@ -140,9 +140,11 @@ public static void main(String[] args) throws Exception { ServiceLock zlock = new ServiceLock(context.getZooSession(), zLockPath, UUID.randomUUID()); + ServerId serverId = + ServerIdUtil.tserver(serverPort.address.getHost(), serverPort.address.getPort()); + MetricsInfo metricsInfo = context.getMetricsInfo(); - metricsInfo.init(MetricsInfo.serviceTags(context.getInstanceName(), "zombie.server", - serverPort.address, ResourceGroupId.DEFAULT)); + metricsInfo.init(MetricsInfo.serviceTags(context.getInstanceName(), "zombie.server", serverId)); LockWatcher lw = new LockWatcher() { @@ -171,8 +173,7 @@ public void unableToMonitorLockNode(Exception e) { } }; - if (zlock.tryLock(lw, new ServiceLockData(UUID.randomUUID(), addressString, ThriftService.TSERV, - ResourceGroupId.DEFAULT))) { + if (zlock.tryLock(lw, new ServiceLockData(UUID.randomUUID(), serverId, ThriftService.TSERV))) { log.debug("Obtained tablet server lock {}", zlock.getLockPath()); } // modify metadata diff --git a/test/src/main/java/org/apache/accumulo/test/lock/ServiceLockIT.java b/test/src/main/java/org/apache/accumulo/test/lock/ServiceLockIT.java index 22f9ec72f14..3f37075e57a 100644 --- a/test/src/main/java/org/apache/accumulo/test/lock/ServiceLockIT.java +++ b/test/src/main/java/org/apache/accumulo/test/lock/ServiceLockIT.java @@ -38,7 +38,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.LockSupport; -import org.apache.accumulo.core.data.ResourceGroupId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.lock.ServiceLock.AccumuloLockWatcher; import org.apache.accumulo.core.lock.ServiceLock.LockLossReason; @@ -216,8 +216,8 @@ public void testDeleteParent() throws Exception { TestALW lw = new TestALW(); - zl.lock(lw, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV, - ResourceGroupId.DEFAULT)); + zl.lock(lw, new ServiceLockData(UUID.randomUUID(), ServerIdUtil.tserver("test1", 0), + ThriftService.TSERV)); lw.waitForChanges(1); @@ -244,8 +244,8 @@ public void testNoParent() throws Exception { TestALW lw = new TestALW(); - zl.lock(lw, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV, - ResourceGroupId.DEFAULT)); + zl.lock(lw, new ServiceLockData(UUID.randomUUID(), ServerIdUtil.tserver("test1", 0), + ThriftService.TSERV)); lw.waitForChanges(1); @@ -272,8 +272,8 @@ public void testDeleteLock() throws Exception { TestALW lw = new TestALW(); - zl.lock(lw, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV, - ResourceGroupId.DEFAULT)); + zl.lock(lw, new ServiceLockData(UUID.randomUUID(), ServerIdUtil.tserver("test1", 0), + ThriftService.TSERV)); lw.waitForChanges(1); @@ -308,8 +308,8 @@ public void testDeleteWaiting() throws Exception { TestALW lw = new TestALW(); - zl.lock(lw, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV, - ResourceGroupId.DEFAULT)); + zl.lock(lw, new ServiceLockData(UUID.randomUUID(), ServerIdUtil.tserver("test1", 0), + ThriftService.TSERV)); lw.waitForChanges(1); @@ -323,8 +323,8 @@ public void testDeleteWaiting() throws Exception { TestALW lw2 = new TestALW(); - zl2.lock(lw2, new ServiceLockData(UUID.randomUUID(), "test2", ThriftService.TSERV, - ResourceGroupId.DEFAULT)); + zl2.lock(lw2, new ServiceLockData(UUID.randomUUID(), ServerIdUtil.tserver("test2", 0), + ThriftService.TSERV)); assertFalse(lw2.locked); assertFalse(zl2.isLocked()); @@ -334,8 +334,8 @@ public void testDeleteWaiting() throws Exception { TestALW lw3 = new TestALW(); - zl3.lock(lw3, new ServiceLockData(UUID.randomUUID(), "test3", ThriftService.TSERV, - ResourceGroupId.DEFAULT)); + zl3.lock(lw3, new ServiceLockData(UUID.randomUUID(), ServerIdUtil.tserver("test3", 0), + ThriftService.TSERV)); List children = ServiceLock.validateAndSort(parent, zrw.getChildren(parent.toString())); @@ -387,8 +387,8 @@ public void testUnexpectedEvent() throws Exception { TestALW lw = new TestALW(); - zl.lock(lw, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV, - ResourceGroupId.DEFAULT)); + zl.lock(lw, new ServiceLockData(UUID.randomUUID(), ServerIdUtil.tserver("test1", 0), + ThriftService.TSERV)); lw.waitForChanges(1); @@ -427,8 +427,8 @@ public void testLockSerial() throws Exception { final RetryLockWatcher zlw1 = new RetryLockWatcher(); ServiceLock zl1 = getZooLock(zk1, parent, UUID.fromString("00000000-0000-0000-0000-aaaaaaaaaaaa")); - zl1.lock(zlw1, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV, - ResourceGroupId.DEFAULT)); + zl1.lock(zlw1, new ServiceLockData(UUID.randomUUID(), ServerIdUtil.tserver("test1", 0), + ThriftService.TSERV)); // The call above creates two nodes in ZK because of the overridden create method in // ZooKeeperWrapper. // The nodes created are: @@ -443,8 +443,8 @@ public void testLockSerial() throws Exception { final RetryLockWatcher zlw2 = new RetryLockWatcher(); ServiceLock zl2 = getZooLock(zk2, parent, UUID.fromString("00000000-0000-0000-0000-bbbbbbbbbbbb")); - zl2.lock(zlw2, new ServiceLockData(UUID.randomUUID(), "test2", ThriftService.TSERV, - ResourceGroupId.DEFAULT)); + zl2.lock(zlw2, new ServiceLockData(UUID.randomUUID(), ServerIdUtil.tserver("test1", 0), + ThriftService.TSERV)); // The call above creates two nodes in ZK because of the overridden create method in // ZooKeeperWrapper. // The nodes created are: @@ -528,8 +528,8 @@ public void run() { ServiceLock zl = getZooLock(zk, parent, uuid); getLockLatch.countDown(); // signal we are done getLockLatch.await(); // wait for others to finish - zl.lock(lockWatcher, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV, - ResourceGroupId.DEFAULT)); // race to the lock + zl.lock(lockWatcher, new ServiceLockData(UUID.randomUUID(), + ServerIdUtil.tserver("test1", 0), ThriftService.TSERV)); // race to the lock lockCompletedLatch.countDown(); unlockLatch.await(); zl.unlock(); @@ -646,8 +646,8 @@ public void testTryLock() throws Exception { TestALW lw = new TestALW(); - boolean ret = zl.tryLock(lw, new ServiceLockData(UUID.randomUUID(), "test1", - ThriftService.TSERV, ResourceGroupId.DEFAULT)); + boolean ret = zl.tryLock(lw, new ServiceLockData(UUID.randomUUID(), + ServerIdUtil.tserver("test1", 0), ThriftService.TSERV)); assertTrue(ret); @@ -675,14 +675,14 @@ public void testChangeData() throws Exception { TestALW lw = new TestALW(); - ServiceLockData sld1 = new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV, - ResourceGroupId.DEFAULT); + ServiceLockData sld1 = new ServiceLockData(UUID.randomUUID(), + ServerIdUtil.tserver("test1", 0), ThriftService.TSERV); zl.lock(lw, sld1); assertEquals(Optional.of(sld1), ServiceLockData.parse(zk.getData(zl.getLockPath(), null, null))); - ServiceLockData sld2 = new ServiceLockData(UUID.randomUUID(), "test2", ThriftService.TSERV, - ResourceGroupId.DEFAULT); + ServiceLockData sld2 = new ServiceLockData(UUID.randomUUID(), + ServerIdUtil.tserver("test2", 0), ThriftService.TSERV); zl.replaceLockData(sld2); assertEquals(Optional.of(sld2), ServiceLockData.parse(zk.getData(zl.getLockPath(), null, null))); diff --git a/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java b/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java index 5fc0563322c..484e39a9e66 100644 --- a/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java @@ -49,12 +49,14 @@ import org.apache.accumulo.core.client.admin.TabletAvailability; import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ClientContext; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.conf.ClientProperty; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.ResourceGroupId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.lock.ServiceLockPaths.ResourceGroupPredicate; +import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.core.metadata.schema.Ample; import org.apache.accumulo.core.metadata.schema.TabletMetadata; import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType; @@ -89,8 +91,8 @@ public class SuspendedTabletsIT extends AccumuloClusterHarness { public static final int TSERVERS = 3; public static final int TABLETS = 30; - private String defaultGroup; - private Set testGroup = new HashSet<>(); + private TServerInstance defaultGroup; + private Set testGroup = new HashSet<>(); private List tabletServerProcesses; @Override @@ -124,10 +126,13 @@ public void setUp() throws Exception { Map hostAndGroup = TabletResourceGroupBalanceIT.getTServerGroups(mac); hostAndGroup.forEach((k, v) -> { + HostAndPort hp = HostAndPort.fromString(k); if (v.equals(ResourceGroupId.DEFAULT)) { - defaultGroup = k; + defaultGroup = + new TServerInstance(ServerIdUtil.tserver(v, hp.getHost(), hp.getPort()), "0"); } else { - testGroup.add(k); + testGroup + .add(new TServerInstance(ServerIdUtil.tserver(v, hp.getHost(), hp.getPort()), "0")); } }); @@ -244,11 +249,11 @@ private void suspensionTestBody(TServerKiller serverStopper, AfterSuspendAction } while (ds.suspended.keySet().size() != (TSERVERS - 1) || (ds.suspendedCount + ds.hostedCount) != TABLETS); - SetMultimap deadTabletsByServer = ds.suspended; + SetMultimap deadTabletsByServer = ds.suspended; // All suspended tablets should "belong" to the dead tablet servers, and should be in exactly // the same place as before any tserver death. - for (HostAndPort server : deadTabletsByServer.keySet()) { + for (ServerId server : deadTabletsByServer.keySet()) { // Comparing pre-death, hosted tablets to suspended tablets on a server assertEquals(beforeDeathState.hosted.get(server), deadTabletsByServer.get(server)); } @@ -266,8 +271,8 @@ private void suspensionTestBody(TServerKiller serverStopper, AfterSuspendAction } } else if (action == AfterSuspendAction.RESUME) { // Restart the first tablet server, making sure it ends up on the same port - HostAndPort restartedServer = deadTabletsByServer.keySet().iterator().next(); - log.info("Restarting " + restartedServer); + ServerId restartedServer = deadTabletsByServer.keySet().iterator().next(); + log.info("Restarting " + restartedServer.toHostPortString()); ((MiniAccumuloClusterImpl) getCluster())._exec(TabletServer.class, ServerType.TABLET_SERVER, Map.of(Property.TSERV_CLIENTPORT.getKey(), "" + restartedServer.getPort(), Property.TSERV_PORTSEARCH.getKey(), "false"), @@ -279,7 +284,6 @@ private void suspensionTestBody(TServerKiller serverStopper, AfterSuspendAction Thread.sleep(1000); ds = TabletLocations.retrieve(ctx, tableName); } - assertEquals(deadTabletsByServer.get(restartedServer), ds.hosted.get(restartedServer)); // Finally, after much longer, remaining suspended tablets should be reassigned. log.info("Awaiting tablet reassignment for remaining tablets (suspension timeout)"); @@ -289,6 +293,7 @@ private void suspensionTestBody(TServerKiller serverStopper, AfterSuspendAction } // Ensure all suspension markers in the metadata table were cleared. + assertTrue(ds.hostedCount == TABLETS); assertTrue(ds.suspended.isEmpty()); } else { throw new IllegalStateException("Unknown action " + action); @@ -310,7 +315,7 @@ public void eliminateTabletServers(ClientContext ctx, TabletLocations locs, int try { ThriftClientTypes.MANAGER.executeVoid(ctx, client -> { log.info("Sending shutdown command to {} via ManagerClientService", ts); - client.shutdownTabletServer(null, ctx.rpcCreds(), ts, false); + client.shutdownTabletServer(null, ctx.rpcCreds(), ts.toZooKeeperPathString(), false); }, ResourceGroupPredicate.DEFAULT_RG_ONLY); } catch (AccumuloSecurityException | AccumuloException e) { throw new RuntimeException("Error calling shutdownTabletServer for " + ts, e); @@ -358,8 +363,8 @@ public static void cleanup() { private static class TabletLocations { public final Map locationStates = new HashMap<>(); - public final SetMultimap hosted = HashMultimap.create(); - public final SetMultimap suspended = HashMultimap.create(); + public final SetMultimap hosted = HashMultimap.create(); + public final SetMultimap suspended = HashMultimap.create(); public int hostedCount = 0; public int assignedCount = 0; public int suspendedCount = 0; @@ -407,10 +412,10 @@ private void scan(ClientContext ctx, String tableName) { } locationStates.put(ke, tm); if (tm.getSuspend() != null) { - suspended.put(tm.getSuspend().server, ke); + suspended.put(tm.getSuspend().server.getServer(), ke); ++suspendedCount; } else if (tm.hasCurrent()) { - hosted.put(tm.getLocation().getHostAndPort(), ke); + hosted.put(tm.getLocation().getServerInstance().getServer(), ke); ++hostedCount; } else if (tm.getLocation() != null && tm.getLocation().getType().equals(LocationType.FUTURE)) { diff --git a/test/src/main/java/org/apache/accumulo/test/metrics/MetricsThriftRpcIT.java b/test/src/main/java/org/apache/accumulo/test/metrics/MetricsThriftRpcIT.java index 6cc46a29f91..f275ce60a57 100644 --- a/test/src/main/java/org/apache/accumulo/test/metrics/MetricsThriftRpcIT.java +++ b/test/src/main/java/org/apache/accumulo/test/metrics/MetricsThriftRpcIT.java @@ -53,8 +53,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.net.HostAndPort; - public class MetricsThriftRpcIT extends ConfigurableMacBase { private static final Logger log = LoggerFactory.getLogger(MetricsThriftRpcIT.class); @@ -113,8 +111,8 @@ public void testRpc() throws Exception { Set managers = client.instanceOperations().getServers(ServerId.Type.MANAGER); assertEquals(1, managers.size()); ServerId managerServer = managers.iterator().next(); - Client metricsClient = ThriftUtil.getClient(ThriftClientTypes.SERVER_PROCESS, - HostAndPort.fromParts(managerServer.getHost(), managerServer.getPort()), cc); + Client metricsClient = + ThriftUtil.getClient(ThriftClientTypes.SERVER_PROCESS, managerServer, cc); try { MetricResponse response = metricsClient.getMetrics(TraceUtil.traceInfo(), getCluster().getServerContext().rpcCreds()); @@ -128,9 +126,8 @@ public void testRpc() throws Exception { assertNotNull(zgcPath, "Garbage Collector not found in ZooKeeper"); Optional sld = cc.getZooCache().getLockData(zgcPath); assertTrue(sld.isPresent(), "Garbage Collector ZooKeeper lock data not found"); - String location = sld.orElseThrow().getAddressString(ThriftService.GC); - HostAndPort hp = HostAndPort.fromString(location); - metricsClient = ThriftUtil.getClient(ThriftClientTypes.SERVER_PROCESS, hp, cc); + ServerId location = sld.orElseThrow().getServer(ThriftService.GC); + metricsClient = ThriftUtil.getClient(ThriftClientTypes.SERVER_PROCESS, location, cc); try { MetricResponse response = metricsClient.getMetrics(TraceUtil.traceInfo(), getCluster().getServerContext().rpcCreds()); @@ -144,8 +141,7 @@ public void testRpc() throws Exception { Set compactors = client.instanceOperations().getServers(ServerId.Type.COMPACTOR); assertEquals(4, compactors.size()); for (ServerId server : compactors) { - metricsClient = ThriftUtil.getClient(ThriftClientTypes.SERVER_PROCESS, - HostAndPort.fromParts(server.getHost(), server.getPort()), cc); + metricsClient = ThriftUtil.getClient(ThriftClientTypes.SERVER_PROCESS, server, cc); try { MetricResponse response = metricsClient.getMetrics(TraceUtil.traceInfo(), getCluster().getServerContext().rpcCreds()); @@ -159,8 +155,7 @@ public void testRpc() throws Exception { Set sservers = client.instanceOperations().getServers(ServerId.Type.SCAN_SERVER); assertEquals(3, sservers.size()); for (ServerId server : sservers) { - metricsClient = ThriftUtil.getClient(ThriftClientTypes.SERVER_PROCESS, - HostAndPort.fromParts(server.getHost(), server.getPort()), cc); + metricsClient = ThriftUtil.getClient(ThriftClientTypes.SERVER_PROCESS, server, cc); try { MetricResponse response = metricsClient.getMetrics(TraceUtil.traceInfo(), getCluster().getServerContext().rpcCreds()); @@ -174,8 +169,7 @@ public void testRpc() throws Exception { Set tservers = client.instanceOperations().getServers(ServerId.Type.TABLET_SERVER); assertEquals(2, tservers.size()); for (ServerId server : tservers) { - metricsClient = ThriftUtil.getClient(ThriftClientTypes.SERVER_PROCESS, - HostAndPort.fromParts(server.getHost(), server.getPort()), cc); + metricsClient = ThriftUtil.getClient(ThriftClientTypes.SERVER_PROCESS, server, cc); try { MetricResponse response = metricsClient.getMetrics(TraceUtil.traceInfo(), getCluster().getServerContext().rpcCreds()); diff --git a/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java index 785eeacfcae..4e6fa351c56 100644 --- a/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java +++ b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java @@ -30,12 +30,12 @@ import org.apache.accumulo.core.cli.ConfigOpts; import org.apache.accumulo.core.cli.Help; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.clientImpl.thrift.ClientService; import org.apache.accumulo.core.clientImpl.thrift.TInfo; import org.apache.accumulo.core.conf.DefaultConfiguration; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.conf.SiteConfiguration; -import org.apache.accumulo.core.data.ResourceGroupId; import org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan; import org.apache.accumulo.core.dataImpl.thrift.InitialScan; import org.apache.accumulo.core.dataImpl.thrift.IterInfo; @@ -355,8 +355,8 @@ public void failedToAcquireLock(Exception e) { } catch (KeeperException | InterruptedException e) { throw new IllegalStateException("Error creating path in ZooKeeper", e); } - ServiceLockData sld = new ServiceLockData(nullTServerUUID, "localhost", ThriftService.TSERV, - ResourceGroupId.DEFAULT); + ServiceLockData sld = new ServiceLockData(nullTServerUUID, + ServerIdUtil.tserver("localhost", 0), ThriftService.TSERV); miniLock = new ServiceLock(zk, slp, UUID.randomUUID()); miniLock.lock(miniLockWatcher, sld); context.setServiceLock(miniLock); @@ -366,7 +366,8 @@ public void failedToAcquireLock(Exception e) { List assignments = new ArrayList<>(); try (var tablets = context.getAmple().readTablets().forLevel(DataLevel.USER).build()) { long randomSessionID = opts.port; - TServerInstance instance = new TServerInstance(addr, randomSessionID); + TServerInstance instance = new TServerInstance( + ServerIdUtil.tserver(addr.getHost(), addr.getPort()), randomSessionID); var s = tablets.iterator(); while (s.hasNext()) { diff --git a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java index 286a3587f72..fb76601b018 100644 --- a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java +++ b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java @@ -365,14 +365,17 @@ private static List findTablets(ClientContext context, boolean select for (var tabletMeta : tabletsMeta) { var loc = tabletMeta.getLocation(); if (loc != null && loc.getType() == TabletMetadata.LocationType.CURRENT) { - boolean isLocal = loc.getHost().equals(localaddress.getHostName()); + boolean isLocal = + loc.getServerInstance().getServer().getHost().equals(localaddress.getHostName()); if (selectLocalTablets && isLocal) { candidates.add(tabletMeta.getExtent()); - tabletLocations.put(tabletMeta.getExtent(), loc.getHostPort()); + tabletLocations.put(tabletMeta.getExtent(), + loc.getServerInstance().getServer().toHostPortString()); } else if (!selectLocalTablets && !isLocal) { candidates.add(tabletMeta.getExtent()); - tabletLocations.put(tabletMeta.getExtent(), loc.getHostPort()); + tabletLocations.put(tabletMeta.getExtent(), + loc.getServerInstance().getServer().toHostPortString()); } } } diff --git a/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java index af45bee4ff1..8a284f142d4 100644 --- a/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java +++ b/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java @@ -84,6 +84,7 @@ import org.apache.accumulo.core.lock.ServiceLockPaths.AddressSelector; import org.apache.accumulo.core.lock.ServiceLockPaths.ResourceGroupPredicate; import org.apache.accumulo.core.metadata.SystemTables; +import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.core.metadata.UnreferencedTabletFile; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily; import org.apache.accumulo.core.security.Authorizations; @@ -2389,7 +2390,8 @@ public void testListTablets() throws Exception { String[] tokens = line.split("\\s+"); if (tokens[1].startsWith("loc")) { String loc = tokens[3]; - assertTrue(results.contains(loc)); + TServerInstance tsi = TServerInstance.deserialize(loc); + assertTrue(results.contains(tsi.getServer().toHostPortString())); } if (tokens[1].startsWith("file")) { String[] parts = tokens[1].split("/"); diff --git a/test/src/test/java/org/apache/accumulo/test/ChaoticLoadBalancerTest.java b/test/src/test/java/org/apache/accumulo/test/ChaoticLoadBalancerTest.java index b9145377cd7..92ae4b98013 100644 --- a/test/src/test/java/org/apache/accumulo/test/ChaoticLoadBalancerTest.java +++ b/test/src/test/java/org/apache/accumulo/test/ChaoticLoadBalancerTest.java @@ -144,7 +144,7 @@ public void testUnevenAssignment() { servers.clear(); for (char c : "abcdefghijklmnopqrstuvwxyz".toCharArray()) { String cString = Character.toString(c); - TabletServerId tsi = new TabletServerIdImpl("127.0.0.1", c, cString); + TabletServerId tsi = new TabletServerIdImpl("127.0.0.1", 1100 + c, cString); FakeTServer fakeTServer = new FakeTServer(); servers.put(tsi, fakeTServer); fakeTServer.tablets.add(makeTablet(cString, null, null));