diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/CheckAccumuloProperties.java b/server/base/src/main/java/org/apache/accumulo/server/conf/CheckAccumuloProperties.java index 08684a52085..682fd31e594 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/conf/CheckAccumuloProperties.java +++ b/server/base/src/main/java/org/apache/accumulo/server/conf/CheckAccumuloProperties.java @@ -24,6 +24,7 @@ import org.apache.accumulo.core.conf.SiteConfiguration; import org.apache.accumulo.server.ServerDirs; import org.apache.accumulo.server.fs.VolumeManagerImpl; +import org.apache.accumulo.server.util.Admin; import org.apache.accumulo.start.spi.KeywordExecutable; import org.apache.hadoop.conf.Configuration; @@ -45,7 +46,8 @@ public String description() { return "Checks the provided Accumulo configuration file for errors. " + "This only checks the contents of the file and not any running Accumulo system, " + "so it can be used prior to init, but only performs a subset of the checks done by " - + (new CheckServerConfig().keyword()); + + "the admin " + Admin.CheckCommand.class.getSimpleName() + " check " + + Admin.CheckCommand.Check.SERVER_CONFIG; } @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "intentional user-provided path") diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/CheckServerConfig.java b/server/base/src/main/java/org/apache/accumulo/server/conf/CheckServerConfig.java deleted file mode 100644 index b62c7fc17ac..00000000000 --- a/server/base/src/main/java/org/apache/accumulo/server/conf/CheckServerConfig.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * https://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.accumulo.server.conf; - -import org.apache.accumulo.core.conf.SiteConfiguration; -import org.apache.accumulo.server.ServerContext; -import org.apache.accumulo.start.spi.KeywordExecutable; - -import com.google.auto.service.AutoService; - -@AutoService(KeywordExecutable.class) -public class CheckServerConfig implements KeywordExecutable { - - public static void main(String[] args) { - try (var context = new ServerContext(SiteConfiguration.auto())) { - context.getConfiguration(); - } - } - - @Override - public String keyword() { - return "check-server-config"; - } - - @Override - public String description() { - return "Checks server config"; - } - - @Override - public void execute(String[] args) { - main(args); - } - -} 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 da4f51dd356..53f65cbddf3 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 @@ -148,7 +148,7 @@ public void walUnreferenced(TServerInstance tsi, Path path) throws WalMarkerExce updateState(tsi, path, WalState.UNREFERENCED); } - private static Pair parse(byte[] data) { + public static Pair parse(byte[] data) { String[] parts = new String(data, UTF_8).split(","); return new Pair<>(WalState.valueOf(parts[0]), new Path(parts[1])); } 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 047c2c47d90..4469d923a58 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 @@ -84,6 +84,7 @@ import org.apache.accumulo.server.util.checkCommand.MetadataTableCheckRunner; import org.apache.accumulo.server.util.checkCommand.RootMetadataCheckRunner; import org.apache.accumulo.server.util.checkCommand.RootTableCheckRunner; +import org.apache.accumulo.server.util.checkCommand.ServerConfigCheckRunner; import org.apache.accumulo.server.util.checkCommand.SystemConfigCheckRunner; import org.apache.accumulo.server.util.checkCommand.SystemFilesCheckRunner; import org.apache.accumulo.server.util.checkCommand.TableLocksCheckRunner; @@ -174,6 +175,8 @@ public enum Check { // Caution should be taken when changing or adding any new checks: order is important SYSTEM_CONFIG(SystemConfigCheckRunner::new, "Validate the system config stored in ZooKeeper", Collections.emptyList()), + SERVER_CONFIG(ServerConfigCheckRunner::new, "Validate the server configuration", + Collections.singletonList(SYSTEM_CONFIG)), TABLE_LOCKS(TableLocksCheckRunner::new, "Ensures that table and namespace locks are valid and are associated with a FATE op", Collections.singletonList(SYSTEM_CONFIG)), diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/MetadataCheckRunner.java b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/MetadataCheckRunner.java index dd1b8d525d6..2d42ea44eb7 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/MetadataCheckRunner.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/MetadataCheckRunner.java @@ -27,7 +27,6 @@ import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.Scanner; -import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.TableId; @@ -43,7 +42,6 @@ import org.apache.accumulo.server.constraints.SystemEnvironment; import org.apache.accumulo.server.util.Admin; import org.apache.hadoop.io.Text; -import org.apache.zookeeper.KeeperException; public interface MetadataCheckRunner extends CheckRunner { @@ -64,8 +62,7 @@ default String scanning() { * that are expected. For the root metadata, ensures that the expected "columns" exist in ZK. */ default Admin.CheckCommand.CheckStatus checkRequiredColumns(ServerContext context, - Admin.CheckCommand.CheckStatus status) - throws TableNotFoundException, InterruptedException, KeeperException { + Admin.CheckCommand.CheckStatus status) throws Exception { Set requiredColFQs; Set requiredColFams; boolean missingReqCol = false; diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/RootMetadataCheckRunner.java b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/RootMetadataCheckRunner.java index b6575b9e86f..8c355b5f5ad 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/RootMetadataCheckRunner.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/RootMetadataCheckRunner.java @@ -23,7 +23,6 @@ import java.util.HashSet; import java.util.Set; -import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.metadata.AccumuloTable; import org.apache.accumulo.core.metadata.RootTable; @@ -35,7 +34,6 @@ import org.apache.accumulo.server.util.Admin; import org.apache.accumulo.server.util.FindOfflineTablets; import org.apache.hadoop.io.Text; -import org.apache.zookeeper.KeeperException; public class RootMetadataCheckRunner implements MetadataCheckRunner { private static final Admin.CheckCommand.Check check = Admin.CheckCommand.Check.ROOT_METADATA; @@ -54,8 +52,7 @@ public TableId tableId() { public Set requiredColFQs() { return Set.of(MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN, MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN, - MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN, - MetadataSchema.TabletsSection.ServerColumnFamily.LOCK_COLUMN); + MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN); } @Override @@ -70,7 +67,7 @@ public String scanning() { @Override public Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtilOpts opts, - boolean fixFiles) throws TableNotFoundException, InterruptedException, KeeperException { + boolean fixFiles) throws Exception { Admin.CheckCommand.CheckStatus status = Admin.CheckCommand.CheckStatus.OK; printRunning(); @@ -97,8 +94,7 @@ public Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtil @Override public Admin.CheckCommand.CheckStatus checkRequiredColumns(ServerContext context, - Admin.CheckCommand.CheckStatus status) - throws TableNotFoundException, InterruptedException, KeeperException { + Admin.CheckCommand.CheckStatus status) throws Exception { final String path = context.getZooKeeperRoot() + RootTable.ZROOT_TABLET; final String json = new String(context.getZooSession().asReader().getData(path), UTF_8); final var rtm = new RootTabletMetadata(json); diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/RootTableCheckRunner.java b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/RootTableCheckRunner.java index fe7fde801b5..adb5b28b7dc 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/RootTableCheckRunner.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/RootTableCheckRunner.java @@ -51,8 +51,7 @@ public TableId tableId() { public Set requiredColFQs() { return Set.of(MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN, MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN, - MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN, - MetadataSchema.TabletsSection.ServerColumnFamily.LOCK_COLUMN); + MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN); } @Override diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/ServerConfigCheckRunner.java b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/ServerConfigCheckRunner.java new file mode 100644 index 00000000000..94e711847d0 --- /dev/null +++ b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/ServerConfigCheckRunner.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.accumulo.server.util.checkCommand; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.server.cli.ServerUtilOpts; +import org.apache.accumulo.server.util.Admin; + +public class ServerConfigCheckRunner implements CheckRunner { + private static final Admin.CheckCommand.Check check = Admin.CheckCommand.Check.SERVER_CONFIG; + + @Override + public Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtilOpts opts, + boolean fixFiles) throws Exception { + Admin.CheckCommand.CheckStatus status = Admin.CheckCommand.CheckStatus.OK; + printRunning(); + + log.trace("********** Checking server configuration **********"); + + log.trace("Checking that all configured properties are valid (valid key and value)"); + final Map definedProps = new HashMap<>(); + final var config = context.getConfiguration(); + config.getProperties(definedProps, s -> true); + for (var entry : definedProps.entrySet()) { + var key = entry.getKey(); + var val = entry.getValue(); + if (!Property.isValidProperty(key, val)) { + log.warn("Invalid property (key={} val={}) found in the config", key, val); + status = Admin.CheckCommand.CheckStatus.FAILED; + } + } + + log.trace("Checking that all required config properties are present"); + // there are many properties that should be set (default value or user set), identifying them + // all and checking them here is unrealistic. Some property that is not set but is expected + // will likely result in some sort of failure eventually anyway. We will just check a few + // obvious required properties here. + Set requiredProps = Set.of(Property.INSTANCE_ZK_HOST, Property.INSTANCE_ZK_TIMEOUT, + Property.INSTANCE_SECRET, Property.INSTANCE_VOLUMES, Property.GENERAL_THREADPOOL_SIZE, + Property.GENERAL_DELEGATION_TOKEN_LIFETIME, + Property.GENERAL_DELEGATION_TOKEN_UPDATE_INTERVAL, Property.GENERAL_IDLE_PROCESS_INTERVAL, + Property.GENERAL_LOW_MEM_DETECTOR_INTERVAL, Property.GENERAL_LOW_MEM_DETECTOR_THRESHOLD, + Property.GENERAL_PROCESS_BIND_ADDRESS, Property.GENERAL_SERVER_LOCK_VERIFICATION_INTERVAL, + Property.MANAGER_CLIENTPORT, Property.TSERV_CLIENTPORT, Property.GC_CYCLE_START, + Property.GC_CYCLE_DELAY, Property.GC_PORT, Property.MONITOR_PORT, Property.TABLE_MAJC_RATIO, + Property.TABLE_SPLIT_THRESHOLD); + for (var reqProp : requiredProps) { + var confPropVal = config.get(reqProp); + // already checked that all set properties are valid, just check that it is set then we know + // it's valid + if (confPropVal == null || confPropVal.isEmpty()) { + log.warn("Required property {} is not set!", reqProp); + status = Admin.CheckCommand.CheckStatus.FAILED; + } + } + + printCompleted(status); + return status; + } + + @Override + public Admin.CheckCommand.Check getCheck() { + return check; + } +} diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/SystemConfigCheckRunner.java b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/SystemConfigCheckRunner.java index 9e8467eb7bf..5b95f965255 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/SystemConfigCheckRunner.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/SystemConfigCheckRunner.java @@ -18,22 +18,205 @@ */ package org.apache.accumulo.server.util.checkCommand; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.lock.ServiceLock; +import org.apache.accumulo.core.metadata.AccumuloTable; +import org.apache.accumulo.core.metadata.TServerInstance; +import org.apache.accumulo.core.metadata.schema.TabletMetadata; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.cli.ServerUtilOpts; +import org.apache.accumulo.server.log.WalStateManager; import org.apache.accumulo.server.util.Admin; public class SystemConfigCheckRunner implements CheckRunner { private static final Admin.CheckCommand.Check check = Admin.CheckCommand.Check.SYSTEM_CONFIG; + public enum ServerProcess { + MANAGER, GC, TSERVER, COMPACTION_COORDINATOR, COMPACTOR, MONITOR, SCAN_SERVER + } + @Override public Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtilOpts opts, boolean fixFiles) throws Exception { Admin.CheckCommand.CheckStatus status = Admin.CheckCommand.CheckStatus.OK; printRunning(); + + log.trace("********** Checking validity of some ZooKeeper nodes **********"); + status = checkZkNodes(context, status); + printCompleted(status); return status; } + private static Admin.CheckCommand.CheckStatus checkZkNodes(ServerContext context, + Admin.CheckCommand.CheckStatus status) throws Exception { + status = checkZKLocks(context, status); + status = checkZKTableNodes(context, status); + status = checkZKWALsMetadata(context, status); + + return status; + } + + private static Admin.CheckCommand.CheckStatus checkZKLocks(ServerContext context, + Admin.CheckCommand.CheckStatus status) throws Exception { + final ServerProcess[] serverProcesses = ServerProcess.values(); + final String zkRoot = context.getZooKeeperRoot(); + final var zs = context.getZooSession(); + + log.trace("Checking ZooKeeper locks for Accumulo server processes..."); + + // check that essential server processes have a ZK lock failing otherwise + // check that nonessential server processes have a ZK lock only if they are running. If they are + // not running, alerts the user that the process is not running which may or may not be expected + for (ServerProcess proc : serverProcesses) { + log.trace("Looking for {} lock(s)...", proc); + switch (proc) { + case MANAGER: + // essential process + status = checkLock(zkRoot + Constants.ZMANAGER_LOCK, proc, true, zs, status); + break; + case GC: + // essential process + status = checkLock(zkRoot + Constants.ZGC_LOCK, proc, true, zs, status); + break; + case TSERVER: + // essential process(es) + final var tservers = TabletMetadata.getLiveTServers(context); + if (tservers.isEmpty()) { + log.warn("Did not find any running tablet servers!"); + status = Admin.CheckCommand.CheckStatus.FAILED; + } + break; + case COMPACTION_COORDINATOR: + // nonessential process + status = checkLock(zkRoot + Constants.ZCOORDINATOR_LOCK, proc, false, zs, status); + break; + case COMPACTOR: + // nonessential process(es) + final var compactors = context.instanceOperations().getCompactors(); + if (compactors.isEmpty()) { + log.debug("No compactors appear to be running... This may or may not be expected"); + } + break; + case MONITOR: + // nonessential process + status = checkLock(zkRoot + Constants.ZMONITOR_LOCK, proc, false, zs, status); + break; + case SCAN_SERVER: + // nonessential process(es) + final var sservers = context.instanceOperations().getScanServers(); + if (sservers.isEmpty()) { + log.debug("No scan servers appear to be running... This may or may not be expected"); + } + break; + default: + throw new IllegalStateException("Unhandled case: " + proc); + } + } + + return status; + } + + private static Admin.CheckCommand.CheckStatus checkLock(String path, ServerProcess proc, + boolean requiredProc, ZooSession zs, Admin.CheckCommand.CheckStatus status) throws Exception { + log.trace("Checking ZooKeeper lock at path {}", path); + + ServiceLock.ServiceLockPath slp = ServiceLock.path(path); + var opData = ServiceLock.getLockData(zs, slp); + if (requiredProc && opData.isEmpty()) { + log.warn("No ZooKeeper lock found for {} at {}! The process may not be running.", proc, path); + status = Admin.CheckCommand.CheckStatus.FAILED; + } else if (!requiredProc && opData.isEmpty()) { + log.debug("No ZooKeeper lock found for {} at {}. The process may not be running. " + + "This may or may not be expected.", proc, path); + } + return status; + } + + private static Admin.CheckCommand.CheckStatus checkZKTableNodes(ServerContext context, + Admin.CheckCommand.CheckStatus status) throws Exception { + log.trace("Checking ZooKeeper table nodes..."); + + final var zrw = context.getZooSession().asReaderWriter(); + final var zkRoot = context.getZooKeeperRoot(); + final var tableNameToId = context.tableOperations().tableIdMap(); + final Map systemTableNameToId = new HashMap<>(); + for (var accumuloTable : AccumuloTable.values()) { + systemTableNameToId.put(accumuloTable.tableName(), accumuloTable.tableId().canonical()); + } + + // ensure all system tables exist + if (!tableNameToId.values().containsAll(systemTableNameToId.values())) { + log.warn( + "Missing essential Accumulo table. One or more of {} are missing from the tables found {}", + systemTableNameToId, tableNameToId); + status = Admin.CheckCommand.CheckStatus.FAILED; + } + for (var nameToId : tableNameToId.entrySet()) { + var tablePath = zkRoot + Constants.ZTABLES + "/" + nameToId.getValue(); + // expect the table path to exist and some data to exist + if (!zrw.exists(tablePath) || zrw.getChildren(tablePath).isEmpty()) { + log.warn("Failed to find table ({}) info at expected path {}", nameToId, tablePath); + status = Admin.CheckCommand.CheckStatus.FAILED; + } + } + + return status; + } + + private static Admin.CheckCommand.CheckStatus checkZKWALsMetadata(ServerContext context, + Admin.CheckCommand.CheckStatus status) throws Exception { + final String zkRoot = context.getZooKeeperRoot(); + final var zs = context.getZooSession(); + final var zrw = zs.asReaderWriter(); + final var rootWalsDir = zkRoot + WalStateManager.ZWALS; + final Set tserverInstances = TabletMetadata.getLiveTServers(context); + final Set seenTServerInstancesAtWals = new HashSet<>(); + + log.trace("Checking that WAL metadata in ZooKeeper is valid..."); + + // each child node of the root wals dir should be a TServerInstance.toString() + var tserverInstancesAtWals = zrw.getChildren(rootWalsDir); + for (var tserverInstanceAtWals : tserverInstancesAtWals) { + final TServerInstance tsi = new TServerInstance(tserverInstanceAtWals); + seenTServerInstancesAtWals.add(tsi); + final var tserverPath = rootWalsDir + "/" + tserverInstanceAtWals; + // each child node of the tserver should be WAL metadata + final var wals = zrw.getChildren(tserverPath); + if (wals.isEmpty()) { + log.debug("No WAL metadata found for tserver {}", tsi); + } + for (var wal : wals) { + // should be able to parse the WAL metadata + final var fullWalPath = tserverPath + "/" + wal; + log.trace("Attempting to parse WAL metadata at {}", fullWalPath); + var parseRes = WalStateManager.parse(zrw.getData(fullWalPath)); + log.trace("Successfully parsed WAL metadata at {} result {}", fullWalPath, parseRes); + log.trace("Checking if the WAL path {} found in the metadata exists in HDFS...", + parseRes.getSecond()); + if (!context.getVolumeManager().exists(parseRes.getSecond())) { + log.warn("WAL metadata for tserver {} references a WAL that does not exist", + tserverInstanceAtWals); + status = Admin.CheckCommand.CheckStatus.FAILED; + } + } + } + if (!tserverInstances.equals(seenTServerInstancesAtWals)) { + log.warn( + "Expected WAL metadata in ZooKeeper for all tservers. tservers={} tservers seen storing WAL metadata={}", + tserverInstances, seenTServerInstancesAtWals); + status = Admin.CheckCommand.CheckStatus.FAILED; + } + + return status; + } + @Override public Admin.CheckCommand.Check getCheck() { return check; diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/TableLocksCheckRunner.java b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/TableLocksCheckRunner.java index 9ab3bee2ab2..47a379049f1 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/TableLocksCheckRunner.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/TableLocksCheckRunner.java @@ -22,15 +22,12 @@ import java.util.Map; import org.apache.accumulo.core.Constants; -import org.apache.accumulo.core.client.AccumuloException; -import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.fate.AdminUtil; import org.apache.accumulo.core.fate.ZooStore; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.cli.ServerUtilOpts; import org.apache.accumulo.server.util.Admin; -import org.apache.zookeeper.KeeperException; public class TableLocksCheckRunner implements CheckRunner { private static final Admin.CheckCommand.Check check = Admin.CheckCommand.Check.TABLE_LOCKS; @@ -54,8 +51,7 @@ public Admin.CheckCommand.Check getCheck() { } private static Admin.CheckCommand.CheckStatus checkTableLocks(ServerContext context, - Admin.CheckCommand.CheckStatus status) - throws InterruptedException, KeeperException, AccumuloException, AccumuloSecurityException { + Admin.CheckCommand.CheckStatus status) throws Exception { final AdminUtil admin = new AdminUtil<>(true); final String zkRoot = context.getZooKeeperRoot(); final var zTableLocksPath = ServiceLock.path(zkRoot + Constants.ZTABLE_LOCKS); diff --git a/test/src/main/java/org/apache/accumulo/test/AdminCheckIT.java b/test/src/main/java/org/apache/accumulo/test/AdminCheckIT.java index 8e32f1c89a2..7ad2e5e5f22 100644 --- a/test/src/main/java/org/apache/accumulo/test/AdminCheckIT.java +++ b/test/src/main/java/org/apache/accumulo/test/AdminCheckIT.java @@ -18,6 +18,7 @@ */ package org.apache.accumulo.test; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; @@ -34,12 +35,21 @@ import java.util.Set; import java.util.TreeMap; import java.util.function.Supplier; -import java.util.regex.Pattern; +import org.apache.accumulo.core.Constants; 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.CompactionConfig; +import org.apache.accumulo.core.data.Mutation; +import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.lock.ServiceLock; +import org.apache.accumulo.core.metadata.AccumuloTable; +import org.apache.accumulo.core.metadata.RootTable; +import org.apache.accumulo.core.metadata.StoredTabletFile; +import org.apache.accumulo.core.metadata.schema.MetadataSchema; +import org.apache.accumulo.core.metadata.schema.RootTabletMetadata; +import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.cli.ServerUtilOpts; import org.apache.accumulo.server.util.Admin; @@ -47,6 +57,7 @@ import org.apache.accumulo.test.functional.ConfigurableMacBase; import org.apache.accumulo.test.functional.ReadWriteIT; import org.apache.accumulo.test.functional.SlowIterator; +import org.apache.hadoop.fs.Path; import org.easymock.EasyMock; import org.easymock.IAnswer; import org.junit.jupiter.api.AfterEach; @@ -181,6 +192,7 @@ public void testAdminCheckRunNoCheckFailures() { String expRunAllRunOrder = "Running dummy check SYSTEM_CONFIG\nDummy check SYSTEM_CONFIG completed with status OK\n" + + "Running dummy check SERVER_CONFIG\nDummy check SERVER_CONFIG completed with status OK\n" + "Running dummy check TABLE_LOCKS\nDummy check TABLE_LOCKS completed with status OK\n" + "Running dummy check ROOT_METADATA\nDummy check ROOT_METADATA completed with status OK\n" + "Running dummy check ROOT_TABLE\nDummy check ROOT_TABLE completed with status OK\n" @@ -193,11 +205,13 @@ public void testAdminCheckRunNoCheckFailures() { + "Running dummy check USER_FILES\nDummy check USER_FILES completed with status OK\n"; // The dashes at the beginning and end of the string marks the begging and end of the // printed table allowing us to ensure the table only includes what is expected - String expRunAllStatusInfo = "-SYSTEM_CONFIG|OKTABLE_LOCKS|OKROOT_METADATA|OKROOT_TABLE|OK" - + "METADATA_TABLE|OKSYSTEM_FILES|OKUSER_FILES|OK-"; - String expRunSubStatusInfo = "-SYSTEM_CONFIG|FILTERED_OUTTABLE_LOCKS|FILTERED_OUT" - + "ROOT_METADATA|FILTERED_OUTROOT_TABLE|OKMETADATA_TABLE|FILTERED_OUT" - + "SYSTEM_FILES|OKUSER_FILES|OK-"; + String expRunAllStatusInfo = + "-SYSTEM_CONFIG|OKSERVER_CONFIG|OKTABLE_LOCKS|OKROOT_METADATA|OKROOT_TABLE|OK" + + "METADATA_TABLE|OKSYSTEM_FILES|OKUSER_FILES|OK-"; + String expRunSubStatusInfo = + "-SYSTEM_CONFIG|FILTERED_OUTSERVER_CONFIG|FILTERED_OUTTABLE_LOCKS|FILTERED_OUT" + + "ROOT_METADATA|FILTERED_OUTROOT_TABLE|OKMETADATA_TABLE|FILTERED_OUT" + + "SYSTEM_FILES|OKUSER_FILES|OK-"; assertTrue(out1.contains(expRunAllRunOrder)); assertTrue(out2.contains(expRunAllRunOrder)); @@ -226,10 +240,10 @@ public void testAdminCheckRunNoCheckFailures() { public void testAdminCheckRunWithCheckFailures() { // tests running checks with some failing - boolean[] rootTableFails = new boolean[] {true, true, true, false, true, true, true}; - boolean[] systemConfigFails = new boolean[] {false, true, true, true, true, true, true}; + boolean[] rootTableFails = new boolean[] {true, true, true, true, false, true, true, true}; + boolean[] systemConfigFails = new boolean[] {false, true, true, true, true, true, true, true}; boolean[] userFilesAndMetadataTableFails = - new boolean[] {true, true, true, true, false, true, false}; + new boolean[] {true, true, true, true, true, false, true, false}; // run all checks with ROOT_TABLE failing: only SYSTEM_CONFIG and ROOT_METADATA should pass // the rest should be filtered out as skipped due to dependency failure @@ -251,6 +265,7 @@ public void testAdminCheckRunWithCheckFailures() { String expRunOrder1 = "Running dummy check SYSTEM_CONFIG\nDummy check SYSTEM_CONFIG completed with status OK\n" + + "Running dummy check SERVER_CONFIG\nDummy check SERVER_CONFIG completed with status OK\n" + "Running dummy check TABLE_LOCKS\nDummy check TABLE_LOCKS completed with status OK\n" + "Running dummy check ROOT_METADATA\nDummy check ROOT_METADATA completed with status OK\n" + "Running dummy check ROOT_TABLE\nDummy check ROOT_TABLE completed with status FAILED"; @@ -267,8 +282,8 @@ public void testAdminCheckRunWithCheckFailures() { assertTrue(out4.contains(expRunOrder3And4)); assertNoOtherChecksRan(out1, true, Admin.CheckCommand.Check.SYSTEM_CONFIG, - Admin.CheckCommand.Check.TABLE_LOCKS, Admin.CheckCommand.Check.ROOT_TABLE, - Admin.CheckCommand.Check.ROOT_METADATA); + Admin.CheckCommand.Check.SERVER_CONFIG, Admin.CheckCommand.Check.TABLE_LOCKS, + Admin.CheckCommand.Check.ROOT_TABLE, Admin.CheckCommand.Check.ROOT_METADATA); assertNoOtherChecksRan(out2, true, Admin.CheckCommand.Check.SYSTEM_CONFIG); assertNoOtherChecksRan(out3, true, Admin.CheckCommand.Check.SYSTEM_CONFIG, Admin.CheckCommand.Check.ROOT_TABLE, Admin.CheckCommand.Check.USER_FILES); @@ -280,16 +295,17 @@ public void testAdminCheckRunWithCheckFailures() { out3 = out3.replaceAll("\\s+", ""); out4 = out4.replaceAll("\\s+", ""); - String expStatusInfo1 = "-SYSTEM_CONFIG|OKTABLE_LOCKS|OKROOT_METADATA|OKROOT_TABLE|FAILED" - + "METADATA_TABLE|SKIPPED_DEPENDENCY_FAILEDSYSTEM_FILES|SKIPPED_DEPENDENCY_FAILED" - + "USER_FILES|SKIPPED_DEPENDENCY_FAILED-"; - String expStatusInfo2 = "-SYSTEM_CONFIG|FAILEDTABLE_LOCKS|SKIPPED_DEPENDENCY_FAILED" - + "ROOT_METADATA|SKIPPED_DEPENDENCY_FAILEDROOT_TABLE|SKIPPED_DEPENDENCY_FAILED" - + "METADATA_TABLE|SKIPPED_DEPENDENCY_FAILEDSYSTEM_FILES|SKIPPED_DEPENDENCY_FAILED" - + "USER_FILES|SKIPPED_DEPENDENCY_FAILED-"; - String expStatusInfo3And4 = "-SYSTEM_CONFIG|OKTABLE_LOCKS|FILTERED_OUT" - + "ROOT_METADATA|FILTERED_OUTROOT_TABLE|OKMETADATA_TABLE|FILTERED_OUT" - + "SYSTEM_FILES|FILTERED_OUTUSER_FILES|FAILED"; + String expStatusInfo1 = "-SYSTEM_CONFIG|OKSERVER_CONFIG|OKTABLE_LOCKS|OKROOT_METADATA|OK" + + "ROOT_TABLE|FAILEDMETADATA_TABLE|SKIPPED_DEPENDENCY_FAILED" + + "SYSTEM_FILES|SKIPPED_DEPENDENCY_FAILEDUSER_FILES|SKIPPED_DEPENDENCY_FAILED-"; + String expStatusInfo2 = "-SYSTEM_CONFIG|FAILEDSERVER_CONFIG|SKIPPED_DEPENDENCY_FAILED" + + "TABLE_LOCKS|SKIPPED_DEPENDENCY_FAILEDROOT_METADATA|SKIPPED_DEPENDENCY_FAILED" + + "ROOT_TABLE|SKIPPED_DEPENDENCY_FAILEDMETADATA_TABLE|SKIPPED_DEPENDENCY_FAILED" + + "SYSTEM_FILES|SKIPPED_DEPENDENCY_FAILEDUSER_FILES|SKIPPED_DEPENDENCY_FAILED-"; + String expStatusInfo3And4 = + "-SYSTEM_CONFIG|OKSERVER_CONFIG|FILTERED_OUTTABLE_LOCKS|FILTERED_OUT" + + "ROOT_METADATA|FILTERED_OUTROOT_TABLE|OKMETADATA_TABLE|FILTERED_OUT" + + "SYSTEM_FILES|FILTERED_OUTUSER_FILES|FAILED"; assertTrue(out1.contains(expStatusInfo1)); assertTrue(out2.contains(expStatusInfo2)); @@ -303,8 +319,7 @@ public void testAdminCheckRunWithCheckFailures() { */ @Test - public void testPassingTableLocksCheck() throws Exception { - // Tests the TABLE_LOCKS check in the case where all checks pass + public void testTableLocksCheck() throws Exception { String table = getUniqueNames(1)[0]; Admin.CheckCommand.Check tableLocksCheck = Admin.CheckCommand.Check.TABLE_LOCKS; @@ -323,21 +338,44 @@ public void testPassingTableLocksCheck() throws Exception { slowCompaction.setIterators(List.of(is)); client.tableOperations().compact(table, slowCompaction); + // test passing case var p = getCluster().exec(Admin.class, "check", "run", tableLocksCheck.name()); assertEquals(0, p.getProcess().waitFor()); String out = p.readStdOut(); assertTrue(out.contains("locks are valid")); assertTrue(out.contains("Check TABLE_LOCKS completed with status OK")); assertNoOtherChecksRan(out, false, tableLocksCheck); + + // test a failing case + // write an invalid table lock + final var context = getCluster().getServerContext(); + final var zkRoot = context.getZooKeeperRoot(); + final var zrw = context.getZooSession().asReaderWriter(); + final var path = ServiceLock.path(zkRoot + Constants.ZTABLE_LOCKS + "/foo"); + zrw.putPersistentData(path.toString(), new byte[0], ZooUtil.NodeExistsPolicy.FAIL); + p = getCluster().exec(Admin.class, "check", "run", tableLocksCheck.name()); + assertEquals(0, p.getProcess().waitFor()); + out = p.readStdOut(); + assertTrue( + out.contains("Some table and namespace locks are INVALID (the table/namespace DNE)")); + assertTrue(out.contains("Check TABLE_LOCKS completed with status FAILED")); + assertNoOtherChecksRan(out, false, tableLocksCheck); } } @Test - public void testPassingMetadataTableCheck() throws Exception { - // Tests the METADATA_TABLE check in the case where all checks pass + public void testMetadataTableCheck() throws Exception { Admin.CheckCommand.Check metaTableCheck = Admin.CheckCommand.Check.METADATA_TABLE; + String table = getUniqueNames(1)[0]; + + try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) { + client.tableOperations().create(table); - // no extra setup needed, just check the metadata table + ReadWriteIT.ingest(client, 10, 10, 10, 0, table); + client.tableOperations().flush(table, null, null, true); + } + + // test passing case var p = getCluster().exec(Admin.class, "check", "run", metaTableCheck.name()); assertEquals(0, p.getProcess().waitFor()); String out = p.readStdOut(); @@ -347,13 +385,31 @@ public void testPassingMetadataTableCheck() throws Exception { assertTrue(out.contains("Looking for invalid columns")); assertTrue(out.contains("Check METADATA_TABLE completed with status OK")); assertNoOtherChecksRan(out, false, metaTableCheck); + + // test a failing case + // delete a required column for the metadata of the table we created + final var context = getCluster().getServerContext(); + final String tableId = context.tableOperations().tableIdMap().get(table); + final String tablet = tableId + "<"; + try (var writer = context.createBatchWriter(AccumuloTable.METADATA.tableName())) { + var mut = new Mutation(tablet); + mut.putDelete(MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN.getColumnFamily(), + MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN.getColumnQualifier()); + writer.addMutation(mut); + } + p = getCluster().exec(Admin.class, "check", "run", metaTableCheck.name()); + assertEquals(0, p.getProcess().waitFor()); + out = p.readStdOut(); + assertTrue(out.contains("Tablet " + tablet + " is missing required columns")); + assertTrue(out.contains("Check METADATA_TABLE completed with status FAILED")); + assertNoOtherChecksRan(out, false, metaTableCheck); } @Test - public void testPassingRootTableCheck() throws Exception { - // Tests the ROOT_TABLE check in the case where all checks pass + public void testRootTableCheck() throws Exception { Admin.CheckCommand.Check rootTableCheck = Admin.CheckCommand.Check.ROOT_TABLE; + // test passing case // no extra setup needed, just check the root table var p = getCluster().exec(Admin.class, "check", "run", rootTableCheck.name()); assertEquals(0, p.getProcess().waitFor()); @@ -364,13 +420,31 @@ public void testPassingRootTableCheck() throws Exception { assertTrue(out.contains("Looking for invalid columns")); assertTrue(out.contains("Check ROOT_TABLE completed with status OK")); assertNoOtherChecksRan(out, false, rootTableCheck); + + // test a failing case + // delete a required column for the metadata of the metadata table + final var context = getCluster().getServerContext(); + final String tableId = AccumuloTable.METADATA.tableId().canonical(); + final String tablet = tableId + "<"; + try (var writer = context.createBatchWriter(AccumuloTable.ROOT.tableName())) { + var mut = new Mutation(tablet); + mut.putDelete(MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN.getColumnFamily(), + MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN.getColumnQualifier()); + writer.addMutation(mut); + } + p = getCluster().exec(Admin.class, "check", "run", rootTableCheck.name()); + assertEquals(0, p.getProcess().waitFor()); + out = p.readStdOut(); + assertTrue(out.contains("Tablet " + tablet + " is missing required columns")); + assertTrue(out.contains("Check ROOT_TABLE completed with status FAILED")); + assertNoOtherChecksRan(out, false, rootTableCheck); } @Test - public void testPassingRootMetadataCheck() throws Exception { - // Tests the ROOT_TABLE check in the case where all checks pass + public void testRootMetadataCheck() throws Exception { Admin.CheckCommand.Check rootMetaCheck = Admin.CheckCommand.Check.ROOT_METADATA; + // test passing case // no extra setup needed, just check the root table metadata var p = getCluster().exec(Admin.class, "check", "run", rootMetaCheck.name()); assertEquals(0, p.getProcess().waitFor()); @@ -380,27 +454,69 @@ public void testPassingRootMetadataCheck() throws Exception { assertTrue(out.contains("Looking for invalid columns")); assertTrue(out.contains("Check ROOT_METADATA completed with status OK")); assertNoOtherChecksRan(out, false, rootMetaCheck); + + // test a failing case + // delete a required column for the metadata of the root tablet + final var context = getCluster().getServerContext(); + final String rootTabletMetaPath = context.getZooKeeperRoot() + RootTable.ZROOT_TABLET; + final var zrw = context.getZooSession().asReaderWriter(); + var json = new String(zrw.getData(rootTabletMetaPath), UTF_8); + var rtm = new RootTabletMetadata(json); + var tablet = rtm.toKeyValues().findFirst().orElseThrow().getKey().getRow(); + var mut = new Mutation(tablet); + mut.putDelete(MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN.getColumnFamily(), + MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN.getColumnQualifier()); + rtm.update(mut); + zrw.putPersistentData(rootTabletMetaPath, rtm.toJson().getBytes(UTF_8), + ZooUtil.NodeExistsPolicy.OVERWRITE); + + p = getCluster().exec(Admin.class, "check", "run", rootMetaCheck.name()); + assertEquals(0, p.getProcess().waitFor()); + out = p.readStdOut(); + assertTrue(out.contains("Tablet " + tablet + " is missing required columns")); + assertTrue(out.contains("Check ROOT_METADATA completed with status FAILED")); + assertNoOtherChecksRan(out, false, rootMetaCheck); } @Test - public void testPassingSystemFilesCheck() throws Exception { - // Tests the SYSTEM_FILES check in the case where it should pass + public void testSystemFilesCheck() throws Exception { Admin.CheckCommand.Check sysFilesCheck = Admin.CheckCommand.Check.SYSTEM_FILES; + // test passing case // no extra setup needed, just run the check var p = getCluster().exec(Admin.class, "check", "run", sysFilesCheck.name()); assertEquals(0, p.getProcess().waitFor()); String out = p.readStdOut(); - assertTrue(Pattern.compile("missing files: 0, total files: [1-9]+").matcher(out).find()); + assertTrue(out.contains("missing files: 0, total files: 1")); + assertTrue(out.contains("Check SYSTEM_FILES completed with status OK")); + assertNoOtherChecksRan(out, false, sysFilesCheck); + + // test a failing case + // read the root table to find where the metadata table rfile is located in HDFS then delete it + Path path; + ServerContext context = getCluster().getServerContext(); + try ( + var scanner = context.createScanner(AccumuloTable.ROOT.tableName(), Authorizations.EMPTY)) { + scanner.fetchColumnFamily(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME); + var pathJsonData = scanner.iterator().next().getKey().getColumnQualifier().toString(); + path = new Path(StoredTabletFile.of(pathJsonData).getMetadataPath()); + getCluster().getServerContext().getVolumeManager().delete(path); + } + p = getCluster().exec(Admin.class, "check", "run", sysFilesCheck.name()); + assertEquals(0, p.getProcess().waitFor()); + out = p.readStdOut(); + assertTrue(out.contains("File " + path + " is missing")); + assertTrue(out.contains("missing files: 1, total files: 1")); + assertTrue(out.contains("Check SYSTEM_FILES completed with status FAILED")); assertNoOtherChecksRan(out, false, sysFilesCheck); } @Test - public void testPassingUserFilesCheck() throws Exception { - // Tests the USER_FILES check in the case where it should pass + public void testUserFilesCheck() throws Exception { Admin.CheckCommand.Check userFilesCheck = Admin.CheckCommand.Check.USER_FILES; try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) { + // test passing case // create a table, insert some data, and flush so there's a file to check String table = getUniqueNames(1)[0]; client.tableOperations().create(table); @@ -410,12 +526,76 @@ public void testPassingUserFilesCheck() throws Exception { var p = getCluster().exec(Admin.class, "check", "run", userFilesCheck.name()); assertEquals(0, p.getProcess().waitFor()); String out = p.readStdOut(); - assertTrue(Pattern.compile("missing files: 0, total files: [1-9]+").matcher(out).find()); + assertTrue(out.contains("missing files: 0, total files: 1")); + assertTrue(out.contains("Check USER_FILES completed with status OK")); + assertNoOtherChecksRan(out, false, userFilesCheck); + + // test a failing case + // read the metadata for the table to find where the rfile is located in HDFS then delete it + Path path; + try (var scanner = + client.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) { + scanner.fetchColumnFamily(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME); + var pathJsonData = scanner.iterator().next().getKey().getColumnQualifier().toString(); + path = new Path(StoredTabletFile.of(pathJsonData).getMetadataPath()); + getCluster().getServerContext().getVolumeManager().delete(path); + } + p = getCluster().exec(Admin.class, "check", "run", userFilesCheck.name()); + assertEquals(0, p.getProcess().waitFor()); + out = p.readStdOut(); + assertTrue(out.contains("File " + path + " is missing")); + assertTrue(out.contains("missing files: 1, total files: 1")); + assertTrue(out.contains("Check USER_FILES completed with status FAILED")); assertNoOtherChecksRan(out, false, userFilesCheck); } } - // TODO 4892 need failing tests... + @Test + public void testSystemConfigCheck() throws Exception { + Admin.CheckCommand.Check sysConfCheck = Admin.CheckCommand.Check.SYSTEM_CONFIG; + + // test passing case + var p = getCluster().exec(Admin.class, "check", "run", sysConfCheck.name()); + assertEquals(0, p.getProcess().waitFor()); + String out = p.readStdOut(); + assertTrue(out.contains("Checking ZooKeeper locks for Accumulo server processes")); + assertTrue(out.contains("Checking ZooKeeper table nodes")); + assertTrue(out.contains("Checking that WAL metadata in ZooKeeper is valid")); + assertTrue(out.contains("Check SYSTEM_CONFIG completed with status OK")); + assertNoOtherChecksRan(out, false, sysConfCheck); + + // test a failing case + // delete the ZK data for the metadata table + var context = getCluster().getServerContext(); + var zrw = context.getZooSession().asReaderWriter(); + zrw.recursiveDelete( + context.getZooKeeperRoot() + Constants.ZTABLES + "/" + AccumuloTable.METADATA.tableId(), + ZooUtil.NodeMissingPolicy.FAIL); + + p = getCluster().exec(Admin.class, "check", "run", sysConfCheck.name()); + assertEquals(0, p.getProcess().waitFor()); + out = p.readStdOut(); + assertTrue(out.contains("Missing essential Accumulo table")); + assertTrue(out.contains("Check SYSTEM_CONFIG completed with status FAILED")); + assertNoOtherChecksRan(out, false, sysConfCheck); + } + + @Test + public void testServerConfigCheck() throws Exception { + Admin.CheckCommand.Check servConfCheck = Admin.CheckCommand.Check.SERVER_CONFIG; + + // test passing case + var p = getCluster().exec(Admin.class, "check", "run", servConfCheck.name()); + assertEquals(0, p.getProcess().waitFor()); + String out = p.readStdOut(); + assertTrue(out.contains("Checking server configuration")); + assertTrue(out.contains("Checking that all configured properties are valid")); + assertTrue(out.contains("Checking that all required config properties are present")); + assertTrue(out.contains("Check SERVER_CONFIG completed with status OK")); + assertNoOtherChecksRan(out, false, servConfCheck); + + // no simple way to test for a failure case + } private String executeCheckCommand(String[] checkCmdArgs, boolean[] checksPass) { String output; @@ -498,6 +678,17 @@ public Admin.CheckCommand.Check getCheck() { } } + static class DummyServerConfigCheckRunner extends DummyCheckRunner { + public DummyServerConfigCheckRunner(boolean passes) { + super(passes); + } + + @Override + public Admin.CheckCommand.Check getCheck() { + return Admin.CheckCommand.Check.SERVER_CONFIG; + } + } + static class DummyTableLocksCheckRunner extends DummyCheckRunner { public DummyTableLocksCheckRunner(boolean passes) { super(passes); @@ -571,15 +762,17 @@ public DummyCheckCommand(boolean[] checksPass) { this.checkRunners = new TreeMap<>(); this.checkRunners.put(Check.SYSTEM_CONFIG, () -> new DummySystemConfigCheckRunner(checksPass[0])); - this.checkRunners.put(Check.TABLE_LOCKS, () -> new DummyTableLocksCheckRunner(checksPass[1])); + this.checkRunners.put(Check.SERVER_CONFIG, + () -> new DummyServerConfigCheckRunner(checksPass[1])); + this.checkRunners.put(Check.TABLE_LOCKS, () -> new DummyTableLocksCheckRunner(checksPass[2])); this.checkRunners.put(Check.ROOT_METADATA, - () -> new DummyRootMetadataCheckRunner(checksPass[2])); - this.checkRunners.put(Check.ROOT_TABLE, () -> new DummyRootTableCheckRunner(checksPass[3])); + () -> new DummyRootMetadataCheckRunner(checksPass[3])); + this.checkRunners.put(Check.ROOT_TABLE, () -> new DummyRootTableCheckRunner(checksPass[4])); this.checkRunners.put(Check.METADATA_TABLE, - () -> new DummyMetadataTableCheckRunner(checksPass[4])); + () -> new DummyMetadataTableCheckRunner(checksPass[5])); this.checkRunners.put(Check.SYSTEM_FILES, - () -> new DummySystemFilesCheckRunner(checksPass[5])); - this.checkRunners.put(Check.USER_FILES, () -> new DummyUserFilesCheckRunner(checksPass[6])); + () -> new DummySystemFilesCheckRunner(checksPass[6])); + this.checkRunners.put(Check.USER_FILES, () -> new DummyUserFilesCheckRunner(checksPass[7])); } @Override diff --git a/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java b/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java index e8e94146a77..55b7b184625 100644 --- a/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java +++ b/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java @@ -54,7 +54,6 @@ import org.apache.accumulo.monitor.MonitorExecutable; import org.apache.accumulo.server.conf.CheckAccumuloProperties; import org.apache.accumulo.server.conf.CheckCompactionConfig; -import org.apache.accumulo.server.conf.CheckServerConfig; import org.apache.accumulo.server.conf.util.ZooInfoViewer; import org.apache.accumulo.server.conf.util.ZooPropEditor; import org.apache.accumulo.server.init.Initialize; @@ -129,7 +128,6 @@ public void testExpectedClasses() { TreeMap> expectSet = new TreeMap<>(); expectSet.put("admin", Admin.class); expectSet.put("check-compaction-config", CheckCompactionConfig.class); - expectSet.put("check-server-config", CheckServerConfig.class); expectSet.put("check-accumulo-properties", CheckAccumuloProperties.class); expectSet.put("compaction-coordinator", CoordinatorExecutable.class); expectSet.put("compactor", CompactorExecutable.class); @@ -199,7 +197,6 @@ public void checkHasMain() { HashSet> expectSet = new HashSet<>(); expectSet.add(Admin.class); expectSet.add(CheckCompactionConfig.class); - expectSet.add(CheckServerConfig.class); expectSet.add(CreateEmpty.class); expectSet.add(CreateToken.class); expectSet.add(DumpZookeeper.class);