From a0172de0e5ad12f08a007eab208c5ce325b02c52 Mon Sep 17 00:00:00 2001 From: Dave Marion Date: Mon, 8 Sep 2025 16:50:55 +0000 Subject: [PATCH 1/9] Use ServerId to represent a server address As part of #5775 ThriftTransportPool.getAnyCachedTransport was modified to perform a ZooKeeper lookup to get the ResourceGroup for a server because that information is not available on the ThriftTransportKey. ThriftTransportKey contains the HostAndPort only. The reason for this change is to make the ResourceGroup available for the ResourceGroupPredicate that is passed into `ThriftTransportPool.getAnyCachedTransport` so that the ZooKeeper lookup can be removed. The server address is currently represented by either a String or a HostAndPort object in the code. Additionally the addresses are persisted in file paths in HDFS and in the root and metadata table. I opted to use the ServerId object as a replacement as it already contains the necessary information and is part of the public API, but making the change touched a lot of code. Closes #5848 --- core/pom.xml | 2 + .../client/MutationsRejectedException.java | 7 +- .../core/client/TimedOutException.java | 10 +- .../accumulo/core/client/admin/Locations.java | 3 +- .../core/client/admin/servers/ServerId.java | 157 +++- .../clientImpl/AccumuloServerException.java | 11 +- .../core/clientImpl/ClientContext.java | 18 +- .../core/clientImpl/ClientTabletCache.java | 24 +- .../clientImpl/ClientTabletCacheImpl.java | 21 +- .../clientImpl/ConditionalWriterImpl.java | 55 +- .../clientImpl/InstanceOperationsImpl.java | 57 +- .../clientImpl/RootClientTabletCache.java | 16 +- .../clientImpl/ScanServerAttemptImpl.java | 7 +- .../clientImpl/ScanServerAttemptsImpl.java | 3 +- .../clientImpl/SyncingClientTabletCache.java | 5 +- .../core/clientImpl/TableOperationsImpl.java | 17 +- .../clientImpl/TabletInformationImpl.java | 4 +- .../TabletServerBatchReaderIterator.java | 62 +- .../clientImpl/TabletServerBatchWriter.java | 63 +- .../core/clientImpl/ThriftScanner.java | 29 +- .../core/clientImpl/ThriftTransportKey.java | 10 +- .../core/clientImpl/ThriftTransportPool.java | 57 +- .../clientImpl/TimeoutClientTabletCache.java | 5 +- .../core/clientImpl/ZookeeperLockChecker.java | 19 +- .../accumulo/core/lock/ServiceLockData.java | 54 +- .../manager/balancer/TabletServerIdImpl.java | 26 +- .../MetadataCachedTabletObtainer.java | 6 +- .../core/metadata/SuspendingTServer.java | 34 +- .../core/metadata/TServerInstance.java | 110 ++- .../core/metadata/schema/TabletMetadata.java | 53 +- .../metadata/schema/TabletMutatorBase.java | 11 +- .../metadata/schema/filters/GcWalsFilter.java | 2 +- .../apache/accumulo/core/rpc/ThriftUtil.java | 21 +- .../clients/ClientServiceThriftClient.java | 3 +- .../core/rpc/clients/ManagerClient.java | 11 +- .../ServerProcessServiceThriftClient.java | 8 +- .../core/rpc/clients/TServerClient.java | 21 +- ...etManagementClientServiceThriftClient.java | 3 +- .../rpc/clients/TabletServerThriftClient.java | 3 +- .../core/spi/balancer/GroupBalancer.java | 5 +- .../balancer/HostRegexTableLoadBalancer.java | 4 +- .../core/spi/balancer/SimpleLoadBalancer.java | 4 +- .../spi/balancer/data/TabletServerId.java | 5 +- .../ConfigurableScanServerHostSelector.java | 28 +- .../scan/ConfigurableScanServerSelector.java | 20 +- .../core/spi/scan/ScanServerAttempt.java | 4 +- .../core/spi/scan/ScanServerInfo.java | 12 +- .../core/spi/scan/ScanServerSelections.java | 3 +- .../accumulo/core/summary/Gatherer.java | 38 +- .../core/tabletserver/log/LogEntry.java | 10 +- .../accumulo/core/util/AddressUtil.java | 22 - .../accumulo/core/util/cache/Caches.java | 1 + .../compaction/ExternalCompactionUtil.java | 35 +- .../util/compaction/RunningCompaction.java | 24 +- .../clientImpl/ClientTabletCacheImplTest.java | 205 ++--- .../core/clientImpl/ScanAttemptsImplTest.java | 23 +- .../clientImpl/ThriftTransportKeyTest.java | 15 +- .../core/lock/ServiceLockDataTest.java | 69 +- .../core/lock/ServiceLockPathsTest.java | 27 +- .../core/metadata/SuspendingTServerTest.java | 19 +- .../metadata/schema/TabletMetadataTest.java | 71 +- .../BaseHostRegexTableLoadBalancerTest.java | 2 +- ...xTableLoadBalancerReconfigurationTest.java | 12 +- .../HostRegexTableLoadBalancerTest.java | 42 +- .../spi/balancer/SimpleLoadBalancerTest.java | 4 +- .../spi/balancer/TableLoadBalancerTest.java | 6 +- ...onfigurableScanServerHostSelectorTest.java | 56 +- .../ConfigurableScanServerSelectorTest.java | 133 ++-- .../accumulo/core/util/AddressUtilTest.java | 25 - .../mapred/AccumuloRecordReader.java | 28 +- .../mapreduce/AccumuloRecordReader.java | 28 +- .../mapreduce/lib/InputConfigurator.java | 17 +- .../MiniAccumuloClusterImpl.java | 4 +- .../accumulo/server/AbstractServer.java | 22 +- .../accumulo/server/log/WalStateManager.java | 9 +- .../server/manager/LiveTServerSet.java | 54 +- .../balancer/BalancerEnvironmentImpl.java | 7 +- .../state/LoggingTabletStateStore.java | 2 +- .../state/TabletManagementParameters.java | 25 +- .../state/TabletManagementScanner.java | 3 +- .../ConditionalTabletMutatorImpl.java | 6 +- .../CurrentLocationNotEqualToIterator.java | 9 +- .../apache/accumulo/server/util/Admin.java | 55 +- .../apache/accumulo/server/util/ECAdmin.java | 11 +- .../accumulo/server/util/ListInstances.java | 2 +- .../server/util/ServiceStatusCmd.java | 2 +- .../server/util/TabletServerLocks.java | 2 +- .../server/util/VerifyTabletAssignments.java | 17 +- .../apache/accumulo/server/util/ZooZap.java | 4 +- .../constraints/MetadataConstraintsTest.java | 22 +- .../server/manager/LiveTServerSetTest.java | 7 +- .../state/AbstractTabletStateStoreTest.java | 7 +- .../state/TabletManagementParametersTest.java | 6 +- .../manager/state/TabletManagementTest.java | 8 +- .../state/ZooTabletStateStoreTest.java | 6 +- .../accumulo/server/util/AdminTest.java | 14 +- .../server/util/ServiceStatusCmdTest.java | 703 ++++++++++++++---- .../apache/accumulo/compactor/Compactor.java | 20 +- .../accumulo/gc/SimpleGarbageCollector.java | 8 +- .../gc/GarbageCollectWriteAheadLogsTest.java | 6 +- server/manager/pom.xml | 4 + .../org/apache/accumulo/manager/Manager.java | 24 +- .../manager/ManagerClientServiceHandler.java | 9 +- .../accumulo/manager/TabletGroupWatcher.java | 8 +- .../coordinator/CompactionCoordinator.java | 80 +- .../coordinator/commit/CommitCompaction.java | 2 +- .../manager/tableOps/bulkVer2/LoadFiles.java | 14 +- .../tableOps/bulkVer2/TabletRefresher.java | 4 +- .../manager/tserverOps/ShutdownTServer.java | 22 +- .../manager/upgrade/Upgrader11to12.java | 43 +- .../compaction/CompactionCoordinatorTest.java | 16 +- .../manager/tableOps/ShutdownTServerTest.java | 10 +- .../tableOps/merge/MergeTabletsTest.java | 14 +- .../tableOps/split/UpdateTabletsTest.java | 15 +- .../manager/upgrade/Upgrader11to12Test.java | 36 +- .../org/apache/accumulo/monitor/Monitor.java | 62 +- .../monitor/next/InformationFetcher.java | 13 +- .../rest/compactions/CompactionInfo.java | 7 +- .../rest/compactions/CompactionsResource.java | 5 +- .../compactions/external/CoordinatorInfo.java | 6 +- .../external/ExternalCompactionInfo.java | 8 +- .../monitor/rest/scans/ScansResource.java | 6 +- .../monitor/rest/tables/TablesResource.java | 4 +- .../rest/tservers/TabletServerResource.java | 6 +- .../apache/accumulo/tserver/ScanServer.java | 9 +- .../apache/accumulo/tserver/TabletServer.java | 23 +- .../tserver/log/TabletServerLogger.java | 2 +- .../tserver/CheckTabletMetadataTest.java | 38 +- .../commands/ListTabletsCommandTest.java | 7 +- ...ehensiveTableOperationsIT_SimpleSuite.java | 4 +- .../accumulo/test/CorruptMutationIT.java | 4 +- .../org/apache/accumulo/test/ECAdminIT.java | 6 +- .../accumulo/test/InstanceOperationsIT.java | 4 +- .../org/apache/accumulo/test/LocatorIT.java | 13 +- .../accumulo/test/TestDualAssignment.java | 8 +- .../apache/accumulo/test/TotalQueuedIT.java | 5 +- .../accumulo/test/TransportCachingIT.java | 9 +- .../usage/TabletFileUpdateIT_SimpleSuite.java | 8 +- .../ClassLoaderContextCompactionIT.java | 7 +- .../ExternalCompactionProgressIT.java | 4 +- .../ExternalCompactionTestUtils.java | 9 +- .../compaction/ExternalCompaction_3_IT.java | 8 +- .../apache/accumulo/test/fate/TestLock.java | 4 +- .../functional/AmpleConditionalWriterIT.java | 23 +- .../test/functional/CompactionIT.java | 3 +- .../test/functional/GarbageCollectorIT.java | 4 +- .../test/functional/GracefulShutdownIT.java | 26 +- .../test/functional/LastLocationIT.java | 15 +- .../test/functional/ManagerAssignmentIT.java | 24 +- .../test/functional/MemoryStarvedMajCIT.java | 11 +- .../test/functional/RegexGroupBalanceIT.java | 4 +- .../test/functional/SplitRecoveryIT.java | 6 +- .../TabletManagementIteratorIT.java | 11 +- .../TabletResourceGroupBalanceIT.java | 20 +- .../test/functional/ZombieTServer.java | 7 +- .../accumulo/test/lock/ServiceLockIT.java | 54 +- .../test/manager/SuspendedTabletsIT.java | 4 +- .../test/metrics/MetricsThriftRpcIT.java | 20 +- .../test/performance/NullTserver.java | 8 +- .../performance/scan/CollectTabletStats.java | 9 +- .../test/ChaoticLoadBalancerTest.java | 2 +- 161 files changed, 2284 insertions(+), 1623 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 69dfa7d7d8c..20e593bd580 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -285,6 +285,8 @@ org[.]apache[.]hadoop[.]mapred[.](FileAlreadyExistsException|InvalidJobConfException) org[.]apache[.]accumulo[.]access[.].* + + com[.]google[.]common[.]net[.]HostAndPort 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..2a8dbadeb98 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,30 @@ */ 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.conf.PropertyType.PortRange; 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; +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 { + + private static final long serialVersionUID = 1L; /** * Server process type names. @@ -38,15 +49,128 @@ 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; + } + + public static record ServerIdInfo(String type, String resourceGroup, String host, int port) { + public ServerId getServerId() { + return new 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(); + + private static ServerId resolve(ServerIdInfo info) { + return cache.get(info, k -> info.getServerId()); + } + + public static ServerId compactor(HostAndPort hp) { + return resolve(new ServerIdInfo(Type.COMPACTOR.name(), ResourceGroupId.DEFAULT.canonical(), + hp.getHost(), hp.getPort())); + } + + 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 compactor(ResourceGroupId rgid, HostAndPort hp) { + return resolve( + new ServerIdInfo(Type.COMPACTOR.name(), rgid.canonical(), hp.getHost(), hp.getPort())); + } + + 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(HostAndPort hp) { + return resolve(new ServerIdInfo(Type.SCAN_SERVER.name(), ResourceGroupId.DEFAULT.canonical(), + hp.getHost(), hp.getPort())); + } + + 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 sserver(ResourceGroupId rgid, HostAndPort hp) { + return resolve( + new ServerIdInfo(Type.SCAN_SERVER.name(), rgid.canonical(), hp.getHost(), hp.getPort())); + } + + public static ServerId tserver(HostAndPort hp) { + return resolve(new ServerIdInfo(Type.TABLET_SERVER.name(), ResourceGroupId.DEFAULT.canonical(), + hp.getHost(), hp.getPort())); + } + + 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 tserver(ResourceGroupId rgid, HostAndPort hp) { + return resolve( + new ServerIdInfo(Type.TABLET_SERVER.name(), rgid.canonical(), hp.getHost(), hp.getPort())); + } + + public static ServerId dynamic(Type type, ResourceGroupId rgid, HostAndPort hp) { + return resolve(new ServerIdInfo(type.name(), rgid.canonical(), hp.getHost(), hp.getPort())); + } + + 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) { + String parts[] = name.split("\\+"); + Preconditions.checkArgument(parts.length == 2, "Invalid server id in wal file: " + name); + // return an uncached tserver object + return ServerId.tserver(parts[0], Integer.parseInt(parts[1])); + } + + 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) { + private ServerId(Type type, ResourceGroupId resourceGroup, String host, int port) { super(); Preconditions.checkArgument(port == 0 || PortRange.VALID_RANGE.contains(port), "invalid server port value: " + port); @@ -54,6 +178,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 +197,13 @@ public int getPort() { return port; } + public synchronized HostAndPort getHostPort() { + if (hostPort == null) { + hostPort = HostAndPort.fromParts(host, port); + } + return hostPort; + } + @Override public int compareTo(ServerId other) { if (this == other) { @@ -116,7 +248,24 @@ public String toString() { + ", port= " + port + "]"; } + public String toWalFileName() { + return host + "+" + port; + } + public String toHostPortString() { - return host + ":" + port; + return getHostPort().toString(); + } + + public ServerIdInfo toServerIdInfo() { + return new ServerIdInfo(getType().name(), getResourceGroup().canonical(), getHost(), getPort()); + } + + public String serialize() { + return GSON.get().toJson(toServerIdInfo()); + } + + 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..54cef2b2aa8 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 ServerId.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 3224eb5e32f..77b8564d4a6 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; @@ -269,7 +271,7 @@ public KeyExtent getExtent() { return tablet_extent; } - public Optional getTserverLocation() { + public Optional getTserverLocation() { return Optional.ofNullable(tserverLocation); } @@ -301,9 +303,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<>(); } @@ -317,7 +319,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 e7b87d3f07a..41aa6803191 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 2cd95897019..e9013157184 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)); } } @@ -449,17 +446,24 @@ 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 { - ping(server.toHostPortString()); + try (TTransport transport = createTransport(server, context)) { + ClientService.Client client = createClient(ThriftClientTypes.CLIENT, transport); + client.ping(context.rpcCreds()); + } catch (TException e) { + throw new AccumuloException(e); + } } @Override @@ -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 ServerId.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/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 07af9bd6ec4..948ec0b6cd6 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; @@ -1428,7 +1429,7 @@ private void waitForTableStateTransition(TableId tableId, TableState expectedSta lastRow = tablet.getExtent().toMetaRow(); if (loc != null) { - serverCounts.increment(loc.getHostPortSession(), 1); + serverCounts.increment(loc.getServerInstance().toHostPortSessionString(), 1); } } @@ -1593,7 +1594,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 @@ -1897,15 +1898,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()); @@ -1923,7 +1924,7 @@ public LocationsImpl(Map>> binnedRanges) { } @Override - public String getTabletLocation(TabletId tabletId) { + public ServerId getTabletLocation(TabletId tabletId) { return tabletLocations.get(tabletId); } @@ -1978,7 +1979,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 0f84154a150..da350d19305 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 e9dea2705b1..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,50 +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 = null; - switch (service) { - case CLIENT: - paths = (selector) -> new CompositeSet( - ctx.getServerPaths().getCompactor(rgp, selector, true), - ctx.getServerPaths().getScanServer(rgp, selector, true), - ctx.getServerPaths().getTabletServer(rgp, selector, true)); - break; - case COMPACTOR: - paths = (selector) -> ctx.getServerPaths().getCompactor(rgp, selector, true); - break; - case MANAGER: - case COORDINATOR: - case FATE: - paths = (selector) -> Set.of(ctx.getServerPaths().getManager(true)); - break; - case GC: - paths = (selector) -> Set.of(ctx.getServerPaths().getGarbageCollector(true)); - break; - case TABLET_SCAN: - paths = (selector) -> new CompositeSet( - ctx.getServerPaths().getTabletServer(rgp, selector, true), - ctx.getServerPaths().getScanServer(rgp, selector, true)); - break; - case TABLET_INGEST: - case TABLET_MANAGEMENT: - case TSERV: - paths = (selector) -> ctx.getServerPaths().getTabletServer(rgp, selector, true); - break; - case NONE: - 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); } } @@ -199,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..90ba31cc72c 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; @@ -31,8 +32,6 @@ import org.apache.accumulo.core.lock.ServiceLockPaths.ServiceLockPath; import org.apache.accumulo.core.zookeeper.ZooCache; -import com.google.common.net.HostAndPort; - public class ZookeeperLockChecker implements TabletServerLockChecker { private final ZooCache zc; @@ -43,20 +42,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(server.getHostPort()), 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(server.getHostPort()), 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..3a3d6af9fde 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.client.admin.servers.ServerId.ServerIdInfo; 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 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 = address.toServerIdInfo(); } } 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..733c01fb6c9 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,10 @@ import static java.util.Objects.requireNonNull; +import org.apache.accumulo.core.client.admin.servers.ServerId; 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 +35,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(ServerId.tserver(host, port), session); + } + + public TabletServerIdImpl(ServerId server, String session) { + requireNonNull(server); + this.tServerInstance = new TServerInstance(server, session); } public TabletServerIdImpl(TServerInstance tServerInstance) { @@ -49,13 +52,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 +85,7 @@ public int hashCode() { @Override public String toString() { - return getHost() + ':' + getPort() + '[' + getSession() + ']'; + return getServer().getHost() + ':' + getServer().getPort() + '[' + getSession() + ']'; } public TServerInstance toThrift() { @@ -98,9 +96,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 335392d30d6..d1075741fc3 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..4ad4e4a56fd 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,18 @@ */ 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.ServerIdInfo; +import org.apache.accumulo.core.client.admin.servers.ServerId.Type; + +import com.google.common.base.Preconditions; import com.google.common.net.HostAndPort; /** @@ -32,41 +38,58 @@ * 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(ServerIdInfo server, String session) { + public TServerInstance getTSI() { + return new TServerInstance(server.getServerId(), session); + } + } + + public static TServerInstance deserialize(String json) { + return GSON.get().fromJson(json, TServerInstanceInfo.class).getTSI(); } - public TServerInstance(String formattedString) { + public static TServerInstance fromHostPortSessionString(String formattedString) { int pos = formattedString.indexOf("["); if (pos < 0 || !formattedString.endsWith("]")) { - throw new IllegalArgumentException(formattedString); + // if no session, then use zero + var hostAndPort = HostAndPort.fromString(formattedString); + return new TServerInstance(ServerId.tserver(hostAndPort), Long.toHexString(0)); + } else { + var hostAndPort = HostAndPort.fromString(formattedString.substring(0, pos)); + var session = formattedString.substring(pos + 1, formattedString.length() - 1); + return new TServerInstance(ServerId.tserver(hostAndPort), session); } - 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 TServerInstance(HostAndPort address, long session) { - this(address, Long.toHexString(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; + this.hostPortSession = server.getHostPort() + "[" + this.session + "]"; } - public TServerInstance(String address, long session) { - this(AddressUtil.parseAddress(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); + this.hostPortSession = server.getHostPort() + "[" + this.session + "]"; } - public TServerInstance(Value address, Text session) { - this(AddressUtil.parseAddress(new String(address.get(), UTF_8)), session.toString()); + public TServerInstance(String json) { + var partial = GSON.get().fromJson(json, TServerInstanceInfo.class).getTSI(); + this.server = partial.server; + this.session = partial.session; + this.hostPortSession = server.getHostPort() + "[" + this.session + "]"; } @Override @@ -74,12 +97,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 +117,33 @@ public boolean equals(Object obj) { return false; } + public String toHostPortSessionString() { + return hostPortSession; + } + @Override public String toString() { - return hostPortSession; + return toHostPortSessionString(); } - 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(server.toServerIdInfo(), 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(); + this.hostPortSession = server.getHostPort() + "[" + this.session + "]"; } } 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 eec27ec3c9d..09110993d7c 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 @@ -181,15 +181,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(); } @@ -250,7 +251,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(); } @@ -397,7 +398,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..8523ab06869 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::toHostPortSessionString).peek(tsi -> { if (tsi.contains(",")) { throw new IllegalArgumentException(tsi); } 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 e7792e3b8d4..2a013e3110a 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.rpc.SaslConnectionParams.SaslMechanism; import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; @@ -44,8 +45,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.net.HostAndPort; - /** * Factory methods for creating Thrift client objects */ @@ -98,7 +97,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); } @@ -110,8 +109,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); @@ -126,8 +125,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); @@ -159,7 +158,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()); @@ -194,7 +193,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; @@ -229,7 +228,7 @@ public static TTransport createClientTransport(HostAndPort address, int timeout, // Make sure a timeout is set try { - transport = TTimeoutTransport.create(address, timeout); + transport = TTimeoutTransport.create(address.getHostPort(), timeout); } catch (TTransportException e) { log.warn("Failed to open transport to {}", address); throw e; @@ -303,7 +302,7 @@ public static TTransport createClientTransport(HostAndPort address, int timeout, transport.open(); } else { try { - transport = TTimeoutTransport.create(address, timeout); + transport = TTimeoutTransport.create(address.getHostPort(), timeout); } catch (TTransportException ex) { log.warn("Failed to open transport to {}", address); throw ex; 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 6265d981bb3..e591bb5ee21 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,12 +107,12 @@ 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()); warned.set(false); - return new Pair(cachedTransport.getFirst(), client); + return new Pair(cachedTransport.getFirst(), client); } } @@ -155,7 +156,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, @@ -165,7 +166,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( @@ -208,10 +209,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); @@ -248,10 +249,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 c5afdaaab67..cb59b084d44 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..7e4fa1be56f 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,7 @@ import java.util.Objects; import java.util.UUID; +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.Value; @@ -36,7 +37,7 @@ public final class LogEntry { private final String path; - private final HostAndPort tserver; + private HostAndPort tserver; private final UUID uniqueId; private final Text columnQualifier; @@ -51,8 +52,9 @@ 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/type+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 @@ -80,7 +82,7 @@ private static LogEntry validatedLogEntry(String path, Text columnQualifier) { } HostAndPort tserver; try { - tserver = HostAndPort.fromString(tserverPart.replace("+", ":")); + tserver = ServerId.fromWalFileName(tserverPart).getHostPort(); } catch (IllegalArgumentException e) { throw new IllegalArgumentException(badTServerMsg); } 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..4393178ba4c 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,6 +34,7 @@ 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.thrift.ThriftSecurityException; import org.apache.accumulo.core.compaction.thrift.CompactorService; @@ -103,25 +104,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 -> { groupsAndAddresses .computeIfAbsent(slp.getResourceGroup().canonical(), (k) -> new HashSet<>()) - .add(HostAndPort.fromString(slp.getServer())); + .add(ServerId.compactor(slp.getResourceGroup(), + HostAndPort.fromString(slp.getServer()))); }); return groupsAndAddresses; } @@ -133,7 +135,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 +158,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 +178,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 +210,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 = ServerId.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 +222,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, ServerId.compactor(rcf.getGroup(), rcf.getCompactor()))); } } catch (InterruptedException | ExecutionException e) { throw new IllegalStateException(e); @@ -237,8 +240,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 = ServerId.compactor(slp.getResourceGroup(), hp.getHost(), hp.getPort()); + futures.add(executor.submit(() -> getRunningCompactionId(sid, context))); }); executor.shutdown(); @@ -271,8 +275,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..8c65d6f43ec 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,7 @@ import java.util.Objects; import java.util.TreeMap; +import org.apache.accumulo.core.client.admin.servers.ServerId; 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 +31,28 @@ 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(), ServerId.compactor(ResourceGroupId.of(tEC.getGroupName()), + HostAndPort.fromString(tEC.getCompactor()))); } public Map getUpdates() { @@ -74,12 +74,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 372af800838..61b2d418ce7 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 @@ -46,11 +46,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; @@ -68,6 +70,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; @@ -82,6 +85,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 ServerId.tserver(host, port); + } + + static ServerId csi(String host) { + return ServerId.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)); @@ -127,15 +138,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; @@ -147,7 +158,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; @@ -188,7 +199,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; @@ -234,15 +245,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); @@ -294,8 +305,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); @@ -315,11 +326,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); @@ -358,12 +369,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); } @@ -504,13 +516,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) { @@ -551,7 +563,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; } @@ -567,14 +579,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"); @@ -583,7 +595,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; } @@ -600,20 +612,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(), @@ -628,7 +641,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(); @@ -642,7 +655,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)); } @@ -661,7 +674,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()); } @@ -868,7 +881,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)))); @@ -898,7 +911,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))), @@ -917,7 +930,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"), @@ -991,7 +1004,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"))))); @@ -1002,7 +1015,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()); @@ -1010,7 +1023,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))); @@ -1018,26 +1031,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()); @@ -1053,14 +1066,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); @@ -1069,7 +1082,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"))), @@ -1081,7 +1094,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"))), @@ -1110,7 +1123,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")))); @@ -1158,7 +1171,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)), @@ -1192,13 +1205,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))); @@ -1212,7 +1225,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))); @@ -1220,12 +1233,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))); @@ -1239,7 +1252,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); @@ -1262,7 +1275,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"); } @@ -1277,7 +1290,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); @@ -1321,7 +1334,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"); @@ -1371,7 +1384,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) { @@ -1381,7 +1394,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) { @@ -1400,7 +1413,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"))))); @@ -1416,7 +1429,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")))), @@ -1483,7 +1496,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)); @@ -1560,8 +1573,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 @@ -1610,7 +1623,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); @@ -1646,7 +1659,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"))), @@ -1727,7 +1740,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); @@ -1754,7 +1767,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); @@ -1763,7 +1776,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); @@ -1782,7 +1795,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"); @@ -1822,7 +1835,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); @@ -1890,11 +1903,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); @@ -1903,11 +1917,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); @@ -2013,11 +2028,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; }); @@ -2078,16 +2093,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); @@ -2117,13 +2132,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()); @@ -2133,7 +2148,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()); @@ -2210,7 +2225,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..fb9f9cc8a02 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 @@ -27,6 +27,7 @@ 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 org.apache.accumulo.core.spi.scan.ScanServerAttempt; import org.junit.jupiter.api.Test; @@ -40,7 +41,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 +58,7 @@ public void testBasic() { var tablet1 = nti("1", "a"); - var reporter1 = sai.createReporter("ss1:1", tablet1); + var reporter1 = sai.createReporter(ServerId.sserver("ss1", 1101), tablet1); reporter1.report(ScanServerAttempt.Result.BUSY); @@ -65,21 +66,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(ServerId.sserver("ss1", 1101), tablet2); var tablet3 = nti("2", "r"); - var reporter3 = sai.createReporter("ss2:2", tablet3); + var reporter3 = sai.createReporter(ServerId.sserver("ss2", 1102), tablet3); reporter2.report(ScanServerAttempt.Result.BUSY); reporter3.report(ScanServerAttempt.Result.ERROR); @@ -87,10 +88,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..a7401a7d4ba 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 @@ -30,6 +30,7 @@ import java.security.PrivilegedExceptionAction; import java.util.Properties; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.client.security.tokens.AuthenticationToken; import org.apache.accumulo.core.client.security.tokens.KerberosToken; import org.apache.accumulo.core.conf.ClientProperty; @@ -43,8 +44,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 +78,7 @@ public void testSslAndSaslErrors() { try { assertThrows(RuntimeException.class, () -> new ThriftTransportKey(ThriftClientTypes.CLIENT, - HostAndPort.fromParts("localhost", 9999), 120_000, clientCtx)); + ServerId.tserver("localhost", 9999), 120_000, clientCtx)); } finally { verify(clientCtx); } @@ -99,9 +98,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); + ServerId.tserver("localhost", 9997), 1L, null, saslParams1); ThriftTransportKey ttk2 = new ThriftTransportKey(ThriftClientTypes.CLIENT, - HostAndPort.fromParts("localhost", 9997), 1L, null, saslParams2); + ServerId.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 +119,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); + ServerId.tserver("localhost", 9997), 1L, null, saslParams1); ThriftTransportKey ttk2 = new ThriftTransportKey(ThriftClientTypes.CLIENT, - HostAndPort.fromParts("localhost", 9997), 1L, null, saslParams2); + ServerId.tserver("localhost", 9997), 1L, null, saslParams2); assertNotEquals(ttk1, ttk2); assertNotEquals(ttk1.hashCode(), ttk2.hashCode()); @@ -138,7 +137,7 @@ public void testSimpleEquivalence() { replay(clientCtx); ThriftTransportKey ttk = new ThriftTransportKey(ThriftClientTypes.CLIENT, - HostAndPort.fromParts("localhost", 9999), 120_000, clientCtx); + ServerId.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..dfb43d8f360 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.client.admin.servers.ServerId; import org.apache.accumulo.core.data.ResourceGroupId; import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptor; import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptors; @@ -42,83 +42,80 @@ public class ServiceLockDataTest { @Test public void testSingleServiceConstructor() throws Exception { ServiceLockData ss = - new ServiceLockData(serverUUID, "127.0.0.1", ThriftService.TSERV, ResourceGroupId.DEFAULT); + new ServiceLockData(serverUUID, ServerId.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, + ServerId.tserver("127.0.0.1", 9997))); + sds.addService(new ServiceDescriptor(serverUUID, ThriftService.TABLET_SCAN, + ServerId.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"), + ss.getServer(ThriftService.TSERV).getHostPort()); 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)); + ss.getServer(ThriftService.TABLET_SCAN).getHostPort()); 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, + ServerId.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, + ServerId.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, + ServerId.tserver(ResourceGroupId.of("meta"), "127.0.0.1", 9997))); + sds.addService(new ServiceDescriptor(serverUUID, ThriftService.TABLET_SCAN, + ServerId.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"), + ss.getServer(ThriftService.TSERV).getHostPort()); 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)); + ss.getServer(ThriftService.TABLET_SCAN).getHostPort()); 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..dde19a15d03 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.client.admin.servers.ServerId; 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, ServerId.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,7 @@ 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, ServerId.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 +264,7 @@ 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, ServerId.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 +300,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 sld1 = new ServiceLockData(uuid, ServerId.compactor(TEST_RESOURCE_GROUP, "localhost", 9876), + ThriftService.COMPACTOR); var sld2 = - new ServiceLockData(uuid, HOSTNAME, ThriftService.COMPACTOR, ResourceGroupId.DEFAULT); + new ServiceLockData(uuid, ServerId.compactor("localhost", 9877), ThriftService.COMPACTOR); expect(zc.getChildren(ZCOMPACTORS)) .andReturn(List.of(TEST_RESOURCE_GROUP.canonical(), ResourceGroupId.DEFAULT.canonical())) @@ -444,9 +446,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 sld1 = new ServiceLockData(uuid, ServerId.sserver(TEST_RESOURCE_GROUP, "localhost", 9876), + ThriftService.TABLET_SCAN); var sld2 = - new ServiceLockData(uuid, HOSTNAME, ThriftService.TABLET_SCAN, ResourceGroupId.DEFAULT); + new ServiceLockData(uuid, ServerId.sserver("localhost", 9877), ThriftService.TABLET_SCAN); expect(zc.getChildren(ZSSERVERS)) .andReturn(List.of(TEST_RESOURCE_GROUP.canonical(), ResourceGroupId.DEFAULT.canonical())) @@ -582,9 +585,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 sld1 = new ServiceLockData(uuid, ServerId.tserver(TEST_RESOURCE_GROUP, "localhost", 9876), + ThriftService.TABLET_SCAN); var sld2 = - new ServiceLockData(uuid, HOSTNAME, ThriftService.TABLET_SCAN, ResourceGroupId.DEFAULT); + new ServiceLockData(uuid, ServerId.tserver("localhost", 9877), ThriftService.TABLET_SCAN); expect(zc.getChildren(ZTSERVERS)) .andReturn(List.of(TEST_RESOURCE_GROUP.canonical(), ResourceGroupId.DEFAULT.canonical())) @@ -720,9 +724,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 sld1 = new ServiceLockData(uuid, ServerId.tserver(TEST_RESOURCE_GROUP, "localhost", 9876), + ThriftService.TABLET_SCAN); var sld2 = - new ServiceLockData(uuid, HOSTNAME, ThriftService.TABLET_SCAN, ResourceGroupId.DEFAULT); + new ServiceLockData(uuid, ServerId.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..502fa48b02d 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.client.admin.servers.ServerId; import org.apache.accumulo.core.util.time.SteadyTime; import org.junit.jupiter.api.Test; @@ -33,14 +34,15 @@ 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(ServerId.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), st1.server.getServer().getHostPort()); assertEquals(suspensionTime, st1.suspensionTime); assertEquals(val1, st1.toValue()); - var st2 = new SuspendingTServer(HostAndPort.fromParts("server1", 8555), suspensionTime); + var st2 = new SuspendingTServer(new TServerInstance(ServerId.tserver("server1", 8555), "s001"), + suspensionTime); assertEquals(st1, st2); assertEquals(st1.hashCode(), st2.hashCode()); assertEquals(st1.toString(), st2.toString()); @@ -48,11 +50,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(ServerId.tserver("server2", 8555), "s001"), + suspensionTime); + var st4 = new SuspendingTServer(new TServerInstance(ServerId.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(ServerId.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..4cf8f004310 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.client.admin.servers.ServerId; 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(ServerId.tserver("server1", 8555), "s001"); + TServerInstance last = new TServerInstance(ServerId.tserver("server2", 8555), "s000"); + mutation.put(MetadataSchema.TabletsSection.CompactedColumnFamily.STR_NAME, fateId1.canonical(), ""); @@ -167,9 +171,10 @@ 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()); le1.addToMutation(mutation); @@ -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(ServerId.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,15 @@ 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), + tm.getLocation().getServerInstance().getServer().getHostPort()); 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), + tm.getLast().getServerInstance().getServer().getHostPort()); + 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 +297,9 @@ public void testAllColumns() { public void testFuture() { KeyExtent extent = new KeyExtent(TableId.of("5"), new Text("df"), new Text("da")); + TServerInstance tsi = new TServerInstance(ServerId.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 +307,9 @@ 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), + tm.getLocation().getServerInstance().getServer().getHostPort()); + assertEquals("s001", tm.getLocation().getServerInstance().getSession()); assertEquals(LocationType.FUTURE, tm.getLocation().getType()); assertFalse(tm.hasCurrent()); } @@ -311,9 +318,10 @@ public void testFuture() { public void testFutureAndCurrent() { KeyExtent extent = new KeyExtent(TableId.of("5"), new Text("df"), new Text("da")); + TServerInstance tsi = new TServerInstance(ServerId.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 +336,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(ServerId.tserver("server1", 8555), "s001"); + TServerInstance ser2 = new TServerInstance(ServerId.tserver("server2", 8111), "s002"); + TServerInstance deadSer = new TServerInstance(ServerId.tserver("server3", 8000), "s003"); Set tservers = new LinkedHashSet<>(); tservers.add(ser1); tservers.add(ser2); @@ -340,7 +348,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 +357,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 +403,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()); } @@ -734,7 +742,7 @@ private SortedMap toRowMap(Mutation mutation) { @Test public void testBuilder() { - TServerInstance ser1 = new TServerInstance(HostAndPort.fromParts("server1", 8555), "s001"); + TServerInstance ser1 = new TServerInstance(ServerId.tserver("server1", 8555), "s001"); KeyExtent extent = new KeyExtent(TableId.of("5"), new Text("df"), new Text("da")); @@ -763,7 +771,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(ServerId.tserver("localhost", 9999), 1000L); TabletMetadata tm = TabletMetadata.builder(extent) .putTabletAvailability(TabletAvailability.UNHOSTED).putLocation(Location.future(ser1)) @@ -849,7 +857,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..34cce8ce58f 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,7 @@ 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 org.apache.accumulo.core.spi.scan.ConfigurableScanServerSelectorTest.InitParams; import org.apache.accumulo.core.spi.scan.ConfigurableScanServerSelectorTest.SelectorParams; @@ -42,19 +43,19 @@ 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 = ServerId.sserver("host1", 2000); + private final ServerId ss2 = ServerId.sserver("host1", 2001); + private final ServerId ss3 = ServerId.sserver("host1", 2002); + private final ServerId ss4 = ServerId.sserver("host1", 2003); + private final ServerId ss5 = ServerId.sserver("host2", 2000); + private final ServerId ss6 = ServerId.sserver("host2", 2001); + private final ServerId ss7 = ServerId.sserver("host2", 2002); + private final ServerId ss8 = ServerId.sserver("host2", 2003); + private final ServerId ss9 = ServerId.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 +67,18 @@ 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 HostAndPort firstHP = firstServer.getHostPort(); - final Set remainingServers = new HashSet<>(); + final Set remainingServers = new HashSet<>(); if (host1Servers.contains(firstServer)) { remainingServers.addAll(host1Servers); firstHostSeen = true; @@ -93,8 +97,8 @@ 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); + ServerId selectedServer = selection.getScanServer(tId); + HostAndPort selectedHP = selectedServer.getHostPort(); assertEquals(selectedHP.getHost(), firstHP.getHost()); assertTrue(remainingServers.remove(selectedServer)); attempts.add(new TestScanServerAttempt(selectedServer, Result.BUSY)); @@ -102,8 +106,8 @@ public void test() { // 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); + ServerId secondServer = selection.getScanServer(tId); + final HostAndPort secondHP = secondServer.getHostPort(); assertFalse(secondHP.getHost().equals(firstHP.getHost())); if (host1Servers.contains(secondServer)) { @@ -132,8 +136,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); + ServerId selectedServer = selection.getScanServer(tId); + HostAndPort selectedHP = selectedServer.getHostPort(); assertEquals(selectedHP.getHost(), secondHP.getHost()); assertTrue(remainingServers.remove(selectedServer)); attempts.add(new TestScanServerAttempt(selectedServer, Result.BUSY)); @@ -142,8 +146,8 @@ 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); + ServerId thirdServer = selection.getScanServer(tId); + final HostAndPort thirdHP = thirdServer.getHostPort(); assertFalse(thirdHP.getHost().equals(firstHP.getHost())); assertFalse(thirdHP.getHost().equals(secondHP.getHost())); @@ -173,8 +177,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); + ServerId selectedServer = selection.getScanServer(tId); + HostAndPort selectedHP = selectedServer.getHostPort(); assertEquals(selectedHP.getHost(), thirdHP.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..fec1a0a893c 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,7 @@ 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.data.ResourceGroupId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.data.TabletId; @@ -49,13 +50,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 +65,18 @@ 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 -> scanServersMap + .put(ServerId.sserver(HostAndPort.fromString(sserv)), 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 +96,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 +148,21 @@ 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) { + this.server = ServerId.sserver(HostAndPort.fromString(server)); + this.result = result; + } + @Override - public String getServer() { + public ServerId getServer() { return server; } @@ -178,10 +180,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 +210,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 +271,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 +304,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 +460,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(ServerId.sserver("ss1", 1101), dg, ServerId.sserver("ss2", 1102), dg, + ServerId.sserver("ss3", 1103), dg, + ServerId.sserver(ResourceGroupId.of("g1"), "ss4", 1104), ResourceGroupId.of("g1"), + ServerId.sserver(ResourceGroupId.of("g1"), "ss5", 1105), ResourceGroupId.of("g1"), + ServerId.sserver(ResourceGroupId.of("g2"), "ss6", 1106), ResourceGroupId.of("g2"), + ServerId.sserver(ResourceGroupId.of("g2"), "ss7", 1107), ResourceGroupId.of("g2"), + ServerId.sserver(ResourceGroupId.of("g2"), "ss8", 1108), ResourceGroupId.of("g2")), opts)); Set servers = new HashSet<>(); @@ -447,10 +476,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 +492,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 +508,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 +524,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 +543,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 +556,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(ServerId.sserver("ss1", 1101), dg, ServerId.sserver("ss2", 1102), dg, + ServerId.sserver("ss3", 1103), dg)); Optional optional = condition.get(); @@ -542,7 +572,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(ServerId.sserver("ss1", 1101), ServerId.sserver("ss2", 1102), + ServerId.sserver("ss3", 1103)).contains(actions.getScanServer(tabletId))); assertFalse(scanServers.get().isEmpty()); } } 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 971aafa860c..182045bdd52 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; @@ -282,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)) { @@ -336,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()) { @@ -366,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 = ServerId.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); @@ -390,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); } @@ -408,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 311c71d4632..f818bcc59a5 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; @@ -317,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)) { @@ -368,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()) { @@ -399,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 = ServerId.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); @@ -423,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); @@ -445,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..6b72acdf092 100644 --- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java +++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java @@ -66,6 +66,7 @@ import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.client.security.tokens.AuthenticationToken; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.conf.AccumuloConfiguration; @@ -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, ServerId.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..37c2c5d688a 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 @@ -78,12 +78,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 +100,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 @@ -117,12 +120,12 @@ protected AbstractServer(ServerId.Type serverType, ConfigOpts opts, 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<>(ServerId.dynamic(serverType, resourceGroup, advertHP)); } 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 +313,7 @@ public void registerMetrics(MeterRegistry registry) { getContext().setMeterRegistry(registry); } - public HostAndPort getAdvertiseAddress() { + public ServerId getAdvertiseAddress() { return advertiseAddress.get(); } @@ -330,12 +333,12 @@ 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 thriftBindAddress; } else { return curr; } @@ -358,7 +361,8 @@ protected void updateThriftServer(ThriftServerSupplier supplier, boolean start) log.info("Starting {} Thrift server, listening on {}", this.getClass().getSimpleName(), thriftServer.address); } - updateAdvertiseAddress(thriftServer.address); + updateAdvertiseAddress( + ServerId.dynamic(this.serverType, this.resourceGroup, thriftServer.address)); } 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..eda598c4807 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.toHostPortSessionString(), data, + NodeExistsPolicy.FAIL); } catch (KeeperException | InterruptedException e) { throw new WalMarkerException(e); } @@ -133,7 +133,8 @@ private void updateState(TServerInstance tsi, Path path, WalState state) policy = NodeExistsPolicy.FAIL; } log.debug("Setting {} to {}", path.getName(), state); - zoo.putPersistentData(root() + "/" + tsi + "/" + path.getName(), data, policy); + zoo.putPersistentData(root() + "/" + tsi.toHostPortSessionString() + "/" + path.getName(), + data, policy); } catch (KeeperException | InterruptedException e) { throw new WalMarkerException(e); } @@ -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.fromHostPortSessionString(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 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 194809d4549..9f14a581c11 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); 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); 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), @@ -297,21 +297,21 @@ 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); + ServerId server = sld.orElseThrow().getServer(ServiceLockData.ThriftService.TSERV); ResourceGroupId resourceGroup = sld.orElseThrow().getGroup(ServiceLockData.ThriftService.TSERV); - TServerInstance instance = new TServerInstance(address, stat.getEphemeralOwner()); + TServerInstance instance = new TServerInstance(server, stat.getEphemeralOwner()); if (info == null) { updates.add(instance); TServerInfo tServerInfo = - new TServerInfo(instance, new TServerConnection(address), resourceGroup); + new TServerInfo(instance, new TServerConnection(server), resourceGroup); 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); + new TServerInfo(instance, new TServerConnection(server), resourceGroup); current.put(tserverPath.getServer(), tServerInfo); } } @@ -460,23 +460,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.fromHostPortSessionString(tabletServer); for (Entry entry : servers.entrySet()) { - if (entry.getValue().instance.getHostAndPort().equals(addr)) { + if (entry.getValue().instance.getServer().getHostPort() + .equals(target.getServer().getHostPort())) { // 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; } } 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..97988327029 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 @@ -108,7 +108,7 @@ public void suspend(Collection tablets, var location = tm.getLocation(); HostAndPort server = null; if (location != null) { - server = location.getHostAndPort(); + server = location.getServerInstance().getServer().getHostPort(); } 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 86c7d0d0e02..dfc184b5edf 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 @@ -81,6 +81,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 +91,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 +715,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 +779,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.getHostPort().toString()); 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.toHostPortSessionString(), force), + ResourceGroupPredicate.DEFAULT_RG_ONLY); } } } @@ -793,19 +798,20 @@ 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(ServerId.tserver(hpObj), 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(ServerId.tserver(hpObj), Long.toHexString(0)); } - return hostAndPort + "[" + Long.toHexString(sessionId) + "]"; + return new TServerInstance(ServerId.tserver(hpObj), Long.toHexString(sessionId)); } private static final String ACCUMULO_SITE_BACKUP_FILE = "accumulo.properties.bak"; @@ -1140,8 +1146,9 @@ private ServiceLock createAdminLock(ServerContext context) throws InterruptedExc ServiceLock adminLock = new ServiceLock(zk, slp, uuid); 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)); + descriptors + .addService(new ServiceLockData.ServiceDescriptor(uuid, ServiceLockData.ThriftService.NONE, + ServerId.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..9cb53927588 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.client.admin.servers.ServerId; 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(ServerId.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/manager/LiveTServerSetTest.java b/server/base/src/test/java/org/apache/accumulo/server/manager/LiveTServerSetTest.java index 15e9d5e3ac5..c234238c53d 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,6 +24,7 @@ import java.util.HashMap; import java.util.Map; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.data.ResourceGroupId; import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection; @@ -31,8 +32,6 @@ import org.easymock.EasyMock; import org.junit.jupiter.api.Test; -import com.google.common.net.HostAndPort; - public class LiveTServerSetTest { @Test @@ -41,8 +40,8 @@ public void testSessionIds() { TServerConnection mockConn = EasyMock.createMock(TServerConnection.class); TServerInfo server1 = - new TServerInfo(new TServerInstance(HostAndPort.fromParts("localhost", 1234), "5555"), - mockConn, ResourceGroupId.DEFAULT); + new TServerInfo(new TServerInstance(ServerId.tserver("localhost", 1234), "5555"), mockConn, + ResourceGroupId.DEFAULT); servers.put("server1", server1); assertEquals(server1.instance, LiveTServerSet.find(servers, "localhost:1234")); 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..48c23978562 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.client.admin.servers.ServerId; 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(ServerId.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(ServerId.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..25e558a1807 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.client.admin.servers.ServerId; 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(ServerId.tserver("127.0.0.1", 10000), 0), + new TServerInstance(ServerId.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..a6caf7d3823 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.client.admin.servers.ServerId; 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,9 +106,11 @@ 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(ServerId.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(ServerId.tserver("server2", 8555), "s000").serialize()); LogEntry le1 = LogEntry.fromPath("localhost+8020/" + UUID.randomUUID()); le1.addToMutation(mutation); 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..f45d0a1e31c 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.client.admin.servers.ServerId; 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 @@ -51,8 +50,7 @@ public void testZooTabletStateStore() throws DistributedStoreException { ZooTabletStateStore tstore = new ZooTabletStateStore(DataLevel.ROOT, context); String sessionId = "this is my unique session data"; - TServerInstance server = - new TServerInstance(HostAndPort.fromParts("127.0.0.1", 10000), sessionId); + TServerInstance server = new TServerInstance(ServerId.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..d2d5e05057a 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 @@ -44,8 +44,8 @@ import java.util.stream.Collectors; import org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ClientContext; -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.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 @@ -78,8 +81,8 @@ public void testQualifySessionId() throws KeeperException, InterruptedException String group = type + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME; String server = "localhost:12345"; final long session = 123456789L; - ServiceLockData sld1 = new ServiceLockData(UUID.randomUUID(), server, ThriftService.TABLET_SCAN, - ResourceGroupId.DEFAULT); + ServiceLockData sld1 = new ServiceLockData(UUID.randomUUID(), + ServerId.tserver(HostAndPort.fromString(server)), ThriftService.TABLET_SCAN); String serverPath = group + "/" + server; String validZLockEphemeralNode = "zlock#" + UUID.randomUUID() + "#0000000000"; @@ -99,7 +102,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(ServerId.tserver("localhost", 12345), session), Admin.qualifyWithZooKeeperSessionId(ctx, zc, server)); verify(ctx, zc); @@ -121,7 +124,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(ServerId.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..5794fd5bc83 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 @@ -139,7 +139,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.net.HostAndPort; import io.micrometer.core.instrument.FunctionCounter; import io.micrometer.core.instrument.Gauge; @@ -365,12 +364,12 @@ 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); + final ServiceLockPath path = getContext().getServerPaths() + .createCompactorPath(getResourceGroup(), clientAddress.getHostPort()); ServiceLockSupport.createNonHaServiceLockPath(Type.COMPACTOR, zoo, path); compactorLock = new ServiceLock(getContext().getZooSession(), path, compactorId); LockWatcher lw = new ServiceLockWatcher(Type.COMPACTOR, () -> getShutdownComplete().get(), @@ -383,8 +382,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 +539,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 +548,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 +751,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.getHostPort(), clientAddress.getResourceGroup()); } private void performFailureProcessing(ConsecutiveErrorHistory errorHistory) @@ -808,7 +806,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/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java index 5b09324d4a3..4adcf718ba4 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) { @@ -184,7 +185,7 @@ public void run() { metricsInfo.addMetricsProducers(this, new GcMetrics(this)); metricsInfo.init(MetricsInfo.serviceTags(getContext().getInstanceName(), getApplicationName(), - getAdvertiseAddress(), getResourceGroup())); + address.getHostPort(), address.getResourceGroup())); 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..08bbfb0c71f 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,7 @@ 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.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.dataImpl.KeyExtent; @@ -52,8 +53,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 = ServerId.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 73dfeee6311..0b7171d7ded 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 @@ -1143,7 +1143,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().getHostPort(), getAdvertiseAddress().getResourceGroup())); ThreadPools.watchCriticalScheduledTask(context.getScheduledExecutor() .scheduleWithFixedDelay(() -> ScanServerMetadataEntries.clean(context), 10, 10, MINUTES)); @@ -1183,7 +1183,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().getHostPort()); // Replace the ServiceLockData information in the Manager lock node in ZooKeeper. // This advertises the address that clients can use to connect to the Manager @@ -1193,8 +1193,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); @@ -1389,8 +1388,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 = @@ -1430,7 +1429,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) { @@ -1439,7 +1438,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); } } @@ -1473,13 +1472,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; } @@ -1562,7 +1561,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(); @@ -1571,7 +1571,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..fc3df3359c5 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); @@ -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 975ac8f2454..5fc45d607b8 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 @@ -859,11 +859,11 @@ 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 = null; - Iterator find = tLists.destinations - .tailMap(new TServerInstance(tm.getSuspend().server, " ")).keySet().iterator(); + Iterator find = + tLists.destinations.tailMap(tm.getSuspend().server).keySet().iterator(); if (find.hasNext()) { TServerInstance found = find.next(); - if (found.getHostAndPort().equals(tm.getSuspend().server)) { + if (found.equals(tm.getSuspend().server)) { returnInstance = found; } } @@ -927,7 +927,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..4152bfb1d33 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 @@ -242,7 +242,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 +419,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 +438,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()); } } }); @@ -522,7 +522,8 @@ public TNextCompactionJob getCompactionJob(TInfo tinfo, TCredentials credentials // 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. RUNNING_CACHE.put(ExternalCompactionId.of(result.getExternalCompactionId()), - new RunningCompaction(result, compactorAddress, groupId)); + new RunningCompaction(result, + ServerId.compactor(groupId, HostAndPort.fromString(compactorAddress)))); TabletLogger.compacting(rcJob.getExtent(), rcJob.getSelectedFateId(), cid, compactorAddress, rcJob); break; @@ -852,9 +853,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 +892,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 +904,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 +1046,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 +1074,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 +1111,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 +1147,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 +1177,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 +1202,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 +1221,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 +1362,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 +1406,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..0362489b78e 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 @@ -83,7 +83,7 @@ public Repo call(FateId fateId, Manager manager) throws Exception { String loc = null; if (tabletMetadata != null && tabletMetadata.getLocation() != null) { - loc = tabletMetadata.getLocation().getHostPortSession(); + loc = tabletMetadata.getLocation().getServerInstance().toHostPortSessionString(); } // 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/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..ade9db2b1dc 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; @@ -35,27 +34,20 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.net.HostAndPort; - 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; @@ -99,11 +91,11 @@ public Repo call(FateId fateId, Manager manager) throws Exception { // suppress assignment of tablets to the server if (force) { ZooReaderWriter zoo = manager.getContext().getZooSession().asReaderWriter(); - var path = - manager.getContext().getServerPaths().createTabletServerPath(resourceGroup, hostAndPort); + var path = manager.getContext().getServerPaths().createTabletServerPath( + server.getServer().getResourceGroup(), server.getServer().getHostPort()); ServiceLock.deleteLock(zoo, path); - path = manager.getContext().getServerPaths().createDeadTabletServerPath(resourceGroup, - hostAndPort); + path = manager.getContext().getServerPaths().createDeadTabletServerPath( + server.getServer().getResourceGroup(), server.getServer().getHostPort()); 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..d90197a0058 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 @@ -48,6 +48,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.clientImpl.Namespace; import org.apache.accumulo.core.clientImpl.NamespaceMapping; import org.apache.accumulo.core.conf.Property; @@ -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,32 @@ 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(ServerId.tserver(hp), 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 +828,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/compaction/CompactionCoordinatorTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/compaction/CompactionCoordinatorTest.java index 3cb474fa408..46eef4f6d91 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 @@ -88,8 +88,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 +96,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 = ServerId.tserver(GROUP_ID, "192.168.1.1", 9090); public MetricsInfo getMockMetrics() { MetricsInfo metricsInfo = createMock(MetricsInfo.class); @@ -219,7 +217,7 @@ protected TExternalCompactionJob createThriftJob(String externalCompactionId, } @Override - protected void cancelCompactionOnCompactor(String address, String externalCompactionId) {} + protected void cancelCompactionOnCompactor(ServerId address, String externalCompactionId) {} } @@ -289,7 +287,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 +305,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 +358,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..4ad2c186135 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.client.admin.servers.ServerId; 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(ServerId.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..7cfff852e47 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.client.admin.servers.ServerId; 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(ServerId.tserver("1.2.3.4", 1234), "123456789")); + var suspendingTServer = + new SuspendingTServer(new TServerInstance(ServerId.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(ServerId.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(ServerId.tserver("1.2.3.4", 1234), 123456789L); var futureLoc = TabletMetadata.Location.future(tserver); testUnexpectedColumn(tmb -> tmb.putLocation(futureLoc), "had location", futureLoc.toString()); 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..33e8ad39530 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.client.admin.servers.ServerId; 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(ServerId.tserver("1.2.3.4", 1234), "123456789")); + var suspendingTServer = + new SuspendingTServer(new TServerInstance(ServerId.tserver("1.2.3.4", 1025), ""), + SteadyTime.from(Duration.ofMillis(56))); + var migration = new TServerInstance(ServerId.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(ServerId.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)); 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 5163ecc5bf5..57c51ce25da 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 @@ -104,7 +104,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 +147,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 +239,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 +324,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 +332,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 +381,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 +392,11 @@ public void run() { throw new RuntimeException("Unable to get hostname for advertise address", e); } } - updateAdvertiseAddress(HostAndPort.fromParts(advertiseHost, livePort)); + updateAdvertiseAddress(ServerId.monitor(advertiseHost, livePort)); } else { - updateAdvertiseAddress(HostAndPort.fromParts(advertiseAddress.getHost(), livePort)); + updateAdvertiseAddress(ServerId.monitor(advertiseAddress.getHost(), livePort)); } - HostAndPort monitorHostAndPort = getAdvertiseAddress(); + ServerId monitorHostAndPort = getAdvertiseAddress(); log.debug("Using {} to advertise monitor location in ZooKeeper", monitorHostAndPort); try { @@ -412,7 +410,7 @@ public void run() { MetricsInfo metricsInfo = getContext().getMetricsInfo(); metricsInfo.addMetricsProducers(this); metricsInfo.init(MetricsInfo.serviceTags(getContext().getInstanceName(), getApplicationName(), - monitorHostAndPort, getResourceGroup())); + monitorHostAndPort.getHostPort(), getResourceGroup())); // Needed to support the existing zk monitor address format if (!rootContext.endsWith("/")) { @@ -557,13 +555,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 +575,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 +583,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 +626,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 +644,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 +721,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 +760,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 +830,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 930725bb527..0b1c58a285c 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 @@ -63,7 +63,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 { @@ -122,8 +121,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); @@ -183,22 +181,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/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..1ff50b151e3 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().getHostPort().toString()); } } 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().getHostPort().toString()); } 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..43877e098bf 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 @@ -36,6 +36,7 @@ import jakarta.ws.rs.QueryParam; import jakarta.ws.rs.core.MediaType; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; @@ -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,7 @@ public TabletServerSummary getTserverDetails( try { ClientContext context = monitor.getContext(); TabletServerClientService.Client client = - ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context); + ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, ServerId.tserver(address), 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/ScanServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java index f218f8f71f4..b418c2dc4eb 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 @@ -315,8 +315,8 @@ 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(), getAdvertiseAddress().getHostPort()); ServiceLockSupport.createNonHaServiceLockPath(Type.SCAN_SERVER, zoo, zLockPath); serverLockUUID = UUID.randomUUID(); scanServerLock = new ServiceLock(getContext().getZooSession(), zLockPath, serverLockUUID); @@ -329,8 +329,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 +377,7 @@ public void run() { metricsInfo.addMetricsProducers(this, scanMetrics, scanServerMetrics, blockCacheMetrics); metricsInfo.init(MetricsInfo.serviceTags(getContext().getInstanceName(), getApplicationName(), - getAdvertiseAddress(), getResourceGroup())); + getAdvertiseAddress().getHostPort(), getResourceGroup())); // 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/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java index 43be28270d6..329f385fb65 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 @@ -154,7 +154,6 @@ import org.slf4j.LoggerFactory; import com.google.common.annotations.VisibleForTesting; -import com.google.common.net.HostAndPort; public class TabletServer extends AbstractServer implements TabletHostingServer { @@ -417,13 +416,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 +431,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; @@ -485,7 +484,7 @@ private void announceExistence() { try { final ServiceLockPath zLockPath = context.getServerPaths() - .createTabletServerPath(getResourceGroup(), getAdvertiseAddress()); + .createTabletServerPath(getResourceGroup(), getAdvertiseAddress().getHostPort()); ServiceLockSupport.createNonHaServiceLockPath(Type.TABLET_SERVER, zoo, zLockPath); UUID tabletServerUUID = UUID.randomUUID(); tabletServerLock = new ServiceLock(getContext().getZooSession(), zLockPath, tabletServerUUID); @@ -500,8 +499,8 @@ 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))) { @@ -562,7 +561,7 @@ public void run() { metricsInfo.addMetricsProducers(this, metrics, updateMetrics, scanMetrics, mincMetrics, pausedMetrics, blockCacheMetrics); metricsInfo.init(MetricsInfo.serviceTags(context.getInstanceName(), getApplicationName(), - getAdvertiseAddress(), getResourceGroup())); + getAdvertiseAddress().getHostPort(), getResourceGroup())); announceExistence(); getContext().setServiceLock(tabletServerLock); @@ -598,7 +597,7 @@ public void run() { evaluateOnDemandTabletsForUnload(); }); - HostAndPort managerHost; + ServerId managerHost; final String advertiseAddressString = getAdvertiseAddress().toString(); while (!isShutdownRequested()) { if (Thread.currentThread().isInterrupted()) { @@ -676,7 +675,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()); + getTabletSession().toHostPortSessionString(), getResourceGroup().canonical()); } boolean managerDown = false; @@ -735,7 +734,7 @@ public TServerInstance getTabletSession() { } try { - return new TServerInstance(getAdvertiseAddress().toString(), lockSessionId); + return new TServerInstance(getAdvertiseAddress(), lockSessionId); } catch (Exception ex) { log.warn("Unable to read session from tablet server lock" + ex); return null; @@ -845,7 +844,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(); 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..e32a54f4d6b 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 @@ -281,7 +281,7 @@ private synchronized void startLogMaker() { try { alog = DfsLogger.createNew(tserver.getContext(), syncCounter, flushCounter, - tserver.getAdvertiseAddress().toString()); + tserver.getAdvertiseAddress().toWalFileName()); } 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/test/java/org/apache/accumulo/tserver/CheckTabletMetadataTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/CheckTabletMetadataTest.java index 65003ab4f92..7d329b77a20 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,7 @@ import java.util.EnumSet; import java.util.TreeMap; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.data.Value; @@ -42,6 +43,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 +61,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 +89,18 @@ private static void assertFail(TreeMap tabletMeta, KeyExtent ke, TSer } } + private static ServerId csi(String location) { + return ServerId.tserver(HostAndPort.fromString(location)); + } + @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 +109,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 +136,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/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..79800dc1dc5 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 @@ -37,6 +37,7 @@ import org.apache.accumulo.core.client.admin.InstanceOperations; 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.ClientContext; import org.apache.accumulo.core.clientImpl.TabletInformationImpl; import org.apache.accumulo.core.data.Range; @@ -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(ServerId.tserver("server1", 8555), "s001"); + TServerInstance ser2 = new TServerInstance(ServerId.tserver("server2", 2354), "s002"); StoredTabletFile sf11 = new ReferencedTabletFile(new Path("hdfs://nn1/acc/tables/1/t-dir1/sf11.rf")).insert(); 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..dc50ccd46cb 100644 --- a/test/src/main/java/org/apache/accumulo/test/InstanceOperationsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/InstanceOperationsIT.java @@ -35,7 +35,6 @@ import org.apache.accumulo.core.client.admin.InstanceOperations; import org.apache.accumulo.core.client.admin.servers.ServerId; 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 +198,7 @@ public void testPing() throws Exception { io.ping(sid); } - ServerId fake = - new ServerId(ServerId.Type.COMPACTOR, ResourceGroupId.DEFAULT, "localhost", 1024); + ServerId fake = ServerId.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..504c8d9521a 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().getHostPort() != 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/TestDualAssignment.java b/test/src/main/java/org/apache/accumulo/test/TestDualAssignment.java index d04ff0d68f0..625dac6ead8 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.client.admin.servers.ServerId; 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(ServerId.tserver("192.168.1.1", 9997), "56")); + var loc2 = TabletMetadata.Location + .future(new TServerInstance(ServerId.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/ample/usage/TabletFileUpdateIT_SimpleSuite.java b/test/src/main/java/org/apache/accumulo/test/ample/usage/TabletFileUpdateIT_SimpleSuite.java index 85178e1a7ed..e163d8e2043 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.client.admin.servers.ServerId; 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(ServerId.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,9 @@ 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(ServerId.tserver("localhost", 9998), 0xabcdef123L)), + Location + .current(new TServerInstance(ServerId.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/TestLock.java b/test/src/main/java/org/apache/accumulo/test/fate/TestLock.java index 598aed5c7e7..69a71da1d30 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.client.admin.servers.ServerId; 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, ServerId.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/AmpleConditionalWriterIT.java b/test/src/main/java/org/apache/accumulo/test/functional/AmpleConditionalWriterIT.java index c051fea6f5c..801594c91e8 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.client.admin.servers.ServerId; 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(ServerId.tserver("localhost", 9997), 5000L); + var ts2 = new TServerInstance(ServerId.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(ServerId.tserver("localhost", 9997), 5000L); + var ts2 = new TServerInstance(ServerId.tserver("localhost", 9997), 6000L); var context = getCluster().getServerContext(); @@ -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(ServerId.tserver("localhost", 9997), 5000L); + var ts2 = new TServerInstance(ServerId.tserver("localhost", 9997), 6000L); var context = getCluster().getServerContext(); @@ -1254,7 +1255,7 @@ 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(ServerId.tserver("server1", 8555), 1L); // on a subset of the tablets, put a location final Set tabletsWithLocation = Set.of(e2, e3, e4); @@ -1371,8 +1372,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(ServerId.tserver("localhost", 9997), 5000L); + var ts2 = new TServerInstance(ServerId.tserver("localhost", 9997), 6000L); try (var ctmi = context.getAmple().conditionallyMutateTablets()) { ctmi.mutateTablet(e1).requireAbsentOperation().requireAbsentLocation() @@ -1909,8 +1910,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(ServerId.tserver("localhost", 1234), 56L); + var otherTsi = new TServerInstance(ServerId.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 c3e4e1df400..9bb93f7444c 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..95aed9658ef 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.client.admin.servers.ServerId.Type; 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(); + Admin.signalGracefulShutdown(ctx, ServerId.dynamic(Type.valueOf(tserverPath.getType()), + tserverPath.getResourceGroup(), HostAndPort.fromString(tserverPath.getServer()))); 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,9 @@ 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 ServerId compactorAddress = ServerId.dynamic(Type.valueOf(compactorPath.getType()), + compactorPath.getResourceGroup(), HostAndPort.fromString(compactorPath.getServer())); final CompactionConfig cc = new CompactionConfig(); final IteratorSetting is = new IteratorSetting(100, SlowIterator.class); @@ -240,8 +240,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 +266,7 @@ 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(); try (final Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) { scanner.setRange(new Range()); scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL); @@ -279,7 +278,8 @@ public void testGracefulShutdown() throws Exception { assertNotNull(e); count++; if (count == 2) { - Admin.signalGracefulShutdown(ctx, sserver); + Admin.signalGracefulShutdown(ctx, ServerId.dynamic(Type.valueOf(sserverPath.getType()), + sserverPath.getResourceGroup(), HostAndPort.fromString(sserverPath.getServer()))); } } 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..14b8c347ae4 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,8 @@ 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().getHostPort(), + newTablet.getLast().getServerInstance().getServer().getHostPort()); // put something in it try (BatchWriter bw = c.createBatchWriter(tableName)) { @@ -70,8 +71,10 @@ 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().getHostPort(), + unflushed.getLocation().getServerInstance().getServer().getHostPort()); + assertEquals(newTablet.getLocation().getServerInstance().getServer().getHostPort(), + unflushed.getLast().getServerInstance().getServer().getHostPort()); assertTrue(newTablet.hasCurrent()); // take the tablet offline @@ -79,14 +82,16 @@ 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().getHostPort(), + offline.getLast().getServerInstance().getServer().getHostPort()); // put it back online, should have the same last location c.tableOperations().online(tableName, true); TabletMetadata online = ManagerAssignmentIT.getTabletMetadata(c, tableId, null); assertTrue(online.hasCurrent()); assertNotNull(online.getLocation()); - assertEquals(newTablet.getLast().getHostPort(), online.getLast().getHostPort()); + assertEquals(newTablet.getLast().getServerInstance().getServer().getHostPort(), + online.getLast().getServerInstance().getServer().getHostPort()); } } 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..2ba62ee658f 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 @@ -156,7 +156,8 @@ 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().getHostPort(), + flushed.getLast().getServerInstance().getServer().getHostPort()); assertFalse(flushed.getLocation().getType().equals(LocationType.FUTURE)); assertEquals(TabletAvailability.ONDEMAND, flushed.getTabletAvailability()); @@ -165,7 +166,8 @@ 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().getHostPort(), + offline.getLast().getServerInstance().getServer().getHostPort()); assertEquals(TabletAvailability.ONDEMAND, offline.getTabletAvailability()); // put it back online @@ -173,7 +175,8 @@ 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().getHostPort(), + online.getLast().getServerInstance().getServer().getHostPort()); assertEquals(TabletAvailability.ONDEMAND, online.getTabletAvailability()); // set the tablet availability to HOSTED @@ -189,7 +192,8 @@ 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().getHostPort(), + always.getLast().getServerInstance().getServer().getHostPort()); assertEquals(TabletAvailability.HOSTED, always.getTabletAvailability()); // set the hosting availability to never @@ -203,7 +207,8 @@ 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().getHostPort(), + unhosted.getLast().getServerInstance().getServer().getHostPort()); assertEquals(TabletAvailability.UNHOSTED, unhosted.getTabletAvailability()); // set the tablet availability to ONDEMAND @@ -215,7 +220,8 @@ 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().getHostPort(), + ondemand.getLast().getServerInstance().getServer().getHostPort()); assertEquals(TabletAvailability.ONDEMAND, ondemand.getTabletAvailability()); } @@ -517,11 +523,11 @@ 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); + HostAndPort address = location.getHostPort(); String addressWithSession = address.toString(); var zLockPath = getCluster().getServerContext().getServerPaths() .createTabletServerPath(ResourceGroupId.DEFAULT, address); @@ -573,7 +579,7 @@ 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); + HostAndPort address = location.getHostPort(); String addressWithSession = address.toString(); var zLockPath = getCluster().getServerContext().getServerPaths() .createTabletServerPath(ResourceGroupId.DEFAULT, address); 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..e1dcdbc1fc3 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()).toHostPortSessionString(); 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..eb69b0f37f1 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 @@ -38,6 +38,7 @@ import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.client.admin.TimeType; +import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.clientImpl.ScannerImpl; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.TableId; @@ -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,8 @@ 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()); + TServerInstance instance = + new TServerInstance(ServerId.tserver(HostAndPort.fromString(location)), 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..e226687f27e 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 @@ -58,6 +58,7 @@ import org.apache.accumulo.core.client.TableNotFoundException; 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.clientImpl.ClientContext; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; @@ -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(ServerId.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(ServerId.tserver("fakeServer", 0), "fakeSession"))); tm.automaticallyPutServerLock(false); tm.mutate(); } @@ -695,7 +699,8 @@ private static TabletManagementParameters createParameters(AccumuloClient client .getTabletServer(ResourceGroupPredicate.ANY, AddressSelector.all(), true)) { try { long sessionId = ServiceLock.getSessionId(context.getZooCache(), tserver); - tservers.add(new TServerInstance(tserver.getServer(), sessionId)); + tservers.add(new TServerInstance(ServerId.tserver(tserver.getResourceGroup(), + HostAndPort.fromString(tserver.getServer())), 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..b8e82c96d55 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().getHostPort().toString())); 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().getHostPort().toString())); 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().getHostPort().toString())); locations.forEach(loc -> assertEquals(l1, loc.getLocation())); // change the resource group property for the table @@ -304,14 +307,15 @@ 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().getHostPort().toString()) + .equals(ResourceGroupId.of("GROUP1")))) { locations = getLocations(ample, tableId); } Location group1Location = locations.get(0).getLocation(); - assertTrue(tserverGroups.get(group1Location.getHostAndPort().toString()) - .equals(ResourceGroupId.of("GROUP1"))); + assertTrue( + tserverGroups.get(group1Location.getServerInstance().getServer().getHostPort().toString()) + .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 2e9768397a9..89760c90428 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,7 @@ 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.thrift.ClientService; import org.apache.accumulo.core.clientImpl.thrift.TInfo; import org.apache.accumulo.core.conf.Property; @@ -131,8 +132,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(); @@ -171,8 +170,8 @@ 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.tserver(serverPort.address), 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..73bc41e49c2 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.client.admin.servers.ServerId; 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(), ServerId.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(), ServerId.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(), ServerId.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(), ServerId.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(), ServerId.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(), ServerId.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(), ServerId.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(), ServerId.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(), ServerId.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(), ServerId.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(), + ServerId.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(), ServerId.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(), ServerId.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..972aae09006 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 @@ -407,10 +407,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().getHostPort(), ke); ++suspendedCount; } else if (tm.hasCurrent()) { - hosted.put(tm.getLocation().getHostAndPort(), ke); + hosted.put(tm.getLocation().getServerInstance().getServer().getHostPort(), 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 b03abdb8a8e..28b4929514e 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.client.admin.servers.ServerId; 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, ServerId.tserver("localhost", 0), + ThriftService.TSERV); miniLock = new ServiceLock(zk, slp, UUID.randomUUID()); miniLock.lock(miniLockWatcher, sld); context.setServiceLock(miniLock); @@ -366,7 +366,7 @@ 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(ServerId.tserver(addr), 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..79a9765a769 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().getHostPort().toString()); } else if (!selectLocalTablets && !isLocal) { candidates.add(tabletMeta.getExtent()); - tabletLocations.put(tabletMeta.getExtent(), loc.getHostPort()); + tabletLocations.put(tabletMeta.getExtent(), + loc.getServerInstance().getServer().getHostPort().toString()); } } } 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)); From 0a358b3c1300d5c62983f7642d9a23ad7eeb054d Mon Sep 17 00:00:00 2001 From: Dave Marion Date: Thu, 11 Sep 2025 18:09:34 +0000 Subject: [PATCH 2/9] Update ITs --- .../metadata/schema/filters/GcWalsFilter.java | 2 +- .../manager/ManagerClientServiceHandler.java | 2 +- .../coordinator/commit/CommitCompaction.java | 5 +++-- .../coordinator/commit/PutGcCandidates.java | 5 +++-- .../coordinator/commit/RefreshTablet.java | 8 +++----- .../apache/accumulo/tserver/TabletServer.java | 2 +- .../accumulo/test/WaitForBalanceIT.java | 17 +++++++++-------- .../test/functional/GracefulShutdownIT.java | 13 ++++++------- .../TabletManagementIteratorIT.java | 6 ++++-- .../test/manager/SuspendedTabletsIT.java | 19 +++++++++---------- .../accumulo/test/shell/ShellServerIT.java | 4 +++- 11 files changed, 43 insertions(+), 40 deletions(-) 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 8523ab06869..d4ca6d85b7d 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 @@ -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::fromHostPortSessionString).collect(Collectors.toUnmodifiableSet()); } filter = tm -> !tm.getLogs().isEmpty() 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 fc3df3359c5..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 @@ -348,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. 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 0362489b78e..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().getServerInstance().toHostPortSessionString(); + 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/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java index 329f385fb65..1209d42039b 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 @@ -675,7 +675,7 @@ public void run() { Halt.halt(1, "Error informing Manager that we are shutting down, exiting!"); } else { iface.tabletServerStopping(TraceUtil.traceInfo(), getContext().rpcCreds(), - getTabletSession().toHostPortSessionString(), getResourceGroup().canonical()); + getTabletSession().serialize(), getResourceGroup().canonical()); } boolean managerDown = false; 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/functional/GracefulShutdownIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GracefulShutdownIT.java index 95aed9658ef..a2ecc4a4325 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,7 +38,6 @@ 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.client.admin.servers.ServerId.Type; import org.apache.accumulo.core.conf.ClientProperty; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Key; @@ -183,8 +182,8 @@ public void testGracefulShutdown() throws Exception { .getTabletServer((rg) -> rg.equals(ResourceGroupId.DEFAULT), AddressSelector.all(), true); assertEquals(2, tservers.size()); final ServiceLockPath tserverPath = tservers.iterator().next(); - Admin.signalGracefulShutdown(ctx, ServerId.dynamic(Type.valueOf(tserverPath.getType()), - tserverPath.getResourceGroup(), HostAndPort.fromString(tserverPath.getServer()))); + Admin.signalGracefulShutdown(ctx, + ServerId.tserver(HostAndPort.fromString(tserverPath.getServer()))); Wait.waitFor(() -> { control.refreshProcesses(ServerType.TABLET_SERVER); return control.getProcesses(ServerType.TABLET_SERVER).size() == 1; @@ -224,8 +223,8 @@ public void testGracefulShutdown() throws Exception { .getCompactor(ResourceGroupPredicate.exact(ResourceGroupId.of(GROUP_NAME)), AddressSelector.all(), true); final ServiceLockPath compactorPath = compactors.iterator().next(); - final ServerId compactorAddress = ServerId.dynamic(Type.valueOf(compactorPath.getType()), - compactorPath.getResourceGroup(), HostAndPort.fromString(compactorPath.getServer())); + final ServerId compactorAddress = ServerId.compactor(compactorPath.getResourceGroup(), + HostAndPort.fromString(compactorPath.getServer())); final CompactionConfig cc = new CompactionConfig(); final IteratorSetting is = new IteratorSetting(100, SlowIterator.class); @@ -278,8 +277,8 @@ public void testGracefulShutdown() throws Exception { assertNotNull(e); count++; if (count == 2) { - Admin.signalGracefulShutdown(ctx, ServerId.dynamic(Type.valueOf(sserverPath.getType()), - sserverPath.getResourceGroup(), HostAndPort.fromString(sserverPath.getServer()))); + Admin.signalGracefulShutdown(ctx, ServerId.sserver(sserverPath.getResourceGroup(), + HostAndPort.fromString(sserverPath.getServer()))); } } assertEquals(10, count); 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 e226687f27e..998e9001cf7 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 @@ -460,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(ServerId.tserver("fake", 9005), "1234567").serialize())); try (BatchWriter bw = client.createBatchWriter(table)) { bw.addMutation(m); } @@ -492,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(ServerId.tserver("fake", 9005), "1234567").serialize())); try (BatchWriter bw = client.createBatchWriter(table)) { bw.addMutation(m); } 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 972aae09006..f2df4112dd9 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 @@ -79,7 +79,6 @@ import com.google.common.collect.HashMultimap; import com.google.common.collect.SetMultimap; -import com.google.common.net.HostAndPort; public class SuspendedTabletsIT extends AccumuloClusterHarness { private static final Logger log = LoggerFactory.getLogger(SuspendedTabletsIT.class); @@ -244,11 +243,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 +265,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 +278,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 +287,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); @@ -358,8 +357,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 +406,10 @@ private void scan(ClientContext ctx, String tableName) { } locationStates.put(ke, tm); if (tm.getSuspend() != null) { - suspended.put(tm.getSuspend().server.getServer().getHostPort(), ke); + suspended.put(tm.getSuspend().server.getServer(), ke); ++suspendedCount; } else if (tm.hasCurrent()) { - hosted.put(tm.getLocation().getServerInstance().getServer().getHostPort(), 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/shell/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java index 19a590e3b54..cee175eaa81 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("/"); From dad2c162a9f20e9e6369f2116c33738feab9db6c Mon Sep 17 00:00:00 2001 From: Dave Marion Date: Fri, 12 Sep 2025 18:25:50 +0000 Subject: [PATCH 3/9] Add resource group to zk path and wal file name --- .../core/client/admin/servers/ServerId.java | 6 +-- .../core/clientImpl/TableOperationsImpl.java | 2 +- .../core/metadata/TServerInstance.java | 37 ++++++++------- .../metadata/schema/filters/GcWalsFilter.java | 4 +- .../core/tabletserver/log/LogEntry.java | 18 ++++--- .../metadata/schema/TabletMetadataTest.java | 14 +++--- .../core/tabletserver/log/LogEntryTest.java | 10 ++-- .../accumulo/server/log/WalStateManager.java | 21 +++++---- .../server/manager/LiveTServerSet.java | 32 +++---------- .../apache/accumulo/server/util/Admin.java | 2 +- .../accumulo/server/fs/VolumeUtilTest.java | 12 ++--- .../server/manager/LiveTServerSetTest.java | 12 ++--- .../manager/state/TabletManagementTest.java | 4 +- .../gc/GarbageCollectWriteAheadLogs.java | 1 + .../tableOps/merge/MergeTabletsTest.java | 2 +- .../tableOps/split/UpdateTabletsTest.java | 2 +- .../accumulo/tserver/AssignmentHandler.java | 4 +- .../accumulo/tserver/TabletClientHandler.java | 2 +- .../apache/accumulo/tserver/TabletServer.java | 47 +++++++++---------- .../tserver/tablet/ScanfileManager.java | 2 +- .../accumulo/tserver/tablet/Tablet.java | 14 +++--- .../accumulo/tserver/WalRemovalOrderTest.java | 3 +- .../tserver/log/RecoveryLogsIteratorTest.java | 2 +- .../tserver/log/SortedLogRecoveryTest.java | 2 +- .../commands/ListTabletsCommandTest.java | 4 +- .../test/functional/RegexGroupBalanceIT.java | 2 +- 26 files changed, 120 insertions(+), 141 deletions(-) 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 2a8dbadeb98..206fdb84790 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 @@ -155,9 +155,9 @@ public static ServerId dynamic(Type type, ResourceGroupId rgid, String host, int public static ServerId fromWalFileName(String name) { String parts[] = name.split("\\+"); - Preconditions.checkArgument(parts.length == 2, "Invalid server id in wal file: " + name); + Preconditions.checkArgument(parts.length == 3, "Invalid server id in wal file: " + name); // return an uncached tserver object - return ServerId.tserver(parts[0], Integer.parseInt(parts[1])); + return ServerId.tserver(ResourceGroupId.of(parts[0]), parts[1], Integer.parseInt(parts[2])); } public static final ServerId deserialize(String json) { @@ -249,7 +249,7 @@ public String toString() { } public String toWalFileName() { - return host + "+" + port; + return resourceGroup + "+" + host + "+" + port; } public String toHostPortString() { 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 0f406400549..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 @@ -1424,7 +1424,7 @@ private void waitForTableStateTransition(TableId tableId, TableState expectedSta lastRow = tablet.getExtent().toMetaRow(); if (loc != null) { - serverCounts.increment(loc.getServerInstance().toHostPortSessionString(), 1); + serverCounts.increment(loc.getServerInstance().toZooKeeperPathString(), 1); } } 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 4ad4e4a56fd..c57ea1dfb52 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 @@ -28,6 +28,7 @@ import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.client.admin.servers.ServerId.ServerIdInfo; import org.apache.accumulo.core.client.admin.servers.ServerId.Type; +import org.apache.accumulo.core.data.ResourceGroupId; import com.google.common.base.Preconditions; import com.google.common.net.HostAndPort; @@ -52,17 +53,14 @@ public static TServerInstance deserialize(String json) { return GSON.get().fromJson(json, TServerInstanceInfo.class).getTSI(); } - public static TServerInstance fromHostPortSessionString(String formattedString) { - int pos = formattedString.indexOf("["); - if (pos < 0 || !formattedString.endsWith("]")) { - // if no session, then use zero - var hostAndPort = HostAndPort.fromString(formattedString); - return new TServerInstance(ServerId.tserver(hostAndPort), Long.toHexString(0)); - } else { - var hostAndPort = HostAndPort.fromString(formattedString.substring(0, pos)); - var session = formattedString.substring(pos + 1, formattedString.length() - 1); - return new TServerInstance(ServerId.tserver(hostAndPort), session); - } + public static TServerInstance fromZooKeeperPathString(String zkPath) { + 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(ServerId.tserver(rgid, hostAndPort), session); } private final ServerId server; @@ -74,7 +72,7 @@ public TServerInstance(ServerId address, String session) { "ServerId type must be TABLET_SERVER"); this.server = address; this.session = session; - this.hostPortSession = server.getHostPort() + "[" + this.session + "]"; + setZooKeeperPathString(); } public TServerInstance(ServerId address, long session) { @@ -82,14 +80,19 @@ public TServerInstance(ServerId address, long session) { "ServerId type must be TABLET_SERVER"); this.server = address; this.session = Long.toHexString(session); - this.hostPortSession = server.getHostPort() + "[" + this.session + "]"; + setZooKeeperPathString(); } public TServerInstance(String json) { var partial = GSON.get().fromJson(json, TServerInstanceInfo.class).getTSI(); this.server = partial.server; this.session = partial.session; - this.hostPortSession = server.getHostPort() + "[" + this.session + "]"; + setZooKeeperPathString(); + } + + private void setZooKeeperPathString() { + this.hostPortSession = + server.getResourceGroup().canonical() + "+" + server.getHostPort() + "+" + this.session; } @Override @@ -117,13 +120,13 @@ public boolean equals(Object obj) { return false; } - public String toHostPortSessionString() { + public String toZooKeeperPathString() { return hostPortSession; } @Override public String toString() { - return toHostPortSessionString(); + return toZooKeeperPathString(); } public String getSession() { @@ -144,6 +147,6 @@ public String serialize() { private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); - this.hostPortSession = server.getHostPort() + "[" + this.session + "]"; + setZooKeeperPathString(); } } 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 d4ca6d85b7d..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::toHostPortSessionString).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::fromHostPortSessionString).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/tabletserver/log/LogEntry.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java index 7e4fa1be56f..0bc9fe460cc 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 @@ -32,16 +32,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 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; @@ -52,9 +51,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/type+group+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 @@ -76,13 +74,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 = ServerId.fromWalFileName(tserverPart).getHostPort(); + tserver = ServerId.fromWalFileName(tserverPart); } catch (IllegalArgumentException e) { throw new IllegalArgumentException(badTServerMsg); } @@ -122,7 +120,7 @@ public static LogEntry fromMetaWalEntry(Entry entry) { @NonNull @VisibleForTesting - HostAndPort getTServer() { + ServerId getTServer() { return tserver; } 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 4cf8f004310..350b0c81c1e 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 @@ -176,9 +176,9 @@ public void testAllColumns() { 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")); @@ -524,7 +524,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()); @@ -542,7 +542,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())); @@ -561,7 +561,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()); @@ -837,8 +837,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, 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..cb17e8c2d43 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.client.admin.servers.ServerId; 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(ServerId.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/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 eda598c4807..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 @@ -112,7 +112,7 @@ private String root() throws WalMarkerException { public void initWalMarker(TServerInstance tsi) throws WalMarkerException { byte[] data = new byte[0]; try { - zoo.putPersistentData(root() + "/" + tsi.toHostPortSessionString(), data, + zoo.putPersistentData(root() + "/" + tsi.toZooKeeperPathString(), data, NodeExistsPolicy.FAIL); } catch (KeeperException | InterruptedException e) { throw new WalMarkerException(e); @@ -132,9 +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.toHostPortSessionString() + "/" + 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); } @@ -154,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; @@ -189,7 +189,7 @@ public Map> getAllMarkers() throws WalMarkerException try { String path = root(); for (String child : zoo.getChildren(path)) { - TServerInstance inst = TServerInstance.fromHostPortSessionString(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 @@ -207,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); @@ -231,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); } @@ -241,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 9f14a581c11..ae71462e755 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 @@ -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; } } @@ -298,20 +295,16 @@ private synchronized void checkServer(final Set updates, log.trace("Lock exists for server: {}, adding to current set", tserverPath.getServer()); locklessServers.remove(tserverPath); ServerId server = sld.orElseThrow().getServer(ServiceLockData.ThriftService.TSERV); - ResourceGroupId resourceGroup = - sld.orElseThrow().getGroup(ServiceLockData.ThriftService.TSERV); TServerInstance instance = new TServerInstance(server, stat.getEphemeralOwner()); if (info == null) { updates.add(instance); - TServerInfo tServerInfo = - new TServerInfo(instance, new TServerConnection(server), 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(server), 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,7 +442,7 @@ public synchronized TServerInstance find(String tabletServer) { } static TServerInstance find(Map servers, String tabletServer) { - var target = TServerInstance.fromHostPortSessionString(tabletServer); + var target = TServerInstance.fromZooKeeperPathString(tabletServer); for (Entry entry : servers.entrySet()) { if (entry.getValue().instance.getServer().getHostPort() .equals(target.getServer().getHostPort())) { @@ -484,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/util/Admin.java b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java index dfc184b5edf..218069aeef1 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 @@ -785,7 +785,7 @@ private static void stopTabletServer(final ClientContext context, List s log.info("Stopping server {}", finalServer); ThriftClientTypes.MANAGER.executeVoid(context, client -> client.shutdownTabletServer(TraceUtil.traceInfo(), context.rpcCreds(), - finalServer.toHostPortSessionString(), force), + finalServer.toZooKeeperPathString(), force), ResourceGroupPredicate.DEFAULT_RG_ONLY); } } 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 c234238c53d..510c966f79c 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 @@ -25,7 +25,6 @@ import java.util.Map; import org.apache.accumulo.core.client.admin.servers.ServerId; -import org.apache.accumulo.core.data.ResourceGroupId; import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection; import org.apache.accumulo.server.manager.LiveTServerSet.TServerInfo; @@ -40,14 +39,13 @@ public void testSessionIds() { TServerConnection mockConn = EasyMock.createMock(TServerConnection.class); TServerInfo server1 = - new TServerInfo(new TServerInstance(ServerId.tserver("localhost", 1234), "5555"), mockConn, - ResourceGroupId.DEFAULT); + new TServerInfo(new TServerInstance(ServerId.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/TabletManagementTest.java b/server/base/src/test/java/org/apache/accumulo/server/manager/state/TabletManagementTest.java index a6caf7d3823..30296cb74ed 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 @@ -112,9 +112,9 @@ private SortedMap createMetadataEntryKV(KeyExtent extent) { mutation.at().family(LastLocationColumnFamily.NAME).qualifier("s000") .put(new TServerInstance(ServerId.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/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/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 7cfff852e47..931155f08f9 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 @@ -322,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 33e8ad39530..dab08e3e3bb 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 @@ -438,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/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/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 1209d42039b..9124e3f86b6 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; @@ -202,7 +203,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); @@ -479,7 +480,7 @@ public ServiceLock getLock() { return tabletServerLock; } - private void announceExistence() { + private long announceExistence() { final ZooReaderWriter zoo = getContext().getZooSession().asReaderWriter(); try { @@ -505,11 +506,11 @@ private void announceExistence() { 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); @@ -563,11 +564,12 @@ public void run() { metricsInfo.init(MetricsInfo.serviceTags(context.getInstanceName(), getApplicationName(), getAdvertiseAddress().getHostPort(), getResourceGroup())); - 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); @@ -675,7 +677,7 @@ public void run() { Halt.halt(1, "Error informing Manager that we are shutting down, exiting!"); } else { iface.tabletServerStopping(TraceUtil.traceInfo(), getContext().rpcCreds(), - getTabletSession().serialize(), getResourceGroup().canonical()); + getTServerInstance().serialize(), getResourceGroup().canonical()); } boolean managerDown = false; @@ -725,20 +727,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(), 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) { @@ -1019,7 +1014,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()); @@ -1034,7 +1029,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 { @@ -1047,7 +1042,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 @@ -1066,7 +1061,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()); } } @@ -1182,7 +1177,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/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/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 79800dc1dc5..086d7c920bb 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 @@ -152,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/functional/RegexGroupBalanceIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RegexGroupBalanceIT.java index e1dcdbc1fc3..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 @@ -197,7 +197,7 @@ private Table getCounts(AccumuloClient client, String group = group.substring(tableId.canonical().length() + 1).substring(0, 2); } String loc = - TServerInstance.deserialize(entry.getValue().toString()).toHostPortSessionString(); + TServerInstance.deserialize(entry.getValue().toString()).toZooKeeperPathString(); MutableInt count = groupLocationCounts.get(group, loc); if (count == null) { From 7f11006056c8b05386f131e246f5851c9765ee5d Mon Sep 17 00:00:00 2001 From: Dave Marion Date: Tue, 16 Sep 2025 18:54:14 +0000 Subject: [PATCH 4/9] Removed HostAndPort from ServerId public API --- core/pom.xml | 2 - .../core/client/admin/servers/ServerId.java | 36 +--------- .../core/clientImpl/ZookeeperLockChecker.java | 6 +- .../accumulo/core/logging/TabletLogger.java | 5 +- .../core/metadata/TServerInstance.java | 7 +- .../accumulo/core/metrics/MetricsInfo.java | 21 +++--- .../accumulo/core/rpc/TTimeoutTransport.java | 9 ++- .../apache/accumulo/core/rpc/ThriftUtil.java | 4 +- .../compaction/ExternalCompactionUtil.java | 8 +-- .../util/compaction/RunningCompaction.java | 6 +- .../core/lock/ServiceLockDataTest.java | 14 ++-- .../core/metadata/SuspendingTServerTest.java | 3 +- .../metadata/schema/TabletMetadataTest.java | 9 ++- ...onfigurableScanServerHostSelectorTest.java | 20 ++---- .../ConfigurableScanServerSelectorTest.java | 10 ++- .../accumulo/server/AbstractServer.java | 8 +-- .../server/manager/LiveTServerSet.java | 4 +- .../state/LoggingTabletStateStore.java | 7 +- .../apache/accumulo/server/util/Admin.java | 11 +-- .../accumulo/server/util/AdminTest.java | 3 +- .../apache/accumulo/compactor/Compactor.java | 8 ++- .../accumulo/gc/SimpleGarbageCollector.java | 4 +- .../org/apache/accumulo/manager/Manager.java | 4 +- .../coordinator/CompactionCoordinator.java | 4 +- .../manager/tserverOps/ShutdownTServer.java | 11 +-- .../manager/upgrade/Upgrader11to12.java | 2 +- .../org/apache/accumulo/monitor/Monitor.java | 2 +- .../monitor/rest/tables/TablesResource.java | 4 +- .../rest/tservers/TabletServerResource.java | 3 +- .../apache/accumulo/tserver/ScanServer.java | 8 ++- .../apache/accumulo/tserver/TabletServer.java | 8 ++- .../tserver/CheckTabletMetadataTest.java | 3 +- .../org/apache/accumulo/test/LocatorIT.java | 2 +- .../test/functional/GracefulShutdownIT.java | 10 +-- .../test/functional/LastLocationIT.java | 30 ++++++--- .../test/functional/ManagerAssignmentIT.java | 67 ++++++++++--------- .../test/functional/SplitRecoveryIT.java | 3 +- .../TabletManagementIteratorIT.java | 5 +- .../TabletResourceGroupBalanceIT.java | 15 ++--- .../test/functional/ZombieTServer.java | 9 +-- .../test/performance/NullTserver.java | 3 +- .../performance/scan/CollectTabletStats.java | 4 +- 42 files changed, 201 insertions(+), 201 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 20e593bd580..69dfa7d7d8c 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -285,8 +285,6 @@ org[.]apache[.]hadoop[.]mapred[.](FileAlreadyExistsException|InvalidJobConfException) org[.]apache[.]accumulo[.]access[.].* - - com[.]google[.]common[.]net[.]HostAndPort 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 206fdb84790..d0d4c672845 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 @@ -68,11 +68,6 @@ private static ServerId resolve(ServerIdInfo info) { return cache.get(info, k -> info.getServerId()); } - public static ServerId compactor(HostAndPort hp) { - return resolve(new ServerIdInfo(Type.COMPACTOR.name(), ResourceGroupId.DEFAULT.canonical(), - hp.getHost(), hp.getPort())); - } - public static ServerId compactor(String host, int port) { return resolve( new ServerIdInfo(Type.COMPACTOR.name(), ResourceGroupId.DEFAULT.canonical(), host, port)); @@ -82,11 +77,6 @@ public static ServerId compactor(ResourceGroupId rgid, String host, int port) { return resolve(new ServerIdInfo(Type.COMPACTOR.name(), rgid.canonical(), host, port)); } - public static ServerId compactor(ResourceGroupId rgid, HostAndPort hp) { - return resolve( - new ServerIdInfo(Type.COMPACTOR.name(), rgid.canonical(), hp.getHost(), hp.getPort())); - } - public static ServerId gc(String host, int port) { return resolve(new ServerIdInfo(Type.GARBAGE_COLLECTOR.name(), ResourceGroupId.DEFAULT.canonical(), host, port)); @@ -107,11 +97,6 @@ public static ServerId monitor(String host, int port) { new ServerIdInfo(Type.MONITOR.name(), ResourceGroupId.DEFAULT.canonical(), host, port)); } - public static ServerId sserver(HostAndPort hp) { - return resolve(new ServerIdInfo(Type.SCAN_SERVER.name(), ResourceGroupId.DEFAULT.canonical(), - hp.getHost(), hp.getPort())); - } - public static ServerId sserver(String host, int port) { return resolve( new ServerIdInfo(Type.SCAN_SERVER.name(), ResourceGroupId.DEFAULT.canonical(), host, port)); @@ -121,16 +106,6 @@ 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 sserver(ResourceGroupId rgid, HostAndPort hp) { - return resolve( - new ServerIdInfo(Type.SCAN_SERVER.name(), rgid.canonical(), hp.getHost(), hp.getPort())); - } - - public static ServerId tserver(HostAndPort hp) { - return resolve(new ServerIdInfo(Type.TABLET_SERVER.name(), ResourceGroupId.DEFAULT.canonical(), - hp.getHost(), hp.getPort())); - } - public static ServerId tserver(String host, int port) { return resolve(new ServerIdInfo(Type.TABLET_SERVER.name(), ResourceGroupId.DEFAULT.canonical(), host, port)); @@ -140,15 +115,6 @@ 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 tserver(ResourceGroupId rgid, HostAndPort hp) { - return resolve( - new ServerIdInfo(Type.TABLET_SERVER.name(), rgid.canonical(), hp.getHost(), hp.getPort())); - } - - public static ServerId dynamic(Type type, ResourceGroupId rgid, HostAndPort hp) { - return resolve(new ServerIdInfo(type.name(), rgid.canonical(), hp.getHost(), hp.getPort())); - } - public static ServerId dynamic(Type type, ResourceGroupId rgid, String host, int port) { return resolve(new ServerIdInfo(type.name(), rgid.canonical(), host, port)); } @@ -197,7 +163,7 @@ public int getPort() { return port; } - public synchronized HostAndPort getHostPort() { + private synchronized HostAndPort getHostPort() { if (hostPort == null) { hostPort = HostAndPort.fromParts(host, port); } 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 90ba31cc72c..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 @@ -32,6 +32,8 @@ import org.apache.accumulo.core.lock.ServiceLockPaths.ServiceLockPath; import org.apache.accumulo.core.zookeeper.ZooCache; +import com.google.common.net.HostAndPort; + public class ZookeeperLockChecker implements TabletServerLockChecker { private final ZooCache zc; @@ -46,7 +48,7 @@ public boolean doesTabletServerLockExist(ServerId server) { // ServiceLockPaths only returns items that have a lock Set tservers = lockPaths.getTabletServer(ResourceGroupPredicate.exact(server.getResourceGroup()), - AddressSelector.exact(server.getHostPort()), true); + AddressSelector.exact(HostAndPort.fromParts(server.getHost(), server.getPort())), true); return !tservers.isEmpty(); } @@ -55,7 +57,7 @@ public boolean isLockHeld(ServerId server, String session) { // ServiceLockPaths only returns items that have a lock Set tservers = lockPaths.getTabletServer(ResourceGroupPredicate.exact(server.getResourceGroup()), - AddressSelector.exact(server.getHostPort()), true); + 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/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/metadata/TServerInstance.java b/core/src/main/java/org/apache/accumulo/core/metadata/TServerInstance.java index c57ea1dfb52..3130a82eb6e 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 @@ -60,7 +60,8 @@ public static TServerInstance fromZooKeeperPathString(String zkPath) { var rgid = ResourceGroupId.of(parts[0]); var hostAndPort = HostAndPort.fromString(parts[1]); var session = parts[2]; - return new TServerInstance(ServerId.tserver(rgid, hostAndPort), session); + return new TServerInstance(ServerId.tserver(rgid, hostAndPort.getHost(), hostAndPort.getPort()), + session); } private final ServerId server; @@ -91,8 +92,8 @@ public TServerInstance(String json) { } private void setZooKeeperPathString() { - this.hostPortSession = - server.getResourceGroup().canonical() + "+" + server.getHostPort() + "+" + this.session; + this.hostPortSession = server.getResourceGroup().canonical() + "+" + server.toHostPortString() + + "+" + this.session; } @Override 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 2a013e3110a..a1a0e3996ad 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 @@ -228,7 +228,7 @@ public static TTransport createClientTransport(ServerId address, int timeout, // Make sure a timeout is set try { - transport = TTimeoutTransport.create(address.getHostPort(), timeout); + transport = TTimeoutTransport.create(address, timeout); } catch (TTransportException e) { log.warn("Failed to open transport to {}", address); throw e; @@ -302,7 +302,7 @@ public static TTransport createClientTransport(ServerId address, int timeout, transport.open(); } else { try { - transport = TTimeoutTransport.create(address.getHostPort(), timeout); + transport = TTimeoutTransport.create(address, timeout); } catch (TTransportException ex) { log.warn("Failed to open transport to {}", address); throw ex; 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 4393178ba4c..df500473855 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 @@ -120,10 +120,10 @@ 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(ServerId.compactor(slp.getResourceGroup(), - HostAndPort.fromString(slp.getServer()))); + .add(ServerId.compactor(slp.getResourceGroup(), hp.getHost(), hp.getPort())); }); return groupsAndAddresses; } @@ -222,8 +222,8 @@ public static List getCompactionsRunningOnCompactors(ClientCo try { TExternalCompactionJob job = rcf.getFuture().get(); if (null != job && null != job.getExternalCompactionId()) { - results.add( - new RunningCompaction(job, ServerId.compactor(rcf.getGroup(), rcf.getCompactor()))); + results.add(new RunningCompaction(job, ServerId.compactor(rcf.getGroup(), + rcf.getCompactor().getHost(), rcf.getCompactor().getPort()))); } } catch (InterruptedException | ExecutionException e) { throw new IllegalStateException(e); 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 8c65d6f43ec..6359a2f169e 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 @@ -51,8 +51,10 @@ public RunningCompaction(TExternalCompactionJob job, ServerId compactor) { } public RunningCompaction(TExternalCompaction tEC) { - this(tEC.getJob(), ServerId.compactor(ResourceGroupId.of(tEC.getGroupName()), - HostAndPort.fromString(tEC.getCompactor()))); + this(tEC.getJob(), + ServerId.compactor(ResourceGroupId.of(tEC.getGroupName()), + HostAndPort.fromString(tEC.getCompactor()).getHost(), + HostAndPort.fromString(tEC.getCompactor()).getPort())); } public Map getUpdates() { 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 dfb43d8f360..f0360efacfe 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 @@ -61,13 +61,14 @@ public void testMultipleServiceConstructor() throws Exception { ServiceLockData ss = new ServiceLockData(sds); assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV)); assertEquals("127.0.0.1:9997", ss.getServer(ThriftService.TSERV).toHostPortString()); - assertEquals(HostAndPort.fromString("127.0.0.1:9997"), - ss.getServer(ThriftService.TSERV).getHostPort()); + 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.getServer(ThriftService.TABLET_SCAN).toHostPortString()); assertEquals(HostAndPort.fromString("127.0.0.1:9998"), - ss.getServer(ThriftService.TABLET_SCAN).getHostPort()); + HostAndPort.fromParts(ss.getServer(ThriftService.TABLET_SCAN).getHost(), + ss.getServer(ThriftService.TABLET_SCAN).getPort())); assertEquals(ResourceGroupId.DEFAULT, ss.getGroup(ThriftService.TSERV)); } @@ -106,13 +107,14 @@ public void testMultipleServiceConstructorWithGroup() throws Exception { ServiceLockData ss = new ServiceLockData(sds); assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV)); assertEquals("127.0.0.1:9997", ss.getServer(ThriftService.TSERV).toHostPortString()); - assertEquals(HostAndPort.fromString("127.0.0.1:9997"), - ss.getServer(ThriftService.TSERV).getHostPort()); + 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.getServer(ThriftService.TABLET_SCAN).toHostPortString()); assertEquals(HostAndPort.fromString("127.0.0.1:9998"), - ss.getServer(ThriftService.TABLET_SCAN).getHostPort()); + 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.getServerUUID(ThriftService.COMPACTOR)); assertNull(ss.getServer(ThriftService.COMPACTOR)); 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 502fa48b02d..56a0ab04444 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 @@ -38,7 +38,8 @@ public void testToFromValue() { var val1 = new SuspendingTServer(ser1, suspensionTime).toValue(); var st1 = SuspendingTServer.fromValue(val1); - assertEquals(HostAndPort.fromParts("server1", 8555), st1.server.getServer().getHostPort()); + 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(new TServerInstance(ServerId.tserver("server1", 8555), "s001"), 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 350b0c81c1e..8133518d8df 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 @@ -256,13 +256,15 @@ public void testAllColumns() { tm.getLoaded()); allColumns.remove(LOADED); assertEquals(HostAndPort.fromParts("server1", 8555), - tm.getLocation().getServerInstance().getServer().getHostPort()); + HostAndPort.fromParts(tm.getLocation().getServerInstance().getServer().getHost(), + tm.getLocation().getServerInstance().getServer().getPort())); allColumns.remove(LOCATION); assertEquals("s001", tm.getLocation().getServerInstance().getSession()); assertEquals(LocationType.CURRENT, tm.getLocation().getType()); assertTrue(tm.hasCurrent()); assertEquals(HostAndPort.fromParts("server2", 8555), - tm.getLast().getServerInstance().getServer().getHostPort()); + 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()); @@ -308,7 +310,8 @@ public void testFuture() { assertEquals(extent, tm.getExtent()); assertEquals(HostAndPort.fromParts("server1", 8555), - tm.getLocation().getServerInstance().getServer().getHostPort()); + 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()); 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 34cce8ce58f..c72334456a2 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 @@ -39,8 +39,6 @@ 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 ServerId ss1 = ServerId.sserver("host1", 2000); @@ -76,7 +74,6 @@ public void test() { assertNotNull(selection); final ServerId firstServer = selection.getScanServer(tId); assertNotNull(firstServer); - final HostAndPort firstHP = firstServer.getHostPort(); final Set remainingServers = new HashSet<>(); if (host1Servers.contains(firstServer)) { @@ -98,8 +95,7 @@ public void test() { while (!remainingServers.isEmpty()) { selection = selector.selectServers(new SelectorParams(tId, Map.of(tId, attempts), Map.of())); ServerId selectedServer = selection.getScanServer(tId); - HostAndPort selectedHP = selectedServer.getHostPort(); - assertEquals(selectedHP.getHost(), firstHP.getHost()); + assertEquals(selectedServer.getHost(), firstServer.getHost()); assertTrue(remainingServers.remove(selectedServer)); attempts.add(new TestScanServerAttempt(selectedServer, Result.BUSY)); } @@ -107,8 +103,7 @@ public void test() { // 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())); ServerId secondServer = selection.getScanServer(tId); - final HostAndPort secondHP = secondServer.getHostPort(); - assertFalse(secondHP.getHost().equals(firstHP.getHost())); + assertFalse(secondServer.getHost().equals(firstServer.getHost())); if (host1Servers.contains(secondServer)) { if (firstHostSeen) { @@ -137,8 +132,7 @@ public void test() { while (!remainingServers.isEmpty()) { selection = selector.selectServers(new SelectorParams(tId, Map.of(tId, attempts), Map.of())); ServerId selectedServer = selection.getScanServer(tId); - HostAndPort selectedHP = selectedServer.getHostPort(); - assertEquals(selectedHP.getHost(), secondHP.getHost()); + assertEquals(selectedServer.getHost(), secondServer.getHost()); assertTrue(remainingServers.remove(selectedServer)); attempts.add(new TestScanServerAttempt(selectedServer, Result.BUSY)); } @@ -147,9 +141,8 @@ public void test() { // selected host selection = selector.selectServers(new SelectorParams(tId, Map.of(tId, attempts), Map.of())); ServerId thirdServer = selection.getScanServer(tId); - final HostAndPort thirdHP = thirdServer.getHostPort(); - assertFalse(thirdHP.getHost().equals(firstHP.getHost())); - assertFalse(thirdHP.getHost().equals(secondHP.getHost())); + assertFalse(thirdServer.getHost().equals(firstServer.getHost())); + assertFalse(thirdServer.getHost().equals(secondServer.getHost())); if (host1Servers.contains(thirdServer)) { if (firstHostSeen) { @@ -178,8 +171,7 @@ public void test() { while (!remainingServers.isEmpty()) { selection = selector.selectServers(new SelectorParams(tId, Map.of(tId, attempts), Map.of())); ServerId selectedServer = selection.getScanServer(tId); - HostAndPort selectedHP = selectedServer.getHostPort(); - assertEquals(selectedHP.getHost(), thirdHP.getHost()); + 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 fec1a0a893c..790e575509e 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 @@ -66,8 +66,11 @@ static class InitParams implements ScanServerSelector.InitParameters { InitParams(Set scanServers, Map opts) { this.opts = opts; var scanServersMap = new HashMap(); - scanServers.forEach(sserv -> scanServersMap - .put(ServerId.sserver(HostAndPort.fromString(sserv)), ResourceGroupId.DEFAULT)); + scanServers.forEach(sserv -> { + var hp = HostAndPort.fromString(sserv); + scanServersMap.put(ServerId.sserver(ResourceGroupId.DEFAULT, hp.getHost(), hp.getPort()), + ResourceGroupId.DEFAULT); + }); this.scanServers = () -> scanServersMap; } @@ -157,7 +160,8 @@ static class TestScanServerAttempt implements ScanServerAttempt { } TestScanServerAttempt(String server, Result result) { - this.server = ServerId.sserver(HostAndPort.fromString(server)); + var hp = HostAndPort.fromString(server); + this.server = ServerId.sserver(hp.getHost(), hp.getPort()); this.result = result; } 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 37c2c5d688a..e50a11a9468 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 @@ -120,8 +120,8 @@ protected AbstractServer(ServerId.Type serverType, ConfigOpts opts, if (advertHP.getHost().equals(ConfigOpts.BIND_ALL_ADDRESSES)) { throw new IllegalArgumentException("Advertise address cannot be 0.0.0.0"); } - advertiseAddress = - new AtomicReference<>(ServerId.dynamic(serverType, resourceGroup, advertHP)); + advertiseAddress = new AtomicReference<>( + ServerId.dynamic(serverType, resourceGroup, advertHP.getHost(), advertHP.getPort())); } else { advertiseAddress = new AtomicReference<>(); } @@ -361,8 +361,8 @@ protected void updateThriftServer(ThriftServerSupplier supplier, boolean start) log.info("Starting {} Thrift server, listening on {}", this.getClass().getSimpleName(), thriftServer.address); } - updateAdvertiseAddress( - ServerId.dynamic(this.serverType, this.resourceGroup, thriftServer.address)); + updateAdvertiseAddress(ServerId.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/manager/LiveTServerSet.java b/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java index ae71462e755..d1476789a5c 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 @@ -444,8 +444,8 @@ public synchronized TServerInstance find(String tabletServer) { static TServerInstance find(Map servers, String tabletServer) { var target = TServerInstance.fromZooKeeperPathString(tabletServer); for (Entry entry : servers.entrySet()) { - if (entry.getValue().instance.getServer().getHostPort() - .equals(target.getServer().getHostPort())) { + 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 (target.getSession() == null || target.getSession().equals(Long.toHexString(0)) || target.getSession().equals(entry.getValue().instance.getSession())) { 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 97988327029..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.getServerInstance().getServer().getHostPort(); + server = location.getServerInstance().getServer(); } TabletLogger.suspended(tm.getExtent(), server, suspensionTimestamp, logsForDeadServers.size()); 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 218069aeef1..9b1b137fff4 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 @@ -781,7 +781,7 @@ private static void stopTabletServer(final ClientContext context, List s } for (ServerId sid : runningServers) { final TServerInstance finalServer = - qualifyWithZooKeeperSessionId(context, zc, sid.getHostPort().toString()); + qualifyWithZooKeeperSessionId(context, zc, sid.toHostPortString()); log.info("Stopping server {}", finalServer); ThriftClientTypes.MANAGER.executeVoid(context, client -> client.shutdownTabletServer(TraceUtil.traceInfo(), context.rpcCreds(), @@ -804,14 +804,17 @@ static TServerInstance qualifyWithZooKeeperSessionId(ClientContext context, ZooC Set paths = context.getServerPaths() .getTabletServer(ResourceGroupPredicate.ANY, AddressSelector.exact(hpObj), true); if (paths.size() != 1) { - return new TServerInstance(ServerId.tserver(hpObj), Long.toHexString(0)); + return new TServerInstance(ServerId.tserver(hpObj.getHost(), hpObj.getPort()), + Long.toHexString(0)); } ServiceLockPath slp = paths.iterator().next(); long sessionId = ServiceLock.getSessionId(zooCache, slp); if (sessionId == 0) { - return new TServerInstance(ServerId.tserver(hpObj), Long.toHexString(0)); + return new TServerInstance(ServerId.tserver(hpObj.getHost(), hpObj.getPort()), + Long.toHexString(0)); } - return new TServerInstance(ServerId.tserver(hpObj), Long.toHexString(sessionId)); + return new TServerInstance(ServerId.tserver(hpObj.getHost(), hpObj.getPort()), + Long.toHexString(sessionId)); } private static final String ACCUMULO_SITE_BACKUP_FILE = "accumulo.properties.bak"; 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 d2d5e05057a..aedbdf19297 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 @@ -80,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(), - ServerId.tserver(HostAndPort.fromString(server)), ThriftService.TABLET_SCAN); + ServerId.tserver(hp.getHost(), hp.getPort()), ThriftService.TABLET_SCAN); String serverPath = group + "/" + server; String validZLockEphemeralNode = "zlock#" + UUID.randomUUID() + "#0000000000"; 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 5794fd5bc83..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 @@ -139,6 +139,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.net.HostAndPort; import io.micrometer.core.instrument.FunctionCounter; import io.micrometer.core.instrument.Gauge; @@ -368,8 +369,9 @@ protected void announceExistence(ServerId clientAddress) throws KeeperException, InterruptedException { final ZooReaderWriter zoo = getContext().getZooSession().asReaderWriter(); - final ServiceLockPath path = getContext().getServerPaths() - .createCompactorPath(getResourceGroup(), clientAddress.getHostPort()); + final ServiceLockPath path = + 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(), @@ -753,7 +755,7 @@ protected long getWaitTimeBetweenCompactionChecks(int numCompactors) { protected Collection getServiceTags(ServerId clientAddress) { return MetricsInfo.serviceTags(getContext().getInstanceName(), getApplicationName(), - clientAddress.getHostPort(), clientAddress.getResourceGroup()); + clientAddress); } private void performFailureProcessing(ConsecutiveErrorHistory errorHistory) 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 4adcf718ba4..e22e49c0670 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 @@ -184,8 +184,8 @@ public void run() { MetricsInfo metricsInfo = getContext().getMetricsInfo(); metricsInfo.addMetricsProducers(this, new GcMetrics(this)); - metricsInfo.init(MetricsInfo.serviceTags(getContext().getInstanceName(), getApplicationName(), - address.getHostPort(), address.getResourceGroup())); + metricsInfo.init( + MetricsInfo.serviceTags(getContext().getInstanceName(), getApplicationName(), address)); try { long delay = getStartDelay(); log.debug("Sleeping for {} milliseconds before beginning garbage collection cycles", delay); 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 9f733502579..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().getHostPort(), 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().getHostPort()); + 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 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 4152bfb1d33..d3adbedcbff 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 @@ -521,9 +521,9 @@ 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, - ServerId.compactor(groupId, HostAndPort.fromString(compactorAddress)))); + new RunningCompaction(result, ServerId.compactor(groupId, hp.getHost(), hp.getPort()))); TabletLogger.compacting(rcJob.getExtent(), rcJob.getSelectedFateId(), cid, compactorAddress, rcJob); break; 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 ade9db2b1dc..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 @@ -34,6 +34,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.net.HostAndPort; + public class ShutdownTServer extends ManagerRepo { private static final long serialVersionUID = 2L; @@ -90,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( - server.getServer().getResourceGroup(), server.getServer().getHostPort()); + var path = manager.getContext().getServerPaths() + .createTabletServerPath(server.getServer().getResourceGroup(), hp); ServiceLock.deleteLock(zoo, path); - path = manager.getContext().getServerPaths().createDeadTabletServerPath( - server.getServer().getResourceGroup(), server.getServer().getHostPort()); + 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 d90197a0058..2db870c836b 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 @@ -785,7 +785,7 @@ static void upgradeLocationCF(final Key key, final Value value, final Mutation m } catch (JsonSyntaxException e) { final String session = key.getColumnQualifier().toString(); final HostAndPort hp = HostAndPort.fromString(value.toString()); - final var tsi = new TServerInstance(ServerId.tserver(hp), session); + final var tsi = new TServerInstance(ServerId.tserver(hp.getHost(), hp.getPort()), session); switch (key.getColumnFamily().toString()) { case (CurrentLocationColumnFamily.STR_NAME): m.at().family(CurrentLocationColumnFamily.NAME).qualifier(session).put(tsi.serialize()); 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 6fd39c839cc..346497e56be 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 @@ -410,7 +410,7 @@ public void run() { MetricsInfo metricsInfo = getContext().getMetricsInfo(); metricsInfo.addMetricsProducers(this); metricsInfo.init(MetricsInfo.serviceTags(getContext().getInstanceName(), getApplicationName(), - monitorHostAndPort.getHostPort(), getResourceGroup())); + monitorHostAndPort)); // Needed to support the existing zk monitor address format if (!rootContext.endsWith("/")) { 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 1ff50b151e3..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.getServerInstance().getServer().getHostPort().toString()); + 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().getServerInstance().getServer().getHostPort().toString()); + 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 43877e098bf..2991935f9a0 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 @@ -178,7 +178,8 @@ public TabletServerSummary getTserverDetails( try { ClientContext context = monitor.getContext(); TabletServerClientService.Client client = - ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, ServerId.tserver(address), context); + ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, + ServerId.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/ScanServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java index c80e31e1f5a..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().getHostPort()); + 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); @@ -377,7 +379,7 @@ public void run() { metricsInfo.addMetricsProducers(this, scanMetrics, scanServerMetrics, blockCacheMetrics); metricsInfo.init(MetricsInfo.serviceTags(getContext().getInstanceName(), getApplicationName(), - getAdvertiseAddress().getHostPort(), 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/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java index 9124e3f86b6..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 @@ -155,6 +155,7 @@ import org.slf4j.LoggerFactory; import com.google.common.annotations.VisibleForTesting; +import com.google.common.net.HostAndPort; public class TabletServer extends AbstractServer implements TabletHostingServer { @@ -484,8 +485,9 @@ private long announceExistence() { final ZooReaderWriter zoo = getContext().getZooSession().asReaderWriter(); try { - final ServiceLockPath zLockPath = context.getServerPaths() - .createTabletServerPath(getResourceGroup(), getAdvertiseAddress().getHostPort()); + 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); @@ -562,7 +564,7 @@ public void run() { metricsInfo.addMetricsProducers(this, metrics, updateMetrics, scanMetrics, mincMetrics, pausedMetrics, blockCacheMetrics); metricsInfo.init(MetricsInfo.serviceTags(context.getInstanceName(), getApplicationName(), - getAdvertiseAddress().getHostPort(), getResourceGroup())); + getAdvertiseAddress())); final long lockSessionId = announceExistence(); getContext().setServiceLock(tabletServerLock); 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 7d329b77a20..841ddeca2e4 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 @@ -90,7 +90,8 @@ private static void assertFail(TreeMap tabletMeta, KeyExtent ke, TSer } private static ServerId csi(String location) { - return ServerId.tserver(HostAndPort.fromString(location)); + var hp = HostAndPort.fromString(location); + return ServerId.tserver(hp.getHost(), hp.getPort()); } @Test 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 504c8d9521a..021b2bfdbdf 100644 --- a/test/src/main/java/org/apache/accumulo/test/LocatorIT.java +++ b/test/src/main/java/org/apache/accumulo/test/LocatorIT.java @@ -98,7 +98,7 @@ public void testBasic() throws Exception { final Predicate hostedAndCurrentNotNull = t -> t.getTabletAvailability() == TabletAvailability.HOSTED && t.hasCurrent() - && t.getLocation().getServerInstance().getServer().getHostPort() != null; + && t.getLocation().getServerInstance().getServer().getHost() != null; try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) { String tableName = getUniqueNames(1)[0]; 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 a2ecc4a4325..d116836c6cf 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 @@ -182,8 +182,8 @@ public void testGracefulShutdown() throws Exception { .getTabletServer((rg) -> rg.equals(ResourceGroupId.DEFAULT), AddressSelector.all(), true); assertEquals(2, tservers.size()); final ServiceLockPath tserverPath = tservers.iterator().next(); - Admin.signalGracefulShutdown(ctx, - ServerId.tserver(HostAndPort.fromString(tserverPath.getServer()))); + final var hp = HostAndPort.fromString(tserverPath.getServer()); + Admin.signalGracefulShutdown(ctx, ServerId.tserver(hp.getHost(), hp.getPort())); Wait.waitFor(() -> { control.refreshProcesses(ServerType.TABLET_SERVER); return control.getProcesses(ServerType.TABLET_SERVER).size() == 1; @@ -223,8 +223,9 @@ public void testGracefulShutdown() throws Exception { .getCompactor(ResourceGroupPredicate.exact(ResourceGroupId.of(GROUP_NAME)), AddressSelector.all(), true); final ServiceLockPath compactorPath = compactors.iterator().next(); + final var compactorHP = HostAndPort.fromString(compactorPath.getServer()); final ServerId compactorAddress = ServerId.compactor(compactorPath.getResourceGroup(), - HostAndPort.fromString(compactorPath.getServer())); + compactorHP.getHost(), compactorHP.getPort()); final CompactionConfig cc = new CompactionConfig(); final IteratorSetting is = new IteratorSetting(100, SlowIterator.class); @@ -266,6 +267,7 @@ public void testGracefulShutdown() throws Exception { getCluster().getServerContext().getServerPaths().getScanServer( (rg) -> rg.equals(ResourceGroupId.of(GROUP_NAME)), AddressSelector.all(), true); 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); @@ -278,7 +280,7 @@ public void testGracefulShutdown() throws Exception { count++; if (count == 2) { Admin.signalGracefulShutdown(ctx, ServerId.sserver(sserverPath.getResourceGroup(), - HostAndPort.fromString(sserverPath.getServer()))); + 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 14b8c347ae4..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,8 +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().getServerInstance().getServer().getHostPort(), - newTablet.getLast().getServerInstance().getServer().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)) { @@ -71,10 +73,14 @@ public void test() throws Exception { // last location should not be set yet TabletMetadata unflushed = ManagerAssignmentIT.getTabletMetadata(c, tableId, null); - assertEquals(newTablet.getLocation().getServerInstance().getServer().getHostPort(), - unflushed.getLocation().getServerInstance().getServer().getHostPort()); - assertEquals(newTablet.getLocation().getServerInstance().getServer().getHostPort(), - unflushed.getLast().getServerInstance().getServer().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 @@ -82,16 +88,20 @@ public void test() throws Exception { TabletMetadata offline = ManagerAssignmentIT.getTabletMetadata(c, tableId, null); assertNull(offline.getLocation()); assertFalse(offline.hasCurrent()); - assertEquals(newTablet.getLocation().getServerInstance().getServer().getHostPort(), - offline.getLast().getServerInstance().getServer().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().getServerInstance().getServer().getHostPort(), - online.getLast().getServerInstance().getServer().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/ManagerAssignmentIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java index 2ba62ee658f..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,8 +157,10 @@ public void test() throws Exception { TabletMetadata flushed = getTabletMetadata(client, tableId, null); assertTrue(flushed.hasCurrent()); assertNotNull(flushed.getLocation()); - assertEquals(flushed.getLocation().getServerInstance().getServer().getHostPort(), - flushed.getLast().getServerInstance().getServer().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()); @@ -166,8 +169,10 @@ public void test() throws Exception { TabletMetadata offline = getTabletMetadata(client, tableId, null); assertFalse(offline.hasCurrent()); assertNull(offline.getLocation()); - assertEquals(flushed.getLocation().getServerInstance().getServer().getHostPort(), - offline.getLast().getServerInstance().getServer().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 @@ -175,8 +180,10 @@ public void test() throws Exception { TabletMetadata online = getTabletMetadata(client, tableId, null); assertTrue(online.hasCurrent()); assertNotNull(online.getLocation()); - assertEquals(online.getLocation().getServerInstance().getServer().getHostPort(), - online.getLast().getServerInstance().getServer().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 @@ -192,8 +199,10 @@ public void test() throws Exception { final TabletMetadata always = getTabletMetadata(client, tableId, null); assertTrue(hostedOrCurrentNotNull.test(always)); assertTrue(always.hasCurrent()); - assertEquals(flushed.getLocation().getServerInstance().getServer().getHostPort(), - always.getLast().getServerInstance().getServer().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 @@ -207,8 +216,10 @@ public void test() throws Exception { final TabletMetadata unhosted = getTabletMetadata(client, tableId, null); assertTrue(unhostedOrCurrentNull.test(unhosted)); assertNull(unhosted.getLocation()); - assertEquals(flushed.getLocation().getServerInstance().getServer().getHostPort(), - unhosted.getLast().getServerInstance().getServer().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 @@ -220,8 +231,10 @@ public void test() throws Exception { final TabletMetadata ondemand = getTabletMetadata(client, tableId, null); assertTrue(ondemandHosted.test(ondemand)); assertNull(ondemand.getLocation()); - assertEquals(flushed.getLocation().getServerInstance().getServer().getHostPort(), - ondemand.getLast().getServerInstance().getServer().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()); } @@ -527,22 +540,17 @@ public void testShutdownOnlyTServerWithUserTable() throws Exception { ((ClientContext) client).getTabletLocationCache(tid).binRanges((ClientContext) client, Collections.singletonList(TabletsSection.getRange()), binnedRanges); binnedRanges.keySet().forEach((location) -> { - HostAndPort address = location.getHostPort(); - 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); @@ -579,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 = location.getHostPort(); - 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/SplitRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java index eb69b0f37f1..902f76b76fd 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 @@ -211,8 +211,9 @@ private void splitPartiallyAndRecover(ServerContext context, KeyExtent extent, K highDatafileSizes, highDatafilesToRemove); SplitRecovery11to12.splitTablet(high, extent.prevEndRow(), splitRatio, context, Set.of()); + final var hp = HostAndPort.fromString(location); TServerInstance instance = - new TServerInstance(ServerId.tserver(HostAndPort.fromString(location)), zl.getSessionId()); + new TServerInstance(ServerId.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 998e9001cf7..120aecb0a84 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 @@ -699,10 +699,11 @@ 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(ServerId.tserver(tserver.getResourceGroup(), - HostAndPort.fromString(tserver.getServer())), sessionId)); + tservers.add(new TServerInstance( + ServerId.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 b8e82c96d55..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 @@ -153,7 +153,7 @@ public void testBalancerWithResourceGroups() throws Exception { assertEquals(26, locations.size()); Location l1 = locations.get(0).getLocation(); assertEquals(ResourceGroupId.DEFAULT, - tserverGroups.get(l1.getServerInstance().getServer().getHostPort().toString())); + tserverGroups.get(l1.getServerInstance().getServer().toHostPortString())); locations.forEach(loc -> assertEquals(l1, loc.getLocation())); // Check table names[1] @@ -163,7 +163,7 @@ public void testBalancerWithResourceGroups() throws Exception { assertEquals(26, locations.size()); Location l2 = locations.get(0).getLocation(); assertEquals(ResourceGroupId.of("GROUP1"), - tserverGroups.get(l2.getServerInstance().getServer().getHostPort().toString())); + tserverGroups.get(l2.getServerInstance().getServer().toHostPortString())); locations.forEach(loc -> assertEquals(l2, loc.getLocation())); client.tableOperations().delete(names[0]); @@ -296,7 +296,7 @@ public void testResourceGroupPropertyChange(AccumuloClient client, String tableN assertEquals(numExpectedSplits, locations.size()); Location l1 = locations.get(0).getLocation(); assertEquals(ResourceGroupId.DEFAULT, - tserverGroups.get(l1.getServerInstance().getServer().getHostPort().toString())); + tserverGroups.get(l1.getServerInstance().getServer().toHostPortString())); locations.forEach(loc -> assertEquals(l1, loc.getLocation())); // change the resource group property for the table @@ -307,15 +307,14 @@ 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().getServerInstance().getServer().getHostPort().toString()) + || (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.getServerInstance().getServer().getHostPort().toString()) - .equals(ResourceGroupId.of("GROUP1"))); + 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 89760c90428..e09b0d421ef 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 @@ -139,9 +139,11 @@ public static void main(String[] args) throws Exception { ServiceLock zlock = new ServiceLock(context.getZooSession(), zLockPath, UUID.randomUUID()); + ServerId serverId = + ServerId.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() { @@ -170,8 +172,7 @@ public void unableToMonitorLockNode(Exception e) { } }; - if (zlock.tryLock(lw, new ServiceLockData(UUID.randomUUID(), - ServerId.tserver(serverPort.address), ThriftService.TSERV))) { + 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/performance/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java index 28b4929514e..38ed0d6c92e 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 @@ -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(ServerId.tserver(addr), randomSessionID); + TServerInstance instance = + new TServerInstance(ServerId.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 79a9765a769..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 @@ -371,11 +371,11 @@ private static List findTablets(ClientContext context, boolean select if (selectLocalTablets && isLocal) { candidates.add(tabletMeta.getExtent()); tabletLocations.put(tabletMeta.getExtent(), - loc.getServerInstance().getServer().getHostPort().toString()); + loc.getServerInstance().getServer().toHostPortString()); } else if (!selectLocalTablets && !isLocal) { candidates.add(tabletMeta.getExtent()); tabletLocations.put(tabletMeta.getExtent(), - loc.getServerInstance().getServer().getHostPort().toString()); + loc.getServerInstance().getServer().toHostPortString()); } } } From fb6acbb9b61df3b175863ee5ad57cee2cc6cabc4 Mon Sep 17 00:00:00 2001 From: Dave Marion Date: Wed, 17 Sep 2025 15:11:52 +0000 Subject: [PATCH 5/9] Created ServerIdUtil --- .../core/client/admin/servers/ServerId.java | 93 +------------- .../core/clientImpl/ClientContext.java | 2 +- .../clientImpl/InstanceOperationsImpl.java | 2 +- .../core/clientImpl/ServerIdUtil.java | 116 ++++++++++++++++++ .../accumulo/core/lock/ServiceLockData.java | 6 +- .../manager/balancer/TabletServerIdImpl.java | 5 +- .../core/metadata/TServerInstance.java | 10 +- .../core/tabletserver/log/LogEntry.java | 3 +- .../compaction/ExternalCompactionUtil.java | 9 +- .../util/compaction/RunningCompaction.java | 3 +- .../clientImpl/ClientTabletCacheImplTest.java | 4 +- .../core/clientImpl/ScanAttemptsImplTest.java | 7 +- .../clientImpl/ThriftTransportKeyTest.java | 13 +- .../core/lock/ServiceLockDataTest.java | 18 +-- .../core/lock/ServiceLockPathsTest.java | 34 ++--- .../core/metadata/SuspendingTServerTest.java | 20 +-- .../metadata/schema/TabletMetadataTest.java | 22 ++-- ...onfigurableScanServerHostSelectorTest.java | 19 +-- .../ConfigurableScanServerSelectorTest.java | 28 +++-- .../core/tabletserver/log/LogEntryTest.java | 4 +- .../mapred/AccumuloRecordReader.java | 3 +- .../mapreduce/AccumuloRecordReader.java | 3 +- .../MiniAccumuloClusterImpl.java | 4 +- .../accumulo/server/AbstractServer.java | 5 +- .../apache/accumulo/server/util/Admin.java | 13 +- .../constraints/MetadataConstraintsTest.java | 4 +- .../server/manager/LiveTServerSetTest.java | 6 +- .../state/AbstractTabletStateStoreTest.java | 6 +- .../state/TabletManagementParametersTest.java | 6 +- .../manager/state/TabletManagementTest.java | 6 +- .../state/ZooTabletStateStoreTest.java | 5 +- .../accumulo/server/util/AdminTest.java | 8 +- .../gc/GarbageCollectWriteAheadLogsTest.java | 3 +- .../coordinator/CompactionCoordinator.java | 4 +- .../manager/upgrade/Upgrader11to12.java | 5 +- .../manager/TabletGroupWatcherTest.java | 26 ++-- .../compaction/CompactionCoordinatorTest.java | 3 +- .../manager/tableOps/ShutdownTServerTest.java | 4 +- .../tableOps/merge/MergeTabletsTest.java | 10 +- .../tableOps/split/UpdateTabletsTest.java | 10 +- .../org/apache/accumulo/monitor/Monitor.java | 5 +- .../monitor/next/SystemInformation.java | 2 +- .../rest/tservers/TabletServerResource.java | 4 +- .../tserver/log/TabletServerLogger.java | 3 +- .../tserver/CheckTabletMetadataTest.java | 3 +- .../commands/ListTabletsCommandTest.java | 6 +- .../accumulo/test/InstanceOperationsIT.java | 3 +- .../accumulo/test/TestDualAssignment.java | 6 +- .../usage/TabletFileUpdateIT_SimpleSuite.java | 9 +- .../apache/accumulo/test/fate/TestLock.java | 4 +- .../functional/AmpleConditionalWriterIT.java | 25 ++-- .../test/functional/GracefulShutdownIT.java | 7 +- .../test/functional/SplitRecoveryIT.java | 4 +- .../TabletManagementIteratorIT.java | 15 +-- .../test/functional/ZombieTServer.java | 3 +- .../accumulo/test/lock/ServiceLockIT.java | 46 +++---- .../test/performance/NullTserver.java | 10 +- 57 files changed, 383 insertions(+), 324 deletions(-) create mode 100644 core/src/main/java/org/apache/accumulo/core/clientImpl/ServerIdUtil.java 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 d0d4c672845..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 @@ -25,12 +25,11 @@ 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 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; import com.google.common.net.HostAndPort; @@ -41,8 +40,6 @@ */ public final class ServerId implements Comparable, Serializable { - private static final long serialVersionUID = 1L; - /** * Server process type names. * @@ -52,79 +49,7 @@ public enum Type { MANAGER, MINI, MONITOR, GARBAGE_COLLECTOR, COMPACTOR, SCAN_SERVER, TABLET_SERVER; } - public static record ServerIdInfo(String type, String resourceGroup, String host, int port) { - public ServerId getServerId() { - return new 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(); - - 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) { - String parts[] = name.split("\\+"); - Preconditions.checkArgument(parts.length == 3, "Invalid server id in wal file: " + name); - // return an uncached tserver object - return ServerId.tserver(ResourceGroupId.of(parts[0]), parts[1], Integer.parseInt(parts[2])); - } + private static final long serialVersionUID = 1L; public static final ServerId deserialize(String json) { return GSON.get().fromJson(json, ServerIdInfo.class).getServerId(); @@ -136,7 +61,7 @@ public static final ServerId deserialize(String json) { private final int port; private transient HostAndPort hostPort; - private ServerId(Type type, ResourceGroupId resourceGroup, String host, int port) { + public ServerId(Type type, ResourceGroupId resourceGroup, String host, int port) { super(); Preconditions.checkArgument(port == 0 || PortRange.VALID_RANGE.contains(port), "invalid server port value: " + port); @@ -214,20 +139,12 @@ public String toString() { + ", port= " + port + "]"; } - public String toWalFileName() { - return resourceGroup + "+" + host + "+" + port; - } - public String toHostPortString() { return getHostPort().toString(); } - public ServerIdInfo toServerIdInfo() { - return new ServerIdInfo(getType().name(), getResourceGroup().canonical(), getHost(), getPort()); - } - public String serialize() { - return GSON.get().toJson(toServerIdInfo()); + return GSON.get().toJson(ServerIdUtil.toServerIdInfo(this)); } private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { 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 54cef2b2aa8..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 @@ -229,7 +229,7 @@ public Supplier> getScanServers() { @Override public ServerId getServer() { HostAndPort hp = HostAndPort.fromString(entry.getServer()); - return ServerId.sserver(entry.getResourceGroup(), hp.getHost(), hp.getPort()); + return ServerIdUtil.sserver(entry.getResourceGroup(), hp.getHost(), hp.getPort()); } }).collect(Collectors.toSet()); } 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 3af0e21f6e2..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 @@ -636,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 ServerId.dynamic(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/ServerIdUtil.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ServerIdUtil.java new file mode 100644 index 00000000000..c647d0bdfa5 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ServerIdUtil.java @@ -0,0 +1,116 @@ +/* + * 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) { + String parts[] = name.split("\\+"); + Preconditions.checkArgument(parts.length == 3, "Invalid server id in wal file: " + name); + // 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/lock/ServiceLockData.java b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockData.java index 3a3d6af9fde..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 @@ -32,7 +32,7 @@ import java.util.stream.Collectors; import org.apache.accumulo.core.client.admin.servers.ServerId; -import org.apache.accumulo.core.client.admin.servers.ServerId.ServerIdInfo; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.data.ResourceGroupId; public class ServiceLockData implements Comparable { @@ -217,7 +217,7 @@ private static ServiceDescriptors deserialize(ServiceDescriptorsGson json) { private static class ServiceDescriptorGson { private UUID uuid; private ThriftService service; - private ServerIdInfo address; + private ServerIdUtil.ServerIdInfo address; // default constructor required for Gson @SuppressWarnings("unused") @@ -226,7 +226,7 @@ public ServiceDescriptorGson() {} public ServiceDescriptorGson(UUID uuid, ThriftService service, ServerId address) { this.uuid = uuid; this.service = service; - this.address = address.toServerIdInfo(); + this.address = ServerIdUtil.toServerIdInfo(address); } } 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 733c01fb6c9..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 @@ -21,6 +21,7 @@ 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; @@ -35,7 +36,7 @@ public static TabletServerIdImpl fromThrift(TServerInstance tsi) { } public TabletServerIdImpl(String host, int port, String session) { - this.tServerInstance = new TServerInstance(ServerId.tserver(host, port), session); + this.tServerInstance = new TServerInstance(ServerIdUtil.tserver(host, port), session); } public TabletServerIdImpl(ServerId server, String session) { @@ -85,7 +86,7 @@ public int hashCode() { @Override public String toString() { - return getServer().getHost() + ':' + getServer().getPort() + '[' + getSession() + ']'; + return getServer().toHostPortString() + '[' + getSession() + ']'; } public TServerInstance toThrift() { 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 3130a82eb6e..39090d0a75c 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 @@ -26,8 +26,8 @@ import java.util.Objects; import org.apache.accumulo.core.client.admin.servers.ServerId; -import org.apache.accumulo.core.client.admin.servers.ServerId.ServerIdInfo; 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; @@ -43,7 +43,7 @@ public class TServerInstance implements Comparable, Serializabl private static final long serialVersionUID = 1L; - public static record TServerInstanceInfo(ServerIdInfo server, String session) { + public static record TServerInstanceInfo(ServerIdUtil.ServerIdInfo server, String session) { public TServerInstance getTSI() { return new TServerInstance(server.getServerId(), session); } @@ -60,8 +60,8 @@ public static TServerInstance fromZooKeeperPathString(String zkPath) { var rgid = ResourceGroupId.of(parts[0]); var hostAndPort = HostAndPort.fromString(parts[1]); var session = parts[2]; - return new TServerInstance(ServerId.tserver(rgid, hostAndPort.getHost(), hostAndPort.getPort()), - session); + return new TServerInstance( + ServerIdUtil.tserver(rgid, hostAndPort.getHost(), hostAndPort.getPort()), session); } private final ServerId server; @@ -139,7 +139,7 @@ public ServerId getServer() { } public TServerInstanceInfo getTServerInstanceInfo() { - return new TServerInstanceInfo(server.toServerIdInfo(), session); + return new TServerInstanceInfo(ServerIdUtil.toServerIdInfo(server), session); } public String serialize() { 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 0bc9fe460cc..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 @@ -23,6 +23,7 @@ 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; @@ -80,7 +81,7 @@ private static LogEntry validatedLogEntry(String path, Text columnQualifier) { } ServerId tserver; try { - tserver = ServerId.fromWalFileName(tserverPart); + tserver = ServerIdUtil.fromWalFileName(tserverPart); } catch (IllegalArgumentException e) { throw new IllegalArgumentException(badTServerMsg); } 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 df500473855..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 @@ -36,6 +36,7 @@ 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; @@ -123,7 +124,7 @@ public static Map> getCompactorAddrs(ClientContext context) var hp = HostAndPort.fromString(slp.getServer()); groupsAndAddresses .computeIfAbsent(slp.getResourceGroup().canonical(), (k) -> new HashSet<>()) - .add(ServerId.compactor(slp.getResourceGroup(), hp.getHost(), hp.getPort())); + .add(ServerIdUtil.compactor(slp.getResourceGroup(), hp.getHost(), hp.getPort())); }); return groupsAndAddresses; } @@ -211,7 +212,7 @@ public static List getCompactionsRunningOnCompactors(ClientCo context.getServerPaths().getCompactor(ResourceGroupPredicate.ANY, AddressSelector.all(), true) .forEach(slp -> { var hp = HostAndPort.fromString(slp.getServer()); - var sid = ServerId.compactor(slp.getResourceGroup(), hp.getHost(), hp.getPort()); + var sid = ServerIdUtil.compactor(slp.getResourceGroup(), hp.getHost(), hp.getPort()); rcFutures.add(new RunningCompactionFuture(slp, executor.submit(() -> getRunningCompaction(sid, context)))); }); @@ -222,7 +223,7 @@ public static List getCompactionsRunningOnCompactors(ClientCo try { TExternalCompactionJob job = rcf.getFuture().get(); if (null != job && null != job.getExternalCompactionId()) { - results.add(new RunningCompaction(job, ServerId.compactor(rcf.getGroup(), + results.add(new RunningCompaction(job, ServerIdUtil.compactor(rcf.getGroup(), rcf.getCompactor().getHost(), rcf.getCompactor().getPort()))); } } catch (InterruptedException | ExecutionException e) { @@ -241,7 +242,7 @@ public static List getCompactionsRunningOnCompactors(ClientCo context.getServerPaths().getCompactor(ResourceGroupPredicate.ANY, AddressSelector.all(), true) .forEach(slp -> { var hp = HostAndPort.fromString(slp.getServer()); - var sid = ServerId.compactor(slp.getResourceGroup(), hp.getHost(), hp.getPort()); + var sid = ServerIdUtil.compactor(slp.getResourceGroup(), hp.getHost(), hp.getPort()); futures.add(executor.submit(() -> getRunningCompactionId(sid, context))); }); executor.shutdown(); 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 6359a2f169e..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 @@ -23,6 +23,7 @@ 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; @@ -52,7 +53,7 @@ public RunningCompaction(TExternalCompactionJob job, ServerId compactor) { public RunningCompaction(TExternalCompaction tEC) { this(tEC.getJob(), - ServerId.compactor(ResourceGroupId.of(tEC.getGroupName()), + ServerIdUtil.compactor(ResourceGroupId.of(tEC.getGroupName()), HostAndPort.fromString(tEC.getCompactor()).getHost(), HostAndPort.fromString(tEC.getCompactor()).getPort())); } 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 8ca7613a716..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 @@ -87,11 +87,11 @@ public class ClientTabletCacheImplTest { new KeyExtent(SystemTables.METADATA.tableId(), null, ROOT_TABLE_EXTENT.endRow()); static ServerId csi(String host, int port) { - return ServerId.tserver(host, port); + return ServerIdUtil.tserver(host, port); } static ServerId csi(String host) { - return ServerId.tserver(host, 0); + return ServerIdUtil.tserver(host, 0); } static KeyExtent createNewKeyExtent(String table, String endRow, String prevEndRow) { 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 fb9f9cc8a02..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 @@ -27,7 +27,6 @@ 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 org.apache.accumulo.core.spi.scan.ScanServerAttempt; import org.junit.jupiter.api.Test; @@ -58,7 +57,7 @@ public void testBasic() { var tablet1 = nti("1", "a"); - var reporter1 = sai.createReporter(ServerId.sserver("ss1", 1101), tablet1); + var reporter1 = sai.createReporter(ServerIdUtil.sserver("ss1", 1101), tablet1); reporter1.report(ScanServerAttempt.Result.BUSY); @@ -78,9 +77,9 @@ public void testBasic() { assertEquals(Map.of(tablet1, Set.of("ss1:1101_BUSY", "ss1:1101_ERROR")), simplify(snap3)); var tablet2 = nti("1", "m"); - var reporter2 = sai.createReporter(ServerId.sserver("ss1", 1101), tablet2); + var reporter2 = sai.createReporter(ServerIdUtil.sserver("ss1", 1101), tablet2); var tablet3 = nti("2", "r"); - var reporter3 = sai.createReporter(ServerId.sserver("ss2", 1102), tablet3); + var reporter3 = sai.createReporter(ServerIdUtil.sserver("ss2", 1102), tablet3); reporter2.report(ScanServerAttempt.Result.BUSY); reporter3.report(ScanServerAttempt.Result.ERROR); 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 a7401a7d4ba..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 @@ -30,7 +30,6 @@ import java.security.PrivilegedExceptionAction; import java.util.Properties; -import org.apache.accumulo.core.client.admin.servers.ServerId; import org.apache.accumulo.core.client.security.tokens.AuthenticationToken; import org.apache.accumulo.core.client.security.tokens.KerberosToken; import org.apache.accumulo.core.conf.ClientProperty; @@ -78,7 +77,7 @@ public void testSslAndSaslErrors() { try { assertThrows(RuntimeException.class, () -> new ThriftTransportKey(ThriftClientTypes.CLIENT, - ServerId.tserver("localhost", 9999), 120_000, clientCtx)); + ServerIdUtil.tserver("localhost", 9999), 120_000, clientCtx)); } finally { verify(clientCtx); } @@ -98,9 +97,9 @@ public void testConnectionCaching() throws IOException, InterruptedException { user1.doAs((PrivilegedExceptionAction) () -> createSaslParams(token)); ThriftTransportKey ttk1 = new ThriftTransportKey(ThriftClientTypes.CLIENT, - ServerId.tserver("localhost", 9997), 1L, null, saslParams1); + ServerIdUtil.tserver("localhost", 9997), 1L, null, saslParams1); ThriftTransportKey ttk2 = new ThriftTransportKey(ThriftClientTypes.CLIENT, - ServerId.tserver("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); @@ -119,9 +118,9 @@ public void testSaslPrincipalIsSignificant() throws IOException, InterruptedExce user2.doAs((PrivilegedExceptionAction) () -> createSaslParams(token)); ThriftTransportKey ttk1 = new ThriftTransportKey(ThriftClientTypes.CLIENT, - ServerId.tserver("localhost", 9997), 1L, null, saslParams1); + ServerIdUtil.tserver("localhost", 9997), 1L, null, saslParams1); ThriftTransportKey ttk2 = new ThriftTransportKey(ThriftClientTypes.CLIENT, - ServerId.tserver("localhost", 9997), 1L, null, saslParams2); + ServerIdUtil.tserver("localhost", 9997), 1L, null, saslParams2); assertNotEquals(ttk1, ttk2); assertNotEquals(ttk1.hashCode(), ttk2.hashCode()); @@ -137,7 +136,7 @@ public void testSimpleEquivalence() { replay(clientCtx); ThriftTransportKey ttk = new ThriftTransportKey(ThriftClientTypes.CLIENT, - ServerId.tserver("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 f0360efacfe..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 @@ -26,7 +26,7 @@ import java.util.Optional; 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.ResourceGroupId; import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptor; import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptors; @@ -41,8 +41,8 @@ public class ServiceLockDataTest { @Test public void testSingleServiceConstructor() throws Exception { - ServiceLockData ss = - new ServiceLockData(serverUUID, ServerId.tserver("127.0.0.1", 9997), ThriftService.TSERV); + 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.getServer(ThriftService.TSERV).getHost()); assertEquals(ResourceGroupId.DEFAULT, ss.getGroup(ThriftService.TSERV)); @@ -55,9 +55,9 @@ public void testSingleServiceConstructor() throws Exception { public void testMultipleServiceConstructor() throws Exception { ServiceDescriptors sds = new ServiceDescriptors(); sds.addService(new ServiceDescriptor(serverUUID, ThriftService.TSERV, - ServerId.tserver("127.0.0.1", 9997))); + ServerIdUtil.tserver("127.0.0.1", 9997))); sds.addService(new ServiceDescriptor(serverUUID, ThriftService.TABLET_SCAN, - ServerId.tserver("127.0.0.1", 9998))); + 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.getServer(ThriftService.TSERV).toHostPortString()); @@ -75,7 +75,7 @@ public void testMultipleServiceConstructor() throws Exception { @Test public void testSingleServiceConstructorWithGroup() throws Exception { ServiceLockData ss = new ServiceLockData(serverUUID, - ServerId.tserver(ResourceGroupId.of("meta"), "127.0.0.1", 9997), ThriftService.TSERV); + ServerIdUtil.tserver(ResourceGroupId.of("meta"), "127.0.0.1", 9997), ThriftService.TSERV); assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV)); assertEquals("127.0.0.1:9997", ss.getServer(ThriftService.TSERV).toHostPortString()); assertEquals(ResourceGroupId.of("meta"), ss.getGroup(ThriftService.TSERV)); @@ -87,7 +87,7 @@ public void testSingleServiceConstructorWithGroup() throws Exception { @Test public void testSingleServiceConstructor2WithGroup() throws Exception { ServiceLockData ss = new ServiceLockData(serverUUID, - ServerId.tserver(ResourceGroupId.of("meta"), "127.0.0.1", 9997), ThriftService.TSERV); + ServerIdUtil.tserver(ResourceGroupId.of("meta"), "127.0.0.1", 9997), ThriftService.TSERV); assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV)); assertEquals("127.0.0.1:9997", ss.getServer(ThriftService.TSERV).toHostPortString()); assertEquals(ResourceGroupId.of("meta"), ss.getGroup(ThriftService.TSERV)); @@ -101,9 +101,9 @@ public void testSingleServiceConstructor2WithGroup() throws Exception { public void testMultipleServiceConstructorWithGroup() throws Exception { ServiceDescriptors sds = new ServiceDescriptors(); sds.addService(new ServiceDescriptor(serverUUID, ThriftService.TSERV, - ServerId.tserver(ResourceGroupId.of("meta"), "127.0.0.1", 9997))); + ServerIdUtil.tserver(ResourceGroupId.of("meta"), "127.0.0.1", 9997))); sds.addService(new ServiceDescriptor(serverUUID, ThriftService.TABLET_SCAN, - ServerId.tserver(ResourceGroupId.of("ns1"), "127.0.0.1", 9998))); + 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.getServer(ThriftService.TSERV).toHostPortString()); 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 dde19a15d03..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,7 +44,7 @@ import java.util.Set; 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.ResourceGroupId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.lock.ServiceLockData.ThriftService; @@ -180,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, ServerId.gc("localhost", 1234), ThriftService.GC); + 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))) @@ -222,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, ServerId.manager("localhost", 9995), ThriftService.MANAGER); + 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))) @@ -264,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, ServerId.monitor("localhost", 9996), ThriftService.NONE); + 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))) @@ -300,10 +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, ServerId.compactor(TEST_RESOURCE_GROUP, "localhost", 9876), + 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); - var sld2 = - new ServiceLockData(uuid, ServerId.compactor("localhost", 9877), ThriftService.COMPACTOR); expect(zc.getChildren(ZCOMPACTORS)) .andReturn(List.of(TEST_RESOURCE_GROUP.canonical(), ResourceGroupId.DEFAULT.canonical())) @@ -446,10 +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, ServerId.sserver(TEST_RESOURCE_GROUP, "localhost", 9876), + 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); - var sld2 = - new ServiceLockData(uuid, ServerId.sserver("localhost", 9877), ThriftService.TABLET_SCAN); expect(zc.getChildren(ZSSERVERS)) .andReturn(List.of(TEST_RESOURCE_GROUP.canonical(), ResourceGroupId.DEFAULT.canonical())) @@ -585,10 +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, ServerId.tserver(TEST_RESOURCE_GROUP, "localhost", 9876), + 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); - var sld2 = - new ServiceLockData(uuid, ServerId.tserver("localhost", 9877), ThriftService.TABLET_SCAN); expect(zc.getChildren(ZTSERVERS)) .andReturn(List.of(TEST_RESOURCE_GROUP.canonical(), ResourceGroupId.DEFAULT.canonical())) @@ -724,10 +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, ServerId.tserver(TEST_RESOURCE_GROUP, "localhost", 9876), + 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); - var sld2 = - new ServiceLockData(uuid, ServerId.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 56a0ab04444..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,7 +24,7 @@ import java.util.List; import java.util.concurrent.TimeUnit; -import org.apache.accumulo.core.client.admin.servers.ServerId; +import org.apache.accumulo.core.clientImpl.ServerIdUtil; import org.apache.accumulo.core.util.time.SteadyTime; import org.junit.jupiter.api.Test; @@ -34,7 +34,7 @@ public class SuspendingTServerTest { @Test public void testToFromValue() { SteadyTime suspensionTime = SteadyTime.from(System.currentTimeMillis(), TimeUnit.MILLISECONDS); - TServerInstance ser1 = new TServerInstance(ServerId.tserver("server1", 8555), "s001"); + TServerInstance ser1 = new TServerInstance(ServerIdUtil.tserver("server1", 8555), "s001"); var val1 = new SuspendingTServer(ser1, suspensionTime).toValue(); var st1 = SuspendingTServer.fromValue(val1); @@ -42,8 +42,8 @@ public void testToFromValue() { HostAndPort.fromParts(st1.server.getServer().getHost(), st1.server.getServer().getPort())); assertEquals(suspensionTime, st1.suspensionTime); assertEquals(val1, st1.toValue()); - var st2 = new SuspendingTServer(new TServerInstance(ServerId.tserver("server1", 8555), "s001"), - 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()); @@ -51,14 +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(new TServerInstance(ServerId.tserver("server2", 8555), "s001"), - suspensionTime); - var st4 = new SuspendingTServer(new TServerInstance(ServerId.tserver("server1", 9555), "s001"), - 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(new TServerInstance(ServerId.tserver("server2", 8555), "s001"), - 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 8133518d8df..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,7 +81,7 @@ import org.apache.accumulo.core.client.admin.TabletAvailability; import org.apache.accumulo.core.client.admin.TimeType; -import org.apache.accumulo.core.client.admin.servers.ServerId; +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; @@ -135,8 +135,8 @@ public void testAllColumns() { FateId fateId1 = FateId.from(type, UUID.randomUUID()); FateId fateId2 = FateId.from(type, UUID.randomUUID()); - TServerInstance current = new TServerInstance(ServerId.tserver("server1", 8555), "s001"); - TServerInstance last = new TServerInstance(ServerId.tserver("server2", 8555), "s000"); + 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(), ""); @@ -210,7 +210,7 @@ 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(ServerId.tserver("localhost", 9997), 5000L); + TServerInstance tsi = new TServerInstance(ServerIdUtil.tserver("localhost", 9997), 5000L); MIGRATION_COLUMN.put(mutation, new Value(tsi.serialize())); @@ -299,7 +299,7 @@ public void testAllColumns() { public void testFuture() { KeyExtent extent = new KeyExtent(TableId.of("5"), new Text("df"), new Text("da")); - TServerInstance tsi = new TServerInstance(ServerId.tserver("server1", 8555), "s001"); + TServerInstance tsi = new TServerInstance(ServerIdUtil.tserver("server1", 8555), "s001"); Mutation mutation = TabletColumnFamily.createPrevRowMutation(extent); mutation.at().family(FutureLocationColumnFamily.NAME).qualifier("s001").put(tsi.serialize()); @@ -321,7 +321,7 @@ public void testFuture() { public void testFutureAndCurrent() { KeyExtent extent = new KeyExtent(TableId.of("5"), new Text("df"), new Text("da")); - TServerInstance tsi = new TServerInstance(ServerId.tserver("server1", 8555), "s001"); + TServerInstance tsi = new TServerInstance(ServerIdUtil.tserver("server1", 8555), "s001"); Mutation mutation = TabletColumnFamily.createPrevRowMutation(extent); mutation.at().family(CurrentLocationColumnFamily.NAME).qualifier("s001").put(tsi.serialize()); mutation.at().family(FutureLocationColumnFamily.NAME).qualifier("s001").put(tsi.serialize()); @@ -339,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(ServerId.tserver("server1", 8555), "s001"); - TServerInstance ser2 = new TServerInstance(ServerId.tserver("server2", 8111), "s002"); - TServerInstance deadSer = new TServerInstance(ServerId.tserver("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); @@ -745,7 +745,7 @@ private SortedMap toRowMap(Mutation mutation) { @Test public void testBuilder() { - TServerInstance ser1 = new TServerInstance(ServerId.tserver("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")); @@ -774,7 +774,7 @@ public void testBuilder() { FateId compactFateId1 = FateId.from(type, UUID.randomUUID()); FateId compactFateId2 = FateId.from(type, UUID.randomUUID()); - TServerInstance migration = new TServerInstance(ServerId.tserver("localhost", 9999), 1000L); + TServerInstance migration = new TServerInstance(ServerIdUtil.tserver("localhost", 9999), 1000L); TabletMetadata tm = TabletMetadata.builder(extent) .putTabletAvailability(TabletAvailability.UNHOSTED).putLocation(Location.future(ser1)) 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 c72334456a2..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 @@ -32,6 +32,7 @@ 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; @@ -41,15 +42,15 @@ public class ConfigurableScanServerHostSelectorTest { - private final ServerId ss1 = ServerId.sserver("host1", 2000); - private final ServerId ss2 = ServerId.sserver("host1", 2001); - private final ServerId ss3 = ServerId.sserver("host1", 2002); - private final ServerId ss4 = ServerId.sserver("host1", 2003); - private final ServerId ss5 = ServerId.sserver("host2", 2000); - private final ServerId ss6 = ServerId.sserver("host2", 2001); - private final ServerId ss7 = ServerId.sserver("host2", 2002); - private final ServerId ss8 = ServerId.sserver("host2", 2003); - private final ServerId ss9 = ServerId.sserver("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); 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 790e575509e..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 @@ -39,6 +39,7 @@ 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; @@ -68,7 +69,8 @@ static class InitParams implements ScanServerSelector.InitParameters { var scanServersMap = new HashMap(); scanServers.forEach(sserv -> { var hp = HostAndPort.fromString(sserv); - scanServersMap.put(ServerId.sserver(ResourceGroupId.DEFAULT, hp.getHost(), hp.getPort()), + scanServersMap.put( + ServerIdUtil.sserver(ResourceGroupId.DEFAULT, hp.getHost(), hp.getPort()), ResourceGroupId.DEFAULT); }); this.scanServers = () -> scanServersMap; @@ -161,7 +163,7 @@ static class TestScanServerAttempt implements ScanServerAttempt { TestScanServerAttempt(String server, Result result) { var hp = HostAndPort.fromString(server); - this.server = ServerId.sserver(hp.getHost(), hp.getPort()); + this.server = ServerIdUtil.sserver(hp.getHost(), hp.getPort()); this.result = result; } @@ -464,13 +466,13 @@ public void testGroups() { ConfigurableScanServerSelector selector = new ConfigurableScanServerSelector(); var dg = ResourceGroupId.DEFAULT; selector.init(new InitParams( - Map.of(ServerId.sserver("ss1", 1101), dg, ServerId.sserver("ss2", 1102), dg, - ServerId.sserver("ss3", 1103), dg, - ServerId.sserver(ResourceGroupId.of("g1"), "ss4", 1104), ResourceGroupId.of("g1"), - ServerId.sserver(ResourceGroupId.of("g1"), "ss5", 1105), ResourceGroupId.of("g1"), - ServerId.sserver(ResourceGroupId.of("g2"), "ss6", 1106), ResourceGroupId.of("g2"), - ServerId.sserver(ResourceGroupId.of("g2"), "ss7", 1107), ResourceGroupId.of("g2"), - ServerId.sserver(ResourceGroupId.of("g2"), "ss8", 1108), 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<>(); @@ -560,8 +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(ServerId.sserver("ss1", 1101), dg, ServerId.sserver("ss2", 1102), dg, - ServerId.sserver("ss3", 1103), dg)); + scanServers.set(Map.of(ServerIdUtil.sserver("ss1", 1101), dg, + ServerIdUtil.sserver("ss2", 1102), dg, ServerIdUtil.sserver("ss3", 1103), dg)); Optional optional = condition.get(); @@ -576,8 +578,8 @@ public Optional waitUntil(Supplier> condition, Duration maxWa ScanServerSelections actions = selector.selectServers(params); - assertTrue(Set.of(ServerId.sserver("ss1", 1101), ServerId.sserver("ss2", 1102), - ServerId.sserver("ss3", 1103)).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 cb17e8c2d43..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,7 +28,7 @@ import java.util.UUID; import java.util.stream.Stream; -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.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily; import org.apache.hadoop.io.Text; @@ -63,7 +63,7 @@ public void testFromMetadata() { private void verifyLogEntry(LogEntry logEntry, Text expectedColumnQualifier) { assertEquals(validPath, logEntry.toString()); assertEquals(validPath, logEntry.getPath()); - assertEquals(ServerId.fromWalFileName(validHost), logEntry.getTServer()); + assertEquals(ServerIdUtil.fromWalFileName(validHost), logEntry.getTServer()); assertEquals(expectedColumnQualifier, logEntry.getColumnQualifier()); assertEquals(validUUID, logEntry.getUniqueID()); } 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 e92cb702654..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 @@ -55,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; @@ -366,7 +367,7 @@ public static InputSplit[] getSplits(JobConf job, Class callingClass) throws tl.invalidateCache(); } } else { - final ServerId unhostedKey = ServerId.tserver("", 0); + final ServerId unhostedKey = ServerIdUtil.tserver("", 0); Map>> unhostedRanges = new HashMap<>(); unhostedRanges.put(unhostedKey, new HashMap<>()); BiConsumer consumer = (ct, r) -> { 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 af395ce8d84..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 @@ -55,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; @@ -399,7 +400,7 @@ public static List getSplits(JobContext context, Class callingCla tl.invalidateCache(); } } else { - final ServerId unhostedKey = ServerId.tserver("", 0); + final ServerId unhostedKey = ServerIdUtil.tserver("", 0); Map>> unhostedRanges = new HashMap<>(); unhostedRanges.put(unhostedKey, new HashMap<>()); BiConsumer consumer = (ct, r) -> { 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 6b72acdf092..7fce5abafb1 100644 --- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java +++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java @@ -66,9 +66,9 @@ import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.TableNotFoundException; -import org.apache.accumulo.core.client.admin.servers.ServerId; 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; @@ -722,7 +722,7 @@ public void failedToAcquireLock(Exception e) { throw new IllegalStateException("Error creating path in ZooKeeper", e); } ServiceLockData sld = - new ServiceLockData(miniUUID, ServerId.mini("localhost", 0), ThriftService.NONE); + 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 e50a11a9468..4c89511e6a1 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; @@ -121,7 +122,7 @@ protected AbstractServer(ServerId.Type serverType, ConfigOpts opts, throw new IllegalArgumentException("Advertise address cannot be 0.0.0.0"); } advertiseAddress = new AtomicReference<>( - ServerId.dynamic(serverType, resourceGroup, advertHP.getHost(), advertHP.getPort())); + ServerIdUtil.dynamic(serverType, resourceGroup, advertHP.getHost(), advertHP.getPort())); } else { advertiseAddress = new AtomicReference<>(); } @@ -361,7 +362,7 @@ protected void updateThriftServer(ThriftServerSupplier supplier, boolean start) log.info("Starting {} Thrift server, listening on {}", this.getClass().getSimpleName(), thriftServer.address); } - updateAdvertiseAddress(ServerId.dynamic(this.serverType, this.resourceGroup, + updateAdvertiseAddress(ServerIdUtil.dynamic(this.serverType, this.resourceGroup, thriftServer.address.getHost(), thriftServer.address.getPort())); } 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 47398eca133..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; @@ -804,16 +805,16 @@ static TServerInstance qualifyWithZooKeeperSessionId(ClientContext context, ZooC Set paths = context.getServerPaths() .getTabletServer(ResourceGroupPredicate.ANY, AddressSelector.exact(hpObj), true); if (paths.size() != 1) { - return new TServerInstance(ServerId.tserver(hpObj.getHost(), hpObj.getPort()), + return new TServerInstance(ServerIdUtil.tserver(hpObj.getHost(), hpObj.getPort()), Long.toHexString(0)); } ServiceLockPath slp = paths.iterator().next(); long sessionId = ServiceLock.getSessionId(zooCache, slp); if (sessionId == 0) { - return new TServerInstance(ServerId.tserver(hpObj.getHost(), hpObj.getPort()), + return new TServerInstance(ServerIdUtil.tserver(hpObj.getHost(), hpObj.getPort()), Long.toHexString(0)); } - return new TServerInstance(ServerId.tserver(hpObj.getHost(), hpObj.getPort()), + return new TServerInstance(ServerIdUtil.tserver(hpObj.getHost(), hpObj.getPort()), Long.toHexString(sessionId)); } @@ -1150,9 +1151,9 @@ private ServiceLock createAdminLock(ServerContext context) throws InterruptedExc ServiceLock adminLock = new ServiceLock(zk, slp, uuid); AdminLockWatcher lw = new AdminLockWatcher(); ServiceLockData.ServiceDescriptors descriptors = new ServiceLockData.ServiceDescriptors(); - descriptors - .addService(new ServiceLockData.ServiceDescriptor(uuid, ServiceLockData.ThriftService.NONE, - ServerId.dynamic(ServerId.Type.MANAGER, ResourceGroupId.DEFAULT, "admin_utility", 0))); + descriptors.addService(new ServiceLockData.ServiceDescriptor(uuid, + 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/test/java/org/apache/accumulo/server/constraints/MetadataConstraintsTest.java b/server/base/src/test/java/org/apache/accumulo/server/constraints/MetadataConstraintsTest.java index 9cb53927588..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 @@ -35,7 +35,7 @@ import java.util.concurrent.TimeUnit; 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.data.Key; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Range; @@ -158,7 +158,7 @@ public void testCheck() { public void testSuspensionCheck() throws Exception { Mutation m = new Mutation(new Text("0;foo")); MetadataConstraints mc = new MetadataConstraints(); - TServerInstance ser1 = new TServerInstance(ServerId.tserver("server1", 8555), "s001"); + TServerInstance ser1 = new TServerInstance(ServerIdUtil.tserver("server1", 8555), "s001"); SuspendLocationColumn.SUSPEND_COLUMN.put(m, new SuspendingTServer(ser1, SteadyTime.from(System.currentTimeMillis(), TimeUnit.MILLISECONDS)).toValue()); 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 510c966f79c..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,7 +24,7 @@ import java.util.HashMap; import java.util.Map; -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.server.manager.LiveTServerSet.TServerConnection; import org.apache.accumulo.server.manager.LiveTServerSet.TServerInfo; @@ -38,8 +38,8 @@ public void testSessionIds() { Map servers = new HashMap<>(); TServerConnection mockConn = EasyMock.createMock(TServerConnection.class); - TServerInfo server1 = - new TServerInfo(new TServerInstance(ServerId.tserver("localhost", 1234), "5555"), mockConn); + TServerInfo server1 = new TServerInfo( + new TServerInstance(ServerIdUtil.tserver("localhost", 1234), "5555"), mockConn); servers.put("server1", server1); assertEquals(server1.instance, LiveTServerSet.find(servers, "default+localhost:1234+0")); 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 48c23978562..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,7 +20,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows; -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.metadata.schema.Ample; import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location; @@ -32,10 +32,10 @@ public class AbstractTabletStateStoreTest { private Ample.TabletMutator tabletMutator; private final TServerInstance server1 = - new TServerInstance(ServerId.tserver("127.0.0.1", 10000), 0); + new TServerInstance(ServerIdUtil.tserver("127.0.0.1", 10000), 0); private final Location last1 = Location.last(server1); private final TServerInstance server2 = - new TServerInstance(ServerId.tserver("127.0.0.2", 10000), 1); + 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 25e558a1807..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,7 +24,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; -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.fate.FateId; @@ -46,8 +46,8 @@ public void testDeSer() { 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(ServerId.tserver("127.0.0.1", 10000), 0), - new TServerInstance(ServerId.tserver("127.0.0.1", 10001), 1)); + 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 30296cb74ed..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,7 +33,7 @@ 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.Key; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.TableId; @@ -107,10 +107,10 @@ private SortedMap createMetadataEntryKV(KeyExtent extent) { mutation.at().family(DataFileColumnFamily.NAME).qualifier(tf2.getMetadata()).put(dfv2.encode()); mutation.at().family(CurrentLocationColumnFamily.NAME).qualifier("s001") - .put(new TServerInstance(ServerId.tserver("server1", 8555), "s001").serialize()); + .put(new TServerInstance(ServerIdUtil.tserver("server1", 8555), "s001").serialize()); mutation.at().family(LastLocationColumnFamily.NAME).qualifier("s000") - .put(new TServerInstance(ServerId.tserver("server2", 8555), "s000").serialize()); + .put(new TServerInstance(ServerIdUtil.tserver("server2", 8555), "s000").serialize()); LogEntry le1 = LogEntry.fromPath("default+localhost+8020/" + UUID.randomUUID()); le1.addToMutation(mutation); 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 f45d0a1e31c..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,7 +23,7 @@ import java.util.List; -import org.apache.accumulo.core.client.admin.servers.ServerId; +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; @@ -50,7 +50,8 @@ public void testZooTabletStateStore() throws DistributedStoreException { ZooTabletStateStore tstore = new ZooTabletStateStore(DataLevel.ROOT, context); String sessionId = "this is my unique session data"; - TServerInstance server = new TServerInstance(ServerId.tserver("127.0.0.1", 10000), sessionId); + TServerInstance server = + 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 aedbdf19297..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 @@ -44,8 +44,8 @@ import java.util.stream.Collectors; import org.apache.accumulo.core.Constants; -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.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.fate.FateId; @@ -83,7 +83,7 @@ public void testQualifySessionId() throws KeeperException, InterruptedException var hp = HostAndPort.fromString(server); final long session = 123456789L; ServiceLockData sld1 = new ServiceLockData(UUID.randomUUID(), - ServerId.tserver(hp.getHost(), hp.getPort()), ThriftService.TABLET_SCAN); + ServerIdUtil.tserver(hp.getHost(), hp.getPort()), ThriftService.TABLET_SCAN); String serverPath = group + "/" + server; String validZLockEphemeralNode = "zlock#" + UUID.randomUUID() + "#0000000000"; @@ -103,7 +103,7 @@ public void testQualifySessionId() throws KeeperException, InterruptedException expect(ctx.getServerPaths()).andReturn(new ServiceLockPaths(zc)).anyTimes(); replay(ctx, zc); - assertEquals(new TServerInstance(ServerId.tserver("localhost", 12345), session), + assertEquals(new TServerInstance(ServerIdUtil.tserver("localhost", 12345), session), Admin.qualifyWithZooKeeperSessionId(ctx, zc, server)); verify(ctx, zc); @@ -125,7 +125,7 @@ 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(new TServerInstance(ServerId.tserver("localhost", 12345), "0"), + assertEquals(new TServerInstance(ServerIdUtil.tserver("localhost", 12345), "0"), Admin.qualifyWithZooKeeperSessionId(ctx, zc, server)); verify(ctx, zc); 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 08bbfb0c71f..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 @@ -32,6 +32,7 @@ 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; @@ -53,7 +54,7 @@ public class GarbageCollectWriteAheadLogsTest { - private final ServerId tserver = ServerId.tserver("localhost", 1234); + 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(); 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 d3adbedcbff..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; @@ -523,7 +524,8 @@ public TNextCompactionJob getCompactionJob(TInfo tinfo, TCredentials credentials // 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, ServerId.compactor(groupId, hp.getHost(), hp.getPort()))); + new RunningCompaction(result, + ServerIdUtil.compactor(groupId, hp.getHost(), hp.getPort()))); TabletLogger.compacting(rcJob.getExtent(), rcJob.getSelectedFateId(), cid, compactorAddress, rcJob); break; 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 2db870c836b..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 @@ -48,9 +48,9 @@ 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.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; @@ -785,7 +785,8 @@ static void upgradeLocationCF(final Key key, final Value value, final Mutation m } catch (JsonSyntaxException e) { final String session = key.getColumnQualifier().toString(); final HostAndPort hp = HostAndPort.fromString(value.toString()); - final var tsi = new TServerInstance(ServerId.tserver(hp.getHost(), hp.getPort()), session); + 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()); 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 265b4296f2a..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,7 +24,7 @@ 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.metadata.TServerInstance; import org.junit.jupiter.api.Test; @@ -33,22 +33,22 @@ public class TabletGroupWatcherTest { public void testFindingServer() { TreeMap servers = new TreeMap<>(); - servers.put(new TServerInstance(ServerId.tserver("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(ServerId.tserver("192.168.1.2", 9997), 70L), "tserver2"); - servers.put(new TServerInstance(ServerId.tserver("192.168.1.4", 9997), -90L), "tserver3"); + 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, ServerId.tserver("192.168.1.1", 9997))); - assertNull(findServerIgnoringSession(servers, ServerId.tserver("192.168.1.2", 9996))); - assertNull(findServerIgnoringSession(servers, ServerId.tserver("192.168.1.2", 9998))); - assertNull(findServerIgnoringSession(servers, ServerId.tserver("192.168.1.3", 9997))); - assertNull(findServerIgnoringSession(servers, ServerId.tserver("192.168.1.5", 9997))); + 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(ServerId.tserver("192.168.1.2", 9997), 50L), - findServerIgnoringSession(servers, ServerId.tserver("192.168.1.2", 9997))); - assertEquals(new TServerInstance(ServerId.tserver("192.168.1.4", 9997), -90L), - findServerIgnoringSession(servers, ServerId.tserver("192.168.1.4", 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 46eef4f6d91..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; @@ -96,7 +97,7 @@ public class CompactionCoordinatorTest { private static final ResourceGroupId GROUP_ID = ResourceGroupId.of("R2DQ"); - private final ServerId tserverAddr = ServerId.tserver(GROUP_ID, "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); 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 4ad2c186135..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.client.admin.servers.ServerId; +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; @@ -44,7 +44,7 @@ public class ShutdownTServerTest { @Test public void testSingleShutdown() throws Exception { final TServerInstance tserver = - new TServerInstance(ServerId.tserver("localhost", 1234), "fake"); + new TServerInstance(ServerIdUtil.tserver("localhost", 1234), "fake"); final boolean force = false; final ShutdownTServer op = new ShutdownTServer(tserver, force); 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 931155f08f9..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 @@ -59,7 +59,7 @@ import java.util.function.Consumer; 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.data.NamespaceId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; @@ -153,12 +153,12 @@ public void testManyColumns() throws Exception { var flushID = OptionalLong.of(40); var availability = TabletAvailability.HOSTED; var lastLocation = TabletMetadata.Location - .last(new TServerInstance(ServerId.tserver("1.2.3.4", 1234), "123456789")); + .last(new TServerInstance(ServerIdUtil.tserver("1.2.3.4", 1234), "123456789")); var suspendingTServer = - new SuspendingTServer(new TServerInstance(ServerId.tserver("1.2.3.4", 1025), ""), + 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(ServerId.tserver("localhost", 1234), 56L); + var migration = new TServerInstance(ServerIdUtil.tserver("localhost", 1234), 56L); var tablet1 = TabletMetadata.builder(ke1).putOperation(opid).putDirName("td1").putFile(file3, dfv3) @@ -310,7 +310,7 @@ public void testMisplacedMerge() throws Exception { @Test public void testUnexpectedColumns() { - var tserver = new TServerInstance(ServerId.tserver("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()); 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 dab08e3e3bb..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 @@ -40,7 +40,7 @@ import org.apache.accumulo.core.client.admin.TabletAvailability; import org.apache.accumulo.core.client.admin.TabletMergeability; -import org.apache.accumulo.core.client.admin.servers.ServerId; +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.dataImpl.KeyExtent; @@ -227,11 +227,11 @@ public void testManyColumns() throws Exception { var flushID = OptionalLong.of(40); var availability = TabletAvailability.HOSTED; var lastLocation = TabletMetadata.Location - .last(new TServerInstance(ServerId.tserver("1.2.3.4", 1234), "123456789")); + .last(new TServerInstance(ServerIdUtil.tserver("1.2.3.4", 1234), "123456789")); var suspendingTServer = - new SuspendingTServer(new TServerInstance(ServerId.tserver("1.2.3.4", 1025), ""), + new SuspendingTServer(new TServerInstance(ServerIdUtil.tserver("1.2.3.4", 1025), ""), SteadyTime.from(Duration.ofMillis(56))); - var migration = new TServerInstance(ServerId.tserver("localhost", 1234), 56L); + var migration = new TServerInstance(ServerIdUtil.tserver("localhost", 1234), 56L); String dir1 = "dir1"; String dir2 = "dir2"; @@ -417,7 +417,7 @@ public void testErrors() throws Exception { // Test splitting a tablet with a location var location = TabletMetadata.Location - .future(new TServerInstance(ServerId.tserver("1.2.3.4", 1234), 123456789L)); + .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)); 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 346497e56be..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; @@ -392,9 +393,9 @@ public void run() { throw new RuntimeException("Unable to get hostname for advertise address", e); } } - updateAdvertiseAddress(ServerId.monitor(advertiseHost, livePort)); + updateAdvertiseAddress(ServerIdUtil.monitor(advertiseHost, livePort)); } else { - updateAdvertiseAddress(ServerId.monitor(advertiseAddress.getHost(), livePort)); + updateAdvertiseAddress(ServerIdUtil.monitor(advertiseAddress.getHost(), livePort)); } ServerId monitorHostAndPort = getAdvertiseAddress(); log.debug("Using {} to advertise monitor location in ZooKeeper", monitorHostAndPort); 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/tservers/TabletServerResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java index 2991935f9a0..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 @@ -36,8 +36,8 @@ import jakarta.ws.rs.QueryParam; import jakarta.ws.rs.core.MediaType; -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.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo; @@ -179,7 +179,7 @@ public TabletServerSummary getTserverDetails( ClientContext context = monitor.getContext(); TabletServerClientService.Client client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, - ServerId.tserver(address.getHost(), address.getPort()), context); + 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/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java index e32a54f4d6b..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().toWalFileName()); + 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/test/java/org/apache/accumulo/tserver/CheckTabletMetadataTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/CheckTabletMetadataTest.java index 841ddeca2e4..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 @@ -28,6 +28,7 @@ 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; @@ -91,7 +92,7 @@ private static void assertFail(TreeMap tabletMeta, KeyExtent ke, TSer private static ServerId csi(String location) { var hp = HostAndPort.fromString(location); - return ServerId.tserver(hp.getHost(), hp.getPort()); + return ServerIdUtil.tserver(hp.getHost(), hp.getPort()); } @Test 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 086d7c920bb..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 @@ -37,8 +37,8 @@ import org.apache.accumulo.core.client.admin.InstanceOperations; 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.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; @@ -131,8 +131,8 @@ public void mockTest() throws Exception { TableId tableId = TableId.of("123"); - TServerInstance ser1 = new TServerInstance(ServerId.tserver("server1", 8555), "s001"); - TServerInstance ser2 = new TServerInstance(ServerId.tserver("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(); 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 dc50ccd46cb..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,6 +34,7 @@ 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.harness.AccumuloClusterHarness; import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; @@ -198,7 +199,7 @@ public void testPing() throws Exception { io.ping(sid); } - ServerId fake = ServerId.compactor("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/TestDualAssignment.java b/test/src/main/java/org/apache/accumulo/test/TestDualAssignment.java index 625dac6ead8..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,7 +30,7 @@ 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.client.admin.servers.ServerId; +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; @@ -78,9 +78,9 @@ public void test() throws Exception { var extent1 = new KeyExtent(tableId, new Text("m"), new Text("l")); var loc1 = TabletMetadata.Location - .current(new TServerInstance(ServerId.tserver("192.168.1.1", 9997), "56")); + .current(new TServerInstance(ServerIdUtil.tserver("192.168.1.1", 9997), "56")); var loc2 = TabletMetadata.Location - .future(new TServerInstance(ServerId.tserver("192.168.1.2", 9997), "67")); + .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/ample/usage/TabletFileUpdateIT_SimpleSuite.java b/test/src/main/java/org/apache/accumulo/test/ample/usage/TabletFileUpdateIT_SimpleSuite.java index e163d8e2043..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,7 +32,7 @@ import org.apache.accumulo.core.client.Accumulo; import org.apache.accumulo.core.client.admin.TimeType; -import org.apache.accumulo.core.client.admin.servers.ServerId; +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; @@ -70,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(ServerId.tserver("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); @@ -204,9 +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(ServerId.tserver("localhost", 9998), 0xabcdef123L)), Location - .current(new TServerInstance(ServerId.tserver("localhost", 9997), 0xabcdef124L)))) { + .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/fate/TestLock.java b/test/src/main/java/org/apache/accumulo/test/fate/TestLock.java index 69a71da1d30..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.client.admin.servers.ServerId; +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, ServerId.tserver("fake_test_host", 0))); + 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/AmpleConditionalWriterIT.java b/test/src/main/java/org/apache/accumulo/test/functional/AmpleConditionalWriterIT.java index 801594c91e8..0e4e95e5581 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,7 +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.client.admin.servers.ServerId; +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; @@ -163,8 +163,8 @@ public void setupTable() throws Exception { @Test public void testLocations() { - var ts1 = new TServerInstance(ServerId.tserver("localhost", 9997), 5000L); - var ts2 = new TServerInstance(ServerId.tserver("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(); @@ -322,8 +322,8 @@ public void testLocations() { @Test public void testFiles() { - var ts1 = new TServerInstance(ServerId.tserver("localhost", 9997), 5000L); - var ts2 = new TServerInstance(ServerId.tserver("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(); @@ -770,8 +770,8 @@ public static String createSelectedFilesJson(FateId fateId, boolean selAll, @Test public void testMultipleExtents() { - var ts1 = new TServerInstance(ServerId.tserver("localhost", 9997), 5000L); - var ts2 = new TServerInstance(ServerId.tserver("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(); @@ -1255,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(ServerId.tserver("server1", 8555), 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); @@ -1372,8 +1373,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(ServerId.tserver("localhost", 9997), 5000L); - var ts2 = new TServerInstance(ServerId.tserver("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() @@ -1910,8 +1911,8 @@ public void testMetadataCheck() { @Test public void testRequireMigration() { var context = getCluster().getServerContext(); - var tsi = new TServerInstance(ServerId.tserver("localhost", 1234), 56L); - var otherTsi = new TServerInstance(ServerId.tserver("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/GracefulShutdownIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GracefulShutdownIT.java index d116836c6cf..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; @@ -183,7 +184,7 @@ public void testGracefulShutdown() throws Exception { assertEquals(2, tservers.size()); final ServiceLockPath tserverPath = tservers.iterator().next(); final var hp = HostAndPort.fromString(tserverPath.getServer()); - Admin.signalGracefulShutdown(ctx, ServerId.tserver(hp.getHost(), hp.getPort())); + Admin.signalGracefulShutdown(ctx, ServerIdUtil.tserver(hp.getHost(), hp.getPort())); Wait.waitFor(() -> { control.refreshProcesses(ServerType.TABLET_SERVER); return control.getProcesses(ServerType.TABLET_SERVER).size() == 1; @@ -224,7 +225,7 @@ public void testGracefulShutdown() throws Exception { AddressSelector.all(), true); final ServiceLockPath compactorPath = compactors.iterator().next(); final var compactorHP = HostAndPort.fromString(compactorPath.getServer()); - final ServerId compactorAddress = ServerId.compactor(compactorPath.getResourceGroup(), + final ServerId compactorAddress = ServerIdUtil.compactor(compactorPath.getResourceGroup(), compactorHP.getHost(), compactorHP.getPort()); final CompactionConfig cc = new CompactionConfig(); @@ -279,7 +280,7 @@ public void testGracefulShutdown() throws Exception { assertNotNull(e); count++; if (count == 2) { - Admin.signalGracefulShutdown(ctx, ServerId.sserver(sserverPath.getResourceGroup(), + Admin.signalGracefulShutdown(ctx, ServerIdUtil.sserver(sserverPath.getResourceGroup(), sserverHP.getHost(), sserverHP.getPort())); } } 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 902f76b76fd..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 @@ -38,8 +38,8 @@ import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.client.admin.TimeType; -import org.apache.accumulo.core.client.admin.servers.ServerId; 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; @@ -213,7 +213,7 @@ private void splitPartiallyAndRecover(ServerContext context, KeyExtent extent, K SplitRecovery11to12.splitTablet(high, extent.prevEndRow(), splitRatio, context, Set.of()); final var hp = HostAndPort.fromString(location); TServerInstance instance = - new TServerInstance(ServerId.tserver(hp.getHost(), hp.getPort()), zl.getSessionId()); + 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 120aecb0a84..3963e2814a9 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 @@ -58,8 +58,8 @@ import org.apache.accumulo.core.client.TableNotFoundException; 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.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; @@ -372,7 +372,7 @@ public void test(String compressionType) throws Exception { KeyExtent nonExistantTable = new KeyExtent(badTableId, null, null); TabletMutator tm = mut.mutateTablet(nonExistantTable); tm.putLocation(Location - .current(new TServerInstance(ServerId.tserver("fakeServer", 0), "fakeSession"))); + .current(new TServerInstance(ServerIdUtil.tserver("fakeServer", 0), "fakeSession"))); tm.automaticallyPutServerLock(false); tm.mutate(); } @@ -393,7 +393,7 @@ public void test(String compressionType) throws Exception { KeyExtent nonExistantTable = new KeyExtent(badTableId, null, null); TabletMutator tm = mut.mutateTablet(nonExistantTable); tm.deleteLocation(Location - .current(new TServerInstance(ServerId.tserver("fakeServer", 0), "fakeSession"))); + .current(new TServerInstance(ServerIdUtil.tserver("fakeServer", 0), "fakeSession"))); tm.automaticallyPutServerLock(false); tm.mutate(); } @@ -461,7 +461,7 @@ private void addDuplicateLocation(AccumuloClient client, String table, String ta 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(new TServerInstance(ServerId.tserver("fake", 9005), "1234567").serialize())); + new Value(new TServerInstance(ServerIdUtil.tserver("fake", 9005), "1234567").serialize())); try (BatchWriter bw = client.createBatchWriter(table)) { bw.addMutation(m); } @@ -493,8 +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(new TServerInstance(ServerId.tserver("fake", 9005), "1234567").serialize())); + entry.getKey().getTimestamp() + 1, new Value( + new TServerInstance(ServerIdUtil.tserver("fake", 9005), "1234567").serialize())); try (BatchWriter bw = client.createBatchWriter(table)) { bw.addMutation(m); } @@ -703,7 +703,8 @@ private static TabletManagementParameters createParameters(AccumuloClient client try { long sessionId = ServiceLock.getSessionId(context.getZooCache(), tserver); tservers.add(new TServerInstance( - ServerId.tserver(tserver.getResourceGroup(), hp.getHost(), hp.getPort()), sessionId)); + 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/ZombieTServer.java b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java index f1fb52d2bca..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 @@ -26,6 +26,7 @@ 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; @@ -140,7 +141,7 @@ public static void main(String[] args) throws Exception { ServiceLock zlock = new ServiceLock(context.getZooSession(), zLockPath, UUID.randomUUID()); ServerId serverId = - ServerId.tserver(serverPort.address.getHost(), serverPort.address.getPort()); + ServerIdUtil.tserver(serverPort.address.getHost(), serverPort.address.getPort()); MetricsInfo metricsInfo = context.getMetricsInfo(); metricsInfo.init(MetricsInfo.serviceTags(context.getInstanceName(), "zombie.server", serverId)); 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 73bc41e49c2..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.client.admin.servers.ServerId; +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(), ServerId.tserver("test1", 0), ThriftService.TSERV)); + 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(), ServerId.tserver("test1", 0), ThriftService.TSERV)); + 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(), ServerId.tserver("test1", 0), ThriftService.TSERV)); + 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(), ServerId.tserver("test1", 0), ThriftService.TSERV)); + 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(), ServerId.tserver("test2", 0), ThriftService.TSERV)); + 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(), ServerId.tserver("test3", 0), ThriftService.TSERV)); + zl3.lock(lw3, new ServiceLockData(UUID.randomUUID(), ServerIdUtil.tserver("test3", 0), + ThriftService.TSERV)); List children = ServiceLock.validateAndSort(parent, zrw.getChildren(parent.toString())); @@ -387,7 +387,7 @@ public void testUnexpectedEvent() throws Exception { TestALW lw = new TestALW(); - zl.lock(lw, new ServiceLockData(UUID.randomUUID(), ServerId.tserver("test1", 0), + zl.lock(lw, new ServiceLockData(UUID.randomUUID(), ServerIdUtil.tserver("test1", 0), ThriftService.TSERV)); lw.waitForChanges(1); @@ -427,7 +427,7 @@ 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(), ServerId.tserver("test1", 0), + 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. @@ -443,7 +443,7 @@ 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(), ServerId.tserver("test1", 0), + 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. @@ -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(), ServerId.tserver("test1", 0), - ThriftService.TSERV)); // 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(); @@ -647,7 +647,7 @@ public void testTryLock() throws Exception { TestALW lw = new TestALW(); boolean ret = zl.tryLock(lw, new ServiceLockData(UUID.randomUUID(), - ServerId.tserver("test1", 0), ThriftService.TSERV)); + 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(), ServerId.tserver("test1", 0), ThriftService.TSERV); + 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(), ServerId.tserver("test2", 0), ThriftService.TSERV); + 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/performance/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java index 4fe4eabf8e6..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,7 +30,7 @@ import org.apache.accumulo.core.cli.ConfigOpts; import org.apache.accumulo.core.cli.Help; -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.DefaultConfiguration; @@ -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, ServerId.tserver("localhost", 0), - ThriftService.TSERV); + 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,8 +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(ServerId.tserver(addr.getHost(), addr.getPort()), randomSessionID); + TServerInstance instance = new TServerInstance( + ServerIdUtil.tserver(addr.getHost(), addr.getPort()), randomSessionID); var s = tablets.iterator(); while (s.hasNext()) { From e342779a93addd4be369f5f111fe62457771c68e Mon Sep 17 00:00:00 2001 From: Dave Marion Date: Wed, 17 Sep 2025 16:56:26 +0000 Subject: [PATCH 6/9] Fixed some ITs, handle older server wal file name serializations --- .../accumulo/core/clientImpl/ServerIdUtil.java | 15 ++++++++++++--- .../accumulo/core/metadata/TServerInstance.java | 6 ++++++ .../test/MissingWalHeaderCompletesRecoveryIT.java | 12 ++++++------ .../test/fate/ManagerRepoIT_SimpleSuite.java | 2 +- .../test/functional/AmpleConditionalWriterIT.java | 13 +++++++------ .../accumulo/test/functional/ManagerApiIT.java | 4 ++-- .../functional/TabletManagementIteratorIT.java | 2 +- 7 files changed, 35 insertions(+), 19 deletions(-) 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 index c647d0bdfa5..56ca253ee93 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ServerIdUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ServerIdUtil.java @@ -103,10 +103,19 @@ public static ServerId dynamic(Type type, ResourceGroupId rgid, String host, int } 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 == 3, "Invalid server id in wal file: " + name); - // return an uncached tserver object - return tserver(ResourceGroupId.of(parts[0]), parts[1], Integer.parseInt(parts[2])); + 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) { 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 39090d0a75c..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 @@ -54,6 +54,12 @@ public static TServerInstance deserialize(String json) { } 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); 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/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/functional/AmpleConditionalWriterIT.java b/test/src/main/java/org/apache/accumulo/test/functional/AmpleConditionalWriterIT.java index 0e4e95e5581..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 @@ -473,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); @@ -491,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); @@ -504,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 @@ -1234,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); @@ -1341,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); @@ -1352,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); 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..002ec2289b6 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 @@ -246,7 +246,7 @@ 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"; + String fakeHostAndPort = "default+" + getUniqueNames(1)[0] + ":0+1234"; op = user -> client -> { client.shutdownTabletServer(TraceUtil.traceInfo(), user.toThrift(instanceId), fakeHostAndPort, false); @@ -261,7 +261,7 @@ public void testPermissions_shutdownTabletServer() throws Exception { public void shutdownTabletServer() throws Exception { op = user -> client -> { client.shutdownTabletServer(TraceUtil.traceInfo(), user.toThrift(instanceId), - "fakeTabletServer:9997", true); + "default+fakeTabletServer:9997+1234", true); return null; }; try (AccumuloClient client = Accumulo.newClient().from(getClientProps()) 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 3963e2814a9..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 @@ -678,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)) { From eb9f0dfa65998db69b123c30edf35bfb204905d8 Mon Sep 17 00:00:00 2001 From: Dave Marion Date: Thu, 18 Sep 2025 21:25:26 +0000 Subject: [PATCH 7/9] Fixed SuspendedTabletsIT --- .../test/manager/SuspendedTabletsIT.java | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) 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 f2df4112dd9..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; @@ -79,6 +81,7 @@ import com.google.common.collect.HashMultimap; import com.google.common.collect.SetMultimap; +import com.google.common.net.HostAndPort; public class SuspendedTabletsIT extends AccumuloClusterHarness { private static final Logger log = LoggerFactory.getLogger(SuspendedTabletsIT.class); @@ -88,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 @@ -123,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")); } }); @@ -309,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); From 20ad0e639d2c5adf4eceaea96cbaffd94ad5cd19 Mon Sep 17 00:00:00 2001 From: Dave Marion Date: Fri, 19 Sep 2025 12:43:18 +0000 Subject: [PATCH 8/9] Fix AdvertiseAndBindIT --- .../accumulo/server/AbstractServer.java | 5 ++- .../test/functional/AdvertiseAndBindIT.java | 45 +++++++++---------- 2 files changed, 23 insertions(+), 27 deletions(-) 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 4c89511e6a1..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 @@ -117,7 +117,7 @@ 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"); } @@ -339,7 +339,8 @@ protected void updateAdvertiseAddress(ServerId thriftBindAddress) { if (curr == null) { return thriftBindAddress; } else if (curr.getPort() == 0) { - return thriftBindAddress; + return ServerIdUtil.dynamic(curr.getType(), curr.getResourceGroup(), curr.getHost(), + thriftBindAddress.getPort()); } else { return curr; } 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; } From 007b2dd7447c870b1971dc8485d3f6b666e02e1b Mon Sep 17 00:00:00 2001 From: Dave Marion Date: Fri, 19 Sep 2025 14:16:08 +0000 Subject: [PATCH 9/9] Update ManagerApiIT to make clear TServerInstance is expected --- .../accumulo/test/functional/ManagerApiIT.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) 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 002ec2289b6..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 = "default+" + getUniqueNames(1)[0] + ":0+1234"; + 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), - "default+fakeTabletServer:9997+1234", true); + inst.toZooKeeperPathString(), true); return null; }; try (AccumuloClient client = Accumulo.newClient().from(getClientProps())