From 9a9b2531c940f6364fd3f1cdc0b53ffb6698b31c Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Tue, 8 Oct 2024 11:58:18 -0400 Subject: [PATCH 1/2] Checks for problems in Accumulo This commit: - Moves existing checks (`checkTablets` and the fate check for dangling locks) into the appropriate new `admin check` command - Adds new checks - New tests in AdminCheckIT - SYSTEM_CONFIG now checks for - valid locked table/namespace ids (the locked table/namespaces exist) - locked table/namespaces are associated with a fate op - ROOT_METADATA now checks for - offline tablets - missing "columns" - invalid "columns" - ROOT_TABLE now checks for - offline tablets - tablets for metadata table have no holes, valid (null) prev end row for first tablet, and valid (null) end row for last tablet - missing columns - invalid columns - METADATA_TABLE now checks for - offline tablets - tablets for user tables (and scanref) have no holes, valid (null) prev end row for first tablet, and valid (null) end row for last tablet - missing columns - invalid columns - SYSTEM_FILES now checks for - missing system files - USER_FILES now checks for - missing user files Part of #4892 --- .../apache/accumulo/core/fate/AdminUtil.java | 12 -- .../core/iterators/user/WholeRowIterator.java | 10 +- .../apache/accumulo/server/util/Admin.java | 66 ++----- .../server/util/CheckForMetadataProblems.java | 52 +++-- .../server/util/FindOfflineTablets.java | 25 ++- .../util/RemoveEntriesForMissingFiles.java | 5 +- .../server/util/checkCommand/CheckRunner.java | 24 ++- .../checkCommand/MetadataCheckRunner.java | 177 ++++++++++++++++++ .../MetadataTableCheckRunner.java | 67 ++++++- .../checkCommand/RootMetadataCheckRunner.java | 112 ++++++++++- .../checkCommand/RootTableCheckRunner.java | 69 ++++++- .../checkCommand/SystemConfigCheckRunner.java | 81 +++++++- .../checkCommand/SystemFilesCheckRunner.java | 18 +- .../checkCommand/UserFilesCheckRunner.java | 23 ++- .../server/util/AdminCommandsTest.java | 7 - .../apache/accumulo/test/AdminCheckIT.java | 126 ++++++++++++- 16 files changed, 741 insertions(+), 133 deletions(-) create mode 100644 server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/MetadataCheckRunner.java diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java index 78f107f33b9..80b21fc04d0 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java @@ -424,18 +424,6 @@ public void print(ReadOnlyTStore zs, ZooReader zk, ServiceLock.ServiceLockPat txStatus.getWaitingLocks(), txStatus.getTop(), txStatus.getTimeCreatedFormatted()); } fmt.format(" %s transactions", fateStatus.getTransactions().size()); - - if (!fateStatus.getDanglingHeldLocks().isEmpty() - || !fateStatus.getDanglingWaitingLocks().isEmpty()) { - fmt.format("%nThe following locks did not have an associated FATE operation%n"); - for (Entry> entry : fateStatus.getDanglingHeldLocks().entrySet()) { - fmt.format("txid: %s locked: %s%n", entry.getKey(), entry.getValue()); - } - - for (Entry> entry : fateStatus.getDanglingWaitingLocks().entrySet()) { - fmt.format("txid: %s locking: %s%n", entry.getKey(), entry.getValue()); - } - } } public boolean prepDelete(TStore zs, ZooReaderWriter zk, ServiceLockPath path, diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/WholeRowIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/WholeRowIterator.java index 3cfc195dbc4..e70ff429961 100644 --- a/core/src/main/java/org/apache/accumulo/core/iterators/user/WholeRowIterator.java +++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/WholeRowIterator.java @@ -86,7 +86,9 @@ private static byte[] readField(DataInputStream din) throws IOException { return b; } - // decode a bunch of key value pairs that have been encoded into a single value + /** + * decode a bunch of key value pairs that have been encoded into a single value + */ public static final SortedMap decodeRow(Key rowKey, Value rowValue) throws IOException { SortedMap map = new TreeMap<>(); @@ -110,8 +112,10 @@ private static void encode(DataOutputStream dout, ByteSequence bs) throws IOExce dout.write(bs.getBackingArray(), bs.offset(), bs.length()); } - // take a stream of keys and values and output a value that encodes everything but their row - // keys and values must be paired one for one + /** + * take a stream of keys and values and output a value that encodes everything but their row keys + * and values must be paired one for one + */ public static final Value encodeRow(List keys, List values) throws IOException { ByteArrayOutputStream out = new ByteArrayOutputStream(); DataOutputStream dout = new DataOutputStream(out); 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 781037f66e6..acfe19c53cf 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 @@ -145,6 +145,10 @@ public static class CheckCommand { @Parameter(description = "[...]") List checks; + @Parameter(names = "--fixFiles", description = "Removes dangling file pointers. Used by the " + + "USER_FILES and SYSTEM_FILES checks.") + boolean fixFiles = false; + /** * This should be used to get the check runner instead of {@link Check#getCheckRunner()}. This * exists so that its functionality can be changed for testing. @@ -214,16 +218,6 @@ public enum CheckStatus { } } - @Parameters(commandDescription = "print tablets that are offline in online tables") - static class CheckTabletsCommand { - @Parameter(names = "--fixFiles", description = "Remove dangling file pointers") - boolean fixFiles = false; - - @Parameter(names = {"-t", "--table"}, - description = "Table to check, if not set checks all tables") - String tableName = null; - } - @Parameters(commandDescription = "stop the manager") static class StopManagerCommand {} @@ -377,9 +371,6 @@ public void execute(final String[] args) { CheckCommand checkCommand = new CheckCommand(); cl.addCommand("check", checkCommand); - CheckTabletsCommand checkTabletsCommand = new CheckTabletsCommand(); - cl.addCommand("checkTablets", checkTabletsCommand); - DeleteZooInstanceCommand deleteZooInstOpts = new DeleteZooInstanceCommand(); cl.addCommand("deleteZooInstance", deleteZooInstOpts); @@ -424,15 +415,13 @@ public void execute(final String[] args) { return; } - ServerContext context = opts.getServerContext(); - - AccumuloConfiguration conf = context.getConfiguration(); - // Login as the server on secure HDFS - if (conf.getBoolean(Property.INSTANCE_RPC_SASL_ENABLED)) { - SecurityUtil.serverLogin(conf); - } + try (ServerContext context = opts.getServerContext()) { - try { + AccumuloConfiguration conf = context.getConfiguration(); + // Login as the server on secure HDFS + if (conf.getBoolean(Property.INSTANCE_RPC_SASL_ENABLED)) { + SecurityUtil.serverLogin(conf); + } int rc = 0; @@ -443,24 +432,6 @@ public void execute(final String[] args) { if (ping(context, pingCommand.args) != 0) { rc = 4; } - } else if (cl.getParsedCommand().equals("checkTablets")) { - System.out.println("\n*** Looking for offline tablets ***\n"); - if (FindOfflineTablets.findOffline(context, checkTabletsCommand.tableName) != 0) { - rc = 5; - } - System.out.println("\n*** Looking for missing files ***\n"); - if (checkTabletsCommand.tableName == null) { - if (RemoveEntriesForMissingFiles.checkAllTables(context, checkTabletsCommand.fixFiles) - != 0) { - rc = 6; - } - } else { - if (RemoveEntriesForMissingFiles.checkTable(context, checkTabletsCommand.tableName, - checkTabletsCommand.fixFiles) != 0) { - rc = 6; - } - } - } else if (cl.getParsedCommand().equals("stop")) { stopTabletServer(context, stopOpts.args, opts.force); } else if (cl.getParsedCommand().equals("dumpConfig")) { @@ -484,7 +455,7 @@ public void execute(final String[] args) { } else if (cl.getParsedCommand().equals("serviceStatus")) { printServiceStatus(context, serviceStatusCommandOpts); } else if (cl.getParsedCommand().equals("check")) { - executeCheckCommand(context, checkCommand); + executeCheckCommand(context, checkCommand, opts); } else { everything = cl.getParsedCommand().equals("stopAll"); @@ -1014,15 +985,16 @@ private EnumSet getCmdLineStatusFilters(List sta } @VisibleForTesting - public static void executeCheckCommand(ServerContext context, CheckCommand cmd) { + public static void executeCheckCommand(ServerContext context, CheckCommand cmd, + ServerUtilOpts opts) throws Exception { validateAndTransformCheckCommand(cmd); if (cmd.list) { listChecks(); } else if (cmd.run) { - var givenChecks = - cmd.checks.stream().map(CheckCommand.Check::valueOf).collect(Collectors.toList()); - executeRunCheckCommand(cmd, givenChecks); + var givenChecks = cmd.checks.stream() + .map(name -> CheckCommand.Check.valueOf(name.toUpperCase())).collect(Collectors.toList()); + executeRunCheckCommand(cmd, givenChecks, context, opts); } } @@ -1064,8 +1036,8 @@ private static void listChecks() { System.out.println(); } - private static void executeRunCheckCommand(CheckCommand cmd, - List givenChecks) { + private static void executeRunCheckCommand(CheckCommand cmd, List givenChecks, + ServerContext context, ServerUtilOpts opts) throws Exception { // Get all the checks in the order they are declared in the enum final var allChecks = CheckCommand.Check.values(); final TreeMap checkStatus = new TreeMap<>(); @@ -1075,7 +1047,7 @@ private static void executeRunCheckCommand(CheckCommand cmd, checkStatus.put(check, CheckCommand.CheckStatus.SKIPPED_DEPENDENCY_FAILED); } else { if (givenChecks.contains(check)) { - checkStatus.put(check, cmd.getCheckRunner(check).runCheck()); + checkStatus.put(check, cmd.getCheckRunner(check).runCheck(context, opts, cmd.fixFiles)); } else { checkStatus.put(check, CheckCommand.CheckStatus.FILTERED_OUT); } diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java b/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java index 1aafbec1459..052f1d9661a 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java @@ -45,13 +45,13 @@ import io.opentelemetry.context.Scope; public class CheckForMetadataProblems { - private static boolean sawProblems = false; - private static ServerUtilOpts opts; - private static void checkTable(TableId tableId, TreeSet tablets) { + private static boolean checkTable(TableId tableId, TreeSet tablets, + ServerUtilOpts opts) { // sanity check of metadata table entries - // make sure tablets has no holes, and that it starts and ends w/ null + // make sure tablets have no holes, and that it starts and ends w/ null String tableName; + boolean sawProblems = false; try { tableName = opts.getServerContext().getTableName(tableId); @@ -59,25 +59,26 @@ private static void checkTable(TableId tableId, TreeSet tablets) { tableName = null; } + System.out.printf("Ensuring tablets for table %s (%s) have: no holes, " + + "valid (null) prev end row for first tablet, and valid (null) end row " + + "for last tablet...\n", tableName, tableId); + if (tablets.isEmpty()) { System.out.println( "...No entries found in metadata table for table " + tableName + " (" + tableId + ")"); - sawProblems = true; - return; + return true; } if (tablets.first().prevEndRow() != null) { System.out.println("...First entry for table " + tableName + " (" + tableId + ") - " + tablets.first() + " - has non null prev end row"); - sawProblems = true; - return; + return true; } if (tablets.last().endRow() != null) { System.out.println("...Last entry for table " + tableName + " (" + tableId + ") - " + tablets.last() + " - has non null end row"); - sawProblems = true; - return; + return true; } Iterator tabIter = tablets.iterator(); @@ -106,14 +107,17 @@ private static void checkTable(TableId tableId, TreeSet tablets) { } else { sawProblems = true; } + + return sawProblems; } - private static void checkMetadataAndRootTableEntries(String tableNameToCheck, ServerUtilOpts opts) - throws Exception { + public static boolean checkMetadataAndRootTableEntries(String tableNameToCheck, + ServerUtilOpts opts) throws Exception { TableId tableCheckId = opts.getServerContext().getTableId(tableNameToCheck); System.out.println("Checking tables whose metadata is found in: " + tableNameToCheck + " (" - + tableCheckId + ")"); + + tableCheckId + ")...\n"); Map> tables = new HashMap<>(); + boolean sawProblems = false; try (AccumuloClient client = Accumulo.newClient().from(opts.getClientProps()).build(); Scanner scanner = client.createScanner(tableNameToCheck, Authorizations.EMPTY)) { @@ -139,7 +143,10 @@ private static void checkMetadataAndRootTableEntries(String tableNameToCheck, Se TreeSet tablets = tables.get(tableId); if (tablets == null) { - tables.forEach(CheckForMetadataProblems::checkTable); + for (var e : tables.entrySet()) { + sawProblems = + CheckForMetadataProblems.checkTable(e.getKey(), e.getValue(), opts) || sawProblems; + } tables.clear(); @@ -167,23 +174,30 @@ private static void checkMetadataAndRootTableEntries(String tableNameToCheck, Se } } - tables.forEach(CheckForMetadataProblems::checkTable); + for (var e : tables.entrySet()) { + sawProblems = + CheckForMetadataProblems.checkTable(e.getKey(), e.getValue(), opts) || sawProblems; + } if (!sawProblems) { - System.out.println("No problems found in " + tableNameToCheck + " (" + tableCheckId + ")"); + System.out + .println("\n...No problems found in " + tableNameToCheck + " (" + tableCheckId + ")"); } // end METADATA table sanity check + return sawProblems; } public static void main(String[] args) throws Exception { - opts = new ServerUtilOpts(); + ServerUtilOpts opts = new ServerUtilOpts(); opts.parseArgs(CheckForMetadataProblems.class.getName(), args); Span span = TraceUtil.startSpan(CheckForMetadataProblems.class, "main"); + boolean sawProblems; try (Scope scope = span.makeCurrent()) { - checkMetadataAndRootTableEntries(AccumuloTable.ROOT.tableName(), opts); + sawProblems = checkMetadataAndRootTableEntries(AccumuloTable.ROOT.tableName(), opts); System.out.println(); - checkMetadataAndRootTableEntries(AccumuloTable.METADATA.tableName(), opts); + sawProblems = + checkMetadataAndRootTableEntries(AccumuloTable.METADATA.tableName(), opts) || sawProblems; if (sawProblems) { throw new IllegalStateException(); } diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java b/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java index 342fd828c1f..8564a8065fb 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java @@ -55,13 +55,14 @@ public static void main(String[] args) throws Exception { Span span = TraceUtil.startSpan(FindOfflineTablets.class, "main"); try (Scope scope = span.makeCurrent()) { ServerContext context = opts.getServerContext(); - findOffline(context, null); + findOffline(context, null, false, false); } finally { span.end(); } } - static int findOffline(ServerContext context, String tableName) throws TableNotFoundException { + public static int findOffline(ServerContext context, String tableName, boolean skipZkScan, + boolean skipRootScan) throws TableNotFoundException { final AtomicBoolean scanning = new AtomicBoolean(false); @@ -85,20 +86,24 @@ public void update(LiveTServerSet current, Set deleted, int offline = 0; - System.out.println("Scanning zookeeper"); - if ((offline = checkTablets(context, zooScanner, tservers)) > 0) { - return offline; + if (!skipZkScan) { + System.out.println("Scanning zookeeper"); + if ((offline = checkTablets(context, zooScanner, tservers)) > 0) { + return offline; + } } if (AccumuloTable.ROOT.tableName().equals(tableName)) { return 0; } - System.out.println("Scanning " + AccumuloTable.ROOT.tableName()); - Iterator rootScanner = new MetaDataTableScanner(context, - TabletsSection.getRange(), AccumuloTable.ROOT.tableName()); - if ((offline = checkTablets(context, rootScanner, tservers)) > 0) { - return offline; + if (!skipRootScan) { + System.out.println("Scanning " + AccumuloTable.ROOT.tableName()); + Iterator rootScanner = new MetaDataTableScanner(context, + TabletsSection.getRange(), AccumuloTable.ROOT.tableName()); + if ((offline = checkTablets(context, rootScanner, tservers)) > 0) { + return offline; + } } if (AccumuloTable.METADATA.tableName().equals(tableName)) { diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java b/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java index 8642b958c63..1fc51add628 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java @@ -181,7 +181,7 @@ private static int checkTable(ServerContext context, String tableName, Range ran writer.close(); } - System.out.printf("Scan finished, %d files of %d missing\n\n", missing.get(), count); + System.out.printf("Scan finished, missing files: %d, total files: %d\n", missing.get(), count); return missing.get(); } @@ -198,7 +198,8 @@ static int checkAllTables(ServerContext context, boolean fix) throws Exception { } } - static int checkTable(ServerContext context, String tableName, boolean fix) throws Exception { + public static int checkTable(ServerContext context, String tableName, boolean fix) + throws Exception { if (tableName.equals(AccumuloTable.ROOT.tableName())) { throw new IllegalArgumentException("Can not check root table"); } else if (tableName.equals(AccumuloTable.METADATA.tableName())) { diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/CheckRunner.java b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/CheckRunner.java index 2e1377e3ed6..7da66dc156a 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/CheckRunner.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/CheckRunner.java @@ -18,6 +18,8 @@ */ package org.apache.accumulo.server.util.checkCommand; +import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.server.cli.ServerUtilOpts; import org.apache.accumulo.server.util.Admin; public interface CheckRunner { @@ -25,9 +27,14 @@ public interface CheckRunner { /** * Runs the check * + * @param context server context + * @param opts server util opts. Only applicable for the checks on the root and metadata tables + * @param fixFiles remove dangling file pointers. Only applicable for the checks on the system and + * user files * @return the {@link Admin.CheckCommand.CheckStatus} resulting from running the check */ - Admin.CheckCommand.CheckStatus runCheck(); + Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtilOpts opts, + boolean fixFiles) throws Exception; /** * @@ -35,4 +42,19 @@ public interface CheckRunner { */ Admin.CheckCommand.Check getCheck(); + default void printRunning() { + String running = "Running check " + getCheck(); + System.out.println(); + System.out.println("-".repeat(running.length())); + System.out.println(running); + System.out.println("-".repeat(running.length())); + } + + default void printCompleted(Admin.CheckCommand.CheckStatus status) { + String completed = "Check " + getCheck() + " completed with status " + status; + System.out.println(); + System.out.println("-".repeat(completed.length())); + System.out.println(completed); + System.out.println("-".repeat(completed.length())); + } } 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 new file mode 100644 index 00000000000..4756f7f14fb --- /dev/null +++ b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/MetadataCheckRunner.java @@ -0,0 +1,177 @@ +/* + * 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.io.IOException; +import java.util.AbstractMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; +import java.util.SortedMap; + +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; +import org.apache.accumulo.core.data.TabletId; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.iterators.user.WholeRowIterator; +import org.apache.accumulo.core.metadata.schema.MetadataSchema; +import org.apache.accumulo.core.security.AuthorizationContainer; +import org.apache.accumulo.core.security.Authorizations; +import org.apache.accumulo.core.util.ColumnFQ; +import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.server.constraints.MetadataConstraints; +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 { + + String tableName(); + + TableId tableId(); + + Set requiredColFQs(); + + Set requiredColFams(); + + default String scanning() { + return String.format("%s (%s) table", tableName(), tableId()); + } + + /** + * Ensures that the {@link #tableName()} table (either metadata or root table) has all columns + * 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 { + Set requiredColFQs; + Set requiredColFams; + boolean missingReqCol = false; + + System.out.printf("Scanning the %s for missing required columns...\n", scanning()); + try (Scanner scanner = context.createScanner(tableName(), Authorizations.EMPTY)) { + var is = new IteratorSetting(100, "tablets", WholeRowIterator.class); + scanner.addScanIterator(is); + scanner.setRange(MetadataSchema.TabletsSection.getRange()); + for (var entry : scanner) { + requiredColFQs = new HashSet<>(requiredColFQs()); + requiredColFams = new HashSet<>(requiredColFams()); + SortedMap rowMap; + try { + rowMap = WholeRowIterator.decodeRow(entry.getKey(), entry.getValue()); + } catch (IOException e) { + throw new RuntimeException(e); + } + for (var e : rowMap.entrySet()) { + var key = e.getKey(); + boolean removed = + requiredColFQs.remove(new ColumnFQ(key.getColumnFamily(), key.getColumnQualifier())); + if (!removed) { + requiredColFams.remove(key.getColumnFamily()); + } + } + if (!requiredColFQs.isEmpty() || !requiredColFams.isEmpty()) { + System.out.printf( + "Tablet %s is missing required columns: col FQs: %s, col fams: %s in the %s\n", + entry.getKey().getRow(), requiredColFQs, requiredColFams, scanning()); + status = Admin.CheckCommand.CheckStatus.FAILED; + missingReqCol = true; + } + } + } + + if (!missingReqCol) { + System.out.printf("...The %s contains all required columns for all tablets\n", scanning()); + } + return status; + } + + /** + * Ensures each column in the root or metadata table (or in ZK for the root metadata) is valid - + * no unexpected columns, and for the columns that are expected, ensures the values are valid + */ + default Admin.CheckCommand.CheckStatus checkColumns(ServerContext context, + Iterator> iter, + Admin.CheckCommand.CheckStatus status) + throws TableNotFoundException, InterruptedException, KeeperException { + boolean invalidCol = false; + + System.out.printf("Scanning the %s for invalid columns...\n", scanning()); + while (iter.hasNext()) { + var entry = iter.next(); + Key key = entry.getKey(); + // create a mutation that's equivalent to the existing data to check validity + Mutation m = new Mutation(key.getRow()); + m.at().family(key.getColumnFamily()).qualifier(key.getColumnQualifier()) + .visibility(key.getColumnVisibility()).timestamp(key.getTimestamp()) + .put(entry.getValue()); + MetadataConstraints mc = new MetadataConstraints(); + var violations = mc.check(new ConstraintEnv(context), m); + if (!violations.isEmpty()) { + violations.forEach( + violationCode -> System.out.println(mc.getViolationDescription(violationCode))); + status = Admin.CheckCommand.CheckStatus.FAILED; + invalidCol = true; + } + } + + if (!invalidCol) { + System.out.printf("...All columns in the %s are valid\n", scanning()); + } + return status; + } + + /** + * A {@link SystemEnvironment} whose only valid operation is + * {@link ConstraintEnv#getServerContext()} + */ + class ConstraintEnv implements SystemEnvironment { + ServerContext context; + + ConstraintEnv(ServerContext context) { + this.context = context; + } + + @Override + public TabletId getTablet() { + throw new UnsupportedOperationException(); + } + + @Override + public String getUser() { + throw new UnsupportedOperationException(); + } + + @Override + public AuthorizationContainer getAuthorizationsContainer() { + throw new UnsupportedOperationException(); + } + + @Override + public ServerContext getServerContext() { + return context; + } + } +} diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/MetadataTableCheckRunner.java b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/MetadataTableCheckRunner.java index aeed7adc230..87c1a10a374 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/MetadataTableCheckRunner.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/MetadataTableCheckRunner.java @@ -18,18 +18,75 @@ */ package org.apache.accumulo.server.util.checkCommand; +import java.util.AbstractMap; +import java.util.Set; + +import org.apache.accumulo.core.client.Scanner; +import org.apache.accumulo.core.data.TableId; +import org.apache.accumulo.core.metadata.AccumuloTable; +import org.apache.accumulo.core.metadata.schema.MetadataSchema; +import org.apache.accumulo.core.security.Authorizations; +import org.apache.accumulo.core.util.ColumnFQ; +import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.server.cli.ServerUtilOpts; import org.apache.accumulo.server.util.Admin; +import org.apache.accumulo.server.util.CheckForMetadataProblems; +import org.apache.accumulo.server.util.FindOfflineTablets; +import org.apache.hadoop.io.Text; -public class MetadataTableCheckRunner implements CheckRunner { +public class MetadataTableCheckRunner implements MetadataCheckRunner { private static final Admin.CheckCommand.Check check = Admin.CheckCommand.Check.METADATA_TABLE; @Override - public Admin.CheckCommand.CheckStatus runCheck() { + public String tableName() { + return AccumuloTable.METADATA.tableName(); + } + + @Override + public TableId tableId() { + return AccumuloTable.METADATA.tableId(); + } + + @Override + public Set requiredColFQs() { + return Set.of(MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN, + MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN, + MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN); + } + + @Override + public Set requiredColFams() { + return Set.of(); + } + + @Override + public Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtilOpts opts, + boolean fixFiles) throws Exception { Admin.CheckCommand.CheckStatus status = Admin.CheckCommand.CheckStatus.OK; + printRunning(); + + System.out.println("\n********** Looking for offline tablets **********\n"); + if (FindOfflineTablets.findOffline(context, null, true, true) != 0) { + status = Admin.CheckCommand.CheckStatus.FAILED; + } else { + System.out.println("All good... No offline tablets found"); + } + + System.out.println("\n********** Checking some references **********\n"); + if (CheckForMetadataProblems.checkMetadataAndRootTableEntries(tableName(), opts)) { + status = Admin.CheckCommand.CheckStatus.FAILED; + } + + System.out.println("\n********** Looking for missing columns **********\n"); + status = checkRequiredColumns(context, status); + + System.out.println("\n********** Looking for invalid columns **********\n"); + try (Scanner scanner = context.createScanner(tableName(), Authorizations.EMPTY)) { + status = checkColumns(context, + scanner.stream().map(AbstractMap.SimpleImmutableEntry::new).iterator(), status); + } - System.out.println("Running check " + check); - // work - System.out.println("Check " + check + " completed with status " + status); + printCompleted(status); return status; } 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 bd4282246dc..bef1c8173a9 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 @@ -18,18 +18,120 @@ */ package org.apache.accumulo.server.util.checkCommand; +import static java.nio.charset.StandardCharsets.UTF_8; + +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; +import org.apache.accumulo.core.metadata.schema.MetadataSchema; +import org.apache.accumulo.core.metadata.schema.RootTabletMetadata; +import org.apache.accumulo.core.util.ColumnFQ; +import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.server.cli.ServerUtilOpts; 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 CheckRunner { +public class RootMetadataCheckRunner implements MetadataCheckRunner { private static final Admin.CheckCommand.Check check = Admin.CheckCommand.Check.ROOT_METADATA; @Override - public Admin.CheckCommand.CheckStatus runCheck() { + public String tableName() { + throw new UnsupportedOperationException(); + } + + @Override + public TableId tableId() { + throw new UnsupportedOperationException(); + } + + @Override + 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); + } + + @Override + public Set requiredColFams() { + return Set.of(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME); + } + + @Override + public String scanning() { + return "root tablet metadata in ZooKeeper"; + } + + @Override + public Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtilOpts opts, + boolean fixFiles) throws TableNotFoundException, InterruptedException, KeeperException { Admin.CheckCommand.CheckStatus status = Admin.CheckCommand.CheckStatus.OK; + printRunning(); + + System.out.println("\n********** Looking for offline tablets **********\n"); + if (FindOfflineTablets.findOffline(context, AccumuloTable.ROOT.tableName(), false, true) != 0) { + status = Admin.CheckCommand.CheckStatus.FAILED; + } else { + System.out.println("All good... No offline tablets found"); + } + + System.out.println("\n********** Looking for missing columns **********\n"); + status = checkRequiredColumns(context, status); + + System.out.println("\n********** Looking for invalid columns **********\n"); + final String path = context.getZooKeeperRoot() + RootTable.ZROOT_TABLET; + final String json = new String(context.getZooReader().getData(path), UTF_8); + final var rtm = new RootTabletMetadata(json); + status = checkColumns(context, rtm.toKeyValues().iterator(), status); + + printCompleted(status); + return status; + } + + @Override + public Admin.CheckCommand.CheckStatus checkRequiredColumns(ServerContext context, + Admin.CheckCommand.CheckStatus status) + throws TableNotFoundException, InterruptedException, KeeperException { + final String path = context.getZooKeeperRoot() + RootTable.ZROOT_TABLET; + final String json = new String(context.getZooReader().getData(path), UTF_8); + final var rtm = new RootTabletMetadata(json); + final Set rowsSeen = new HashSet<>(); + final Set requiredColFQs = new HashSet<>(requiredColFQs()); + final Set requiredColFams = new HashSet<>(requiredColFams()); + + System.out.printf("Scanning the %s for missing required columns...\n", scanning()); + rtm.toKeyValues().forEach(e -> { + var key = e.getKey(); + rowsSeen.add(key.getRow()); + boolean removed = + requiredColFQs.remove(new ColumnFQ(key.getColumnFamily(), key.getColumnQualifier())); + if (!removed) { + requiredColFams.remove(key.getColumnFamily()); + } + }); + + if (rowsSeen.size() != 1) { + status = Admin.CheckCommand.CheckStatus.FAILED; + System.out.println("Did not see one tablet for the root table!"); + } else { + if (!requiredColFQs.isEmpty() || !requiredColFams.isEmpty()) { + System.out.printf( + "Tablet %s is missing required columns: col FQs: %s, col fams: %s in the %s\n", + rowsSeen.stream().findFirst().orElseThrow(), requiredColFQs, requiredColFams, + scanning()); + status = Admin.CheckCommand.CheckStatus.FAILED; + } else { + System.out.printf("...The %s contains all required columns for the root tablet\n", + scanning()); + } + } - System.out.println("Running check " + check); - // work - System.out.println("Check " + check + " completed with status " + status); return status; } 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 85558ed1417..f4e790ab2a4 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 @@ -18,18 +18,77 @@ */ package org.apache.accumulo.server.util.checkCommand; +import java.util.AbstractMap; +import java.util.Set; + +import org.apache.accumulo.core.client.Scanner; +import org.apache.accumulo.core.data.TableId; +import org.apache.accumulo.core.metadata.AccumuloTable; +import org.apache.accumulo.core.metadata.schema.MetadataSchema; +import org.apache.accumulo.core.security.Authorizations; +import org.apache.accumulo.core.util.ColumnFQ; +import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.server.cli.ServerUtilOpts; import org.apache.accumulo.server.util.Admin; +import org.apache.accumulo.server.util.CheckForMetadataProblems; +import org.apache.accumulo.server.util.FindOfflineTablets; +import org.apache.hadoop.io.Text; -public class RootTableCheckRunner implements CheckRunner { +public class RootTableCheckRunner implements MetadataCheckRunner { private static final Admin.CheckCommand.Check check = Admin.CheckCommand.Check.ROOT_TABLE; @Override - public Admin.CheckCommand.CheckStatus runCheck() { + public String tableName() { + return AccumuloTable.ROOT.tableName(); + } + + @Override + public TableId tableId() { + return AccumuloTable.ROOT.tableId(); + } + + @Override + 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); + } + + @Override + public Set requiredColFams() { + return Set.of(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME); + } + + @Override + public Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtilOpts opts, + boolean fixFiles) throws Exception { Admin.CheckCommand.CheckStatus status = Admin.CheckCommand.CheckStatus.OK; + printRunning(); + + System.out.println("\n********** Looking for offline tablets **********\n"); + if (FindOfflineTablets.findOffline(context, AccumuloTable.METADATA.tableName(), true, false) + != 0) { + status = Admin.CheckCommand.CheckStatus.FAILED; + } else { + System.out.println("All good... No offline tablets found"); + } + + System.out.println("\n********** Checking some references **********\n"); + if (CheckForMetadataProblems.checkMetadataAndRootTableEntries(tableName(), opts)) { + status = Admin.CheckCommand.CheckStatus.FAILED; + } + + System.out.println("\n********** Looking for missing columns **********\n"); + status = checkRequiredColumns(context, status); + + System.out.println("\n********** Looking for invalid columns **********\n"); + try (Scanner scanner = context.createScanner(tableName(), Authorizations.EMPTY)) { + status = checkColumns(context, + scanner.stream().map(AbstractMap.SimpleImmutableEntry::new).iterator(), status); + } - System.out.println("Running check " + check); - // work - System.out.println("Check " + check + " completed with status " + status); + printCompleted(status); return status; } 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 56e0b2b136d..0a2e799b288 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,18 +18,91 @@ */ package org.apache.accumulo.server.util.checkCommand; +import java.util.List; +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.fate.zookeeper.ZooReaderWriter; +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 SystemConfigCheckRunner implements CheckRunner { private static final Admin.CheckCommand.Check check = Admin.CheckCommand.Check.SYSTEM_CONFIG; @Override - public Admin.CheckCommand.CheckStatus runCheck() { + public Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtilOpts opts, + boolean fixFiles) throws Exception { Admin.CheckCommand.CheckStatus status = Admin.CheckCommand.CheckStatus.OK; + printRunning(); + + System.out.println("\n********** Checking some references **********\n"); + status = checkTableLocks(context, status); + + printCompleted(status); + return status; + } + + private static Admin.CheckCommand.CheckStatus checkTableLocks(ServerContext context, + Admin.CheckCommand.CheckStatus status) + throws InterruptedException, KeeperException, AccumuloException, AccumuloSecurityException { + final AdminUtil admin = new AdminUtil<>(true); + final String zkRoot = context.getZooKeeperRoot(); + final var zTableLocksPath = ServiceLock.path(zkRoot + Constants.ZTABLE_LOCKS); + final String fateZkPath = zkRoot + Constants.ZFATE; + final ZooReaderWriter zk = context.getZooReaderWriter(); + final ZooStore zs = new ZooStore<>(fateZkPath, zk); + + System.out.println("Ensuring table and namespace locks are valid..."); + + var tableIds = context.tableOperations().tableIdMap().values(); + var namespaceIds = context.namespaceOperations().namespaceIdMap().values(); + List lockedIds = zk.getChildren(zTableLocksPath.toString()); + boolean locksExist = !lockedIds.isEmpty(); + + if (locksExist) { + lockedIds.removeAll(tableIds); + lockedIds.removeAll(namespaceIds); + if (!lockedIds.isEmpty()) { + status = Admin.CheckCommand.CheckStatus.FAILED; + System.out + .println("...Some table and namespace locks are INVALID (the table/namespace DNE): " + + lockedIds); + } else { + System.out.println("...locks are valid"); + } + } else { + System.out.println("...no locks present"); + } + + System.out.println("Ensuring table and namespace locks are associated with a FATE op..."); + + if (locksExist) { + final var fateStatus = admin.getStatus(zs, zk, zTableLocksPath, null, null); + if (!fateStatus.getDanglingHeldLocks().isEmpty() + || !fateStatus.getDanglingWaitingLocks().isEmpty()) { + status = Admin.CheckCommand.CheckStatus.FAILED; + System.out.println("The following locks did not have an associated FATE operation\n"); + for (Map.Entry> entry : fateStatus.getDanglingHeldLocks().entrySet()) { + System.out.println("txid: " + entry.getKey() + " locked: " + entry.getValue()); + } + for (Map.Entry> entry : fateStatus.getDanglingWaitingLocks() + .entrySet()) { + System.out.println("txid: " + entry.getKey() + " locking: " + entry.getValue()); + } + } else { + System.out.println("...locks are valid"); + } + } else { + System.out.println("...no locks present"); + } - System.out.println("Running check " + check); - // work - System.out.println("Check " + check + " completed with status " + status); return status; } diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/SystemFilesCheckRunner.java b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/SystemFilesCheckRunner.java index 284d170d9b0..a7c97b6c79c 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/SystemFilesCheckRunner.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/SystemFilesCheckRunner.java @@ -18,18 +18,28 @@ */ package org.apache.accumulo.server.util.checkCommand; +import org.apache.accumulo.core.metadata.AccumuloTable; +import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.server.cli.ServerUtilOpts; import org.apache.accumulo.server.util.Admin; +import org.apache.accumulo.server.util.RemoveEntriesForMissingFiles; public class SystemFilesCheckRunner implements CheckRunner { private static final Admin.CheckCommand.Check check = Admin.CheckCommand.Check.SYSTEM_FILES; @Override - public Admin.CheckCommand.CheckStatus runCheck() { + public Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtilOpts opts, + boolean fixFiles) throws Exception { Admin.CheckCommand.CheckStatus status = Admin.CheckCommand.CheckStatus.OK; + printRunning(); - System.out.println("Running check " + check); - // work - System.out.println("Check " + check + " completed with status " + status); + System.out.println("\n********** Looking for missing system files **********\n"); + if (RemoveEntriesForMissingFiles.checkTable(context, AccumuloTable.METADATA.tableName(), + fixFiles) != 0) { + status = Admin.CheckCommand.CheckStatus.FAILED; + } + + printCompleted(status); return status; } diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/UserFilesCheckRunner.java b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/UserFilesCheckRunner.java index 088814e0f50..a5570b3c472 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/UserFilesCheckRunner.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/UserFilesCheckRunner.java @@ -20,18 +20,33 @@ import static org.apache.accumulo.server.util.Admin.CheckCommand.Check; +import org.apache.accumulo.core.metadata.AccumuloTable; +import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.server.cli.ServerUtilOpts; import org.apache.accumulo.server.util.Admin; +import org.apache.accumulo.server.util.RemoveEntriesForMissingFiles; public class UserFilesCheckRunner implements CheckRunner { private static final Check check = Check.USER_FILES; @Override - public Admin.CheckCommand.CheckStatus runCheck() { + public Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtilOpts opts, + boolean fixFiles) throws Exception { Admin.CheckCommand.CheckStatus status = Admin.CheckCommand.CheckStatus.OK; + printRunning(); - System.out.println("Running check " + check); - // work - System.out.println("Check " + check + " completed with status " + status); + System.out.println("\n********** Looking for missing user files **********\n"); + for (String tableName : context.tableOperations().list()) { + var tableId = context.getTableId(tableName); + if (!AccumuloTable.allTableIds().contains(context.getTableId(tableName))) { + System.out.printf("Checking table %s (%s) for missing files\n", tableName, tableId); + if (RemoveEntriesForMissingFiles.checkTable(context, tableName, fixFiles) != 0) { + status = Admin.CheckCommand.CheckStatus.FAILED; + } + } + } + + printCompleted(status); return status; } diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java index 36388f96305..e3415b2484b 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java @@ -37,13 +37,6 @@ public void testPingCommand() { assertEquals(0, cmd.args.size()); } - @Test - public void testCheckTabletsCommand() { - Admin.CheckTabletsCommand cmd = new Admin.CheckTabletsCommand(); - assertFalse(cmd.fixFiles); - assertNull(cmd.tableName); - } - @Test public void testStopManagerCommand() { new Admin.StopManagerCommand(); 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 7ebefb047c2..73dde21e730 100644 --- a/test/src/main/java/org/apache/accumulo/test/AdminCheckIT.java +++ b/test/src/main/java/org/apache/accumulo/test/AdminCheckIT.java @@ -32,11 +32,19 @@ import java.util.Map; import java.util.TreeMap; import java.util.function.Supplier; +import java.util.regex.Pattern; +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.server.ServerContext; import org.apache.accumulo.server.cli.ServerUtilOpts; import org.apache.accumulo.server.util.Admin; import org.apache.accumulo.server.util.checkCommand.CheckRunner; import org.apache.accumulo.test.functional.ConfigurableMacBase; +import org.apache.accumulo.test.functional.ReadWriteIT; +import org.apache.accumulo.test.functional.SlowIterator; import org.easymock.EasyMock; import org.easymock.IAnswer; import org.junit.jupiter.api.AfterEach; @@ -54,9 +62,9 @@ public void assertCorrectPostTestState() { } /* - * The following tests test the expected outputs and functionality of the admin check command - * (e.g., are the correct checks run, dependencies run before the actual check, run in the correct - * order, etc.) without actually testing the correct functionality of the checks + * The following tests test the expected outputs and run order of the admin check command (e.g., + * are the correct checks run, dependencies run before the actual check, run in the correct order, + * etc.) without actually testing the correct functionality of the checks */ @Test @@ -264,6 +272,113 @@ public void testAdminCheckRunWithCheckFailures() { assertTrue(out4.contains(expStatusInfo3And4)); } + /* + * The following tests test the expected functionality of the admin check command (e.g., are the + * checks correctly identifying problems) + */ + + @Test + public void testSystemConfigCheck() throws Exception { + String table = getUniqueNames(1)[0]; + + try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) { + client.tableOperations().create(table); + + ReadWriteIT.ingest(client, 10, 10, 10, 0, table); + client.tableOperations().flush(table, null, null, true); + + // the slow compaction is to ensure we hold a table lock when the check runs, so we have + // locks to check + IteratorSetting is = new IteratorSetting(1, SlowIterator.class); + is.addOption("sleepTime", "10000"); + CompactionConfig slowCompaction = new CompactionConfig(); + slowCompaction.setWait(false); + slowCompaction.setIterators(List.of(is)); + client.tableOperations().compact(table, slowCompaction); + + var p = getCluster().exec(Admin.class, "check", "run", "system_config"); + assertEquals(0, p.getProcess().waitFor()); + String out = p.readStdOut(); + assertTrue(out.contains("locks are valid")); + assertTrue(out.contains("Check SYSTEM_CONFIG completed with status OK")); + } + } + + @Test + public void testPassingMetadataTableCheck() throws Exception { + // Tests the METADATA_TABLE check in the case where all checks pass + + // no extra setup needed, just check the metadata table + var p = getCluster().exec(Admin.class, "check", "run", "metadata_table"); + assertEquals(0, p.getProcess().waitFor()); + String out = p.readStdOut(); + assertTrue(out.contains("Looking for offline tablets")); + assertTrue(out.contains("Checking some references")); + assertTrue(out.contains("Looking for missing columns")); + assertTrue(out.contains("Looking for invalid columns")); + assertTrue(out.contains("Check METADATA_TABLE completed with status OK")); + } + + @Test + public void testPassingRootTableCheck() throws Exception { + // Tests the ROOT_TABLE check in the case where all checks pass + + // no extra setup needed, just check the root table + var p = getCluster().exec(Admin.class, "check", "run", "root_table"); + assertEquals(0, p.getProcess().waitFor()); + String out = p.readStdOut(); + assertTrue(out.contains("Looking for offline tablets")); + assertTrue(out.contains("Checking some references")); + assertTrue(out.contains("Looking for missing columns")); + assertTrue(out.contains("Looking for invalid columns")); + assertTrue(out.contains("Check ROOT_TABLE completed with status OK")); + } + + @Test + public void testPassingRootMetadataCheck() throws Exception { + // Tests the ROOT_TABLE check in the case where all checks pass + + // no extra setup needed, just check the root table metadata + var p = getCluster().exec(Admin.class, "check", "run", "root_metadata"); + assertEquals(0, p.getProcess().waitFor()); + String out = p.readStdOut(); + assertTrue(out.contains("Looking for offline tablets")); + assertTrue(out.contains("Looking for missing columns")); + assertTrue(out.contains("Looking for invalid columns")); + assertTrue(out.contains("Check ROOT_METADATA completed with status OK")); + } + + @Test + public void testPassingSystemFilesCheck() throws Exception { + // Tests the SYSTEM_FILES check in the case where it should pass + + // no extra setup needed, just run the check + var p = getCluster().exec(Admin.class, "check", "run", "system_files"); + assertEquals(0, p.getProcess().waitFor()); + String out = p.readStdOut(); + assertTrue(Pattern.compile("missing files: 0, total files: [1-9]+").matcher(out).find()); + } + + @Test + public void testPassingUserFilesCheck() throws Exception { + // Tests the USER_FILES check in the case where it should pass + + try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) { + // create a table, insert some data, and flush so there's a file to check + String table = getUniqueNames(1)[0]; + client.tableOperations().create(table); + ReadWriteIT.ingest(client, 10, 10, 10, 0, table); + client.tableOperations().flush(table, null, null, true); + + var p = getCluster().exec(Admin.class, "check", "run", "user_files"); + assertEquals(0, p.getProcess().waitFor()); + String out = p.readStdOut(); + assertTrue(Pattern.compile("missing files: 0, total files: [1-9]+").matcher(out).find()); + } + } + + // TODO 4892 need failing tests... + private String executeCheckCommand(String[] checkCmdArgs, boolean[] checksPass) { String output; Admin admin = createMockAdmin(checksPass); @@ -294,7 +409,7 @@ private Admin createMockAdmin(boolean[] checksPass) { Admin.CheckCommand dummyCheckCommand = new DummyCheckCommand(checksPass); cl.addCommand("check", dummyCheckCommand); cl.parse(args); - Admin.executeCheckCommand(getServerContext(), dummyCheckCommand); + Admin.executeCheckCommand(getServerContext(), dummyCheckCommand, opts); return null; }); EasyMock.replay(admin); @@ -309,7 +424,8 @@ public DummyCheckRunner(boolean passes) { } @Override - public Admin.CheckCommand.CheckStatus runCheck() { + public Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtilOpts opts, + boolean fixFiles) throws Exception { Admin.CheckCommand.CheckStatus status = passes ? Admin.CheckCommand.CheckStatus.OK : Admin.CheckCommand.CheckStatus.FAILED; From 11f1f7643e51c7f0851a0220b4892e074ff86594 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Tue, 5 Nov 2024 17:02:18 -0500 Subject: [PATCH 2/2] addresses review: - `System.out` -> `log.trace/warn` to avoid flooding output with unnecessary/detailed info. The most important info (e.g., output of `admin check list` command, and the final run status table from the `admin check run` command) is still printed to stdout. Problems found are now logged at warn instead of stdout. Detailed, non-error info logged at trace. - Created new check `Check.TABLE_LOCKS` which ensures that table and namespace locks are valid and are associated with a FATE op. - New check `assertNoOtherChecksRan()` in `AdminCheckIT` which ensures only the expected checks ran - A few misc review changes: `MetadataCheckRunner` code improved to only fetch required columns when scanning, object creation moved outside of a loop --- .../apache/accumulo/server/util/Admin.java | 12 +- .../server/util/CheckForMetadataProblems.java | 72 ++++++----- .../server/util/FindOfflineTablets.java | 24 ++-- .../util/RemoveEntriesForMissingFiles.java | 53 +++++--- .../server/util/checkCommand/CheckRunner.java | 17 +-- .../checkCommand/MetadataCheckRunner.java | 29 +++-- .../MetadataTableCheckRunner.java | 15 +-- .../checkCommand/RootMetadataCheckRunner.java | 21 ++-- .../checkCommand/RootTableCheckRunner.java | 17 +-- .../checkCommand/SystemConfigCheckRunner.java | 72 ----------- .../checkCommand/SystemFilesCheckRunner.java | 4 +- .../checkCommand/TableLocksCheckRunner.java | 112 +++++++++++++++++ .../checkCommand/UserFilesCheckRunner.java | 7 +- .../apache/accumulo/test/AdminCheckIT.java | 117 ++++++++++++++---- .../src/main/resources/log4j2-test.properties | 3 + 15 files changed, 359 insertions(+), 216 deletions(-) create mode 100644 server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/TableLocksCheckRunner.java 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 acfe19c53cf..36eb3b95c5e 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 @@ -86,6 +86,7 @@ import org.apache.accumulo.server.util.checkCommand.RootTableCheckRunner; import org.apache.accumulo.server.util.checkCommand.SystemConfigCheckRunner; import org.apache.accumulo.server.util.checkCommand.SystemFilesCheckRunner; +import org.apache.accumulo.server.util.checkCommand.TableLocksCheckRunner; import org.apache.accumulo.server.util.checkCommand.UserFilesCheckRunner; import org.apache.accumulo.server.util.fateCommand.FateSummaryReport; import org.apache.accumulo.start.spi.KeywordExecutable; @@ -163,6 +164,9 @@ 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()), + TABLE_LOCKS(TableLocksCheckRunner::new, + "Ensures that table and namespace locks are valid and are associated with a FATE op", + Collections.singletonList(SYSTEM_CONFIG)), ROOT_METADATA(RootMetadataCheckRunner::new, "Checks integrity of the root tablet metadata stored in ZooKeeper", Collections.singletonList(SYSTEM_CONFIG)), @@ -1025,14 +1029,14 @@ private static void validateAndTransformCheckCommand(CheckCommand cmd) { private static void listChecks() { System.out.println(); - System.out.printf("%-20s | %-80s | %-20s%n", "Check Name", "Description", "Depends on"); - System.out.println("-".repeat(120)); + System.out.printf("%-20s | %-90s | %-20s%n", "Check Name", "Description", "Depends on"); + System.out.println("-".repeat(130)); for (CheckCommand.Check check : CheckCommand.Check.values()) { - System.out.printf("%-20s | %-80s | %-20s%n", check.name(), check.getDescription(), + System.out.printf("%-20s | %-90s | %-20s%n", check.name(), check.getDescription(), check.getDependencies().stream().map(CheckCommand.Check::name) .collect(Collectors.joining(", "))); } - System.out.println("-".repeat(120)); + System.out.println("-".repeat(130)); System.out.println(); } diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java b/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java index 052f1d9661a..a93dc9ae048 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java @@ -23,6 +23,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.TreeSet; +import java.util.function.Consumer; import org.apache.accumulo.core.client.Accumulo; import org.apache.accumulo.core.client.AccumuloClient; @@ -47,7 +48,7 @@ public class CheckForMetadataProblems { private static boolean checkTable(TableId tableId, TreeSet tablets, - ServerUtilOpts opts) { + ServerUtilOpts opts, Consumer printInfoMethod, Consumer printProblemMethod) { // sanity check of metadata table entries // make sure tablets have no holes, and that it starts and ends w/ null String tableName; @@ -59,25 +60,27 @@ private static boolean checkTable(TableId tableId, TreeSet tablets, tableName = null; } - System.out.printf("Ensuring tablets for table %s (%s) have: no holes, " + printInfoMethod.accept(String.format("Ensuring tablets for table %s (%s) have: no holes, " + "valid (null) prev end row for first tablet, and valid (null) end row " - + "for last tablet...\n", tableName, tableId); + + "for last tablet...\n", tableName, tableId)); if (tablets.isEmpty()) { - System.out.println( - "...No entries found in metadata table for table " + tableName + " (" + tableId + ")"); + printProblemMethod.accept(String + .format("...No entries found in metadata table for table %s (%s)", tableName, tableId)); return true; } if (tablets.first().prevEndRow() != null) { - System.out.println("...First entry for table " + tableName + " (" + tableId + ") - " - + tablets.first() + " - has non null prev end row"); + printProblemMethod + .accept(String.format("...First entry for table %s (%s) - %s - has non-null prev end row", + tableName, tableId, tablets.first())); return true; } if (tablets.last().endRow() != null) { - System.out.println("...Last entry for table " + tableName + " (" + tableId + ") - " - + tablets.last() + " - has non null end row"); + printProblemMethod + .accept(String.format("...Last entry for table %s (%s) - %s - has non-null end row", + tableName, tableId, tablets.last())); return true; } @@ -85,25 +88,26 @@ private static boolean checkTable(TableId tableId, TreeSet tablets, Text lastEndRow = tabIter.next().endRow(); boolean everythingLooksGood = true; while (tabIter.hasNext()) { - KeyExtent tabke = tabIter.next(); + KeyExtent table = tabIter.next(); boolean broke = false; - if (tabke.prevEndRow() == null) { - System.out.println("...Table " + tableName + " (" + tableId - + ") has null prev end row in middle of table " + tabke); + if (table.prevEndRow() == null) { + printProblemMethod + .accept(String.format("...Table %s (%s) has null prev end row in middle of table %s", + tableName, tableId, table)); broke = true; - } else if (!tabke.prevEndRow().equals(lastEndRow)) { - System.out.println("...Table " + tableName + " (" + tableId + ") has a hole " - + tabke.prevEndRow() + " != " + lastEndRow); + } else if (!table.prevEndRow().equals(lastEndRow)) { + printProblemMethod.accept(String.format("...Table %s (%s) has a hole %s != %s", tableName, + tableId, table.prevEndRow(), lastEndRow)); broke = true; } if (broke) { everythingLooksGood = false; } - lastEndRow = tabke.endRow(); + lastEndRow = table.endRow(); } if (everythingLooksGood) { - System.out.println("...All is well for table " + tableName + " (" + tableId + ")"); + printInfoMethod.accept(String.format("...All is well for table %s (%s)", tableName, tableId)); } else { sawProblems = true; } @@ -112,10 +116,11 @@ private static boolean checkTable(TableId tableId, TreeSet tablets, } public static boolean checkMetadataAndRootTableEntries(String tableNameToCheck, - ServerUtilOpts opts) throws Exception { + ServerUtilOpts opts, Consumer printInfoMethod, Consumer printProblemMethod) + throws Exception { TableId tableCheckId = opts.getServerContext().getTableId(tableNameToCheck); - System.out.println("Checking tables whose metadata is found in: " + tableNameToCheck + " (" - + tableCheckId + ")...\n"); + printInfoMethod.accept(String.format("Checking tables whose metadata is found in: %s (%s)...\n", + tableNameToCheck, tableCheckId)); Map> tables = new HashMap<>(); boolean sawProblems = false; @@ -144,8 +149,8 @@ public static boolean checkMetadataAndRootTableEntries(String tableNameToCheck, if (tablets == null) { for (var e : tables.entrySet()) { - sawProblems = - CheckForMetadataProblems.checkTable(e.getKey(), e.getValue(), opts) || sawProblems; + sawProblems = CheckForMetadataProblems.checkTable(e.getKey(), e.getValue(), opts, + printInfoMethod, printProblemMethod) || sawProblems; } tables.clear(); @@ -160,7 +165,7 @@ public static boolean checkMetadataAndRootTableEntries(String tableNameToCheck, justLoc = false; } else if (colf.equals(CurrentLocationColumnFamily.NAME)) { if (justLoc) { - System.out.println("Problem at key " + entry.getKey()); + printProblemMethod.accept("Problem at key " + entry.getKey()); sawProblems = true; } justLoc = true; @@ -168,20 +173,20 @@ public static boolean checkMetadataAndRootTableEntries(String tableNameToCheck, } if (count == 0) { - System.err - .println("ERROR : table " + tableNameToCheck + " (" + tableCheckId + ") is empty"); + printProblemMethod.accept( + String.format("ERROR : table %s (%s) is empty", tableNameToCheck, tableCheckId)); sawProblems = true; } } for (var e : tables.entrySet()) { - sawProblems = - CheckForMetadataProblems.checkTable(e.getKey(), e.getValue(), opts) || sawProblems; + sawProblems = CheckForMetadataProblems.checkTable(e.getKey(), e.getValue(), opts, + printInfoMethod, printProblemMethod) || sawProblems; } if (!sawProblems) { - System.out - .println("\n...No problems found in " + tableNameToCheck + " (" + tableCheckId + ")"); + printInfoMethod.accept( + String.format("\n...No problems found in %s (%s)", tableNameToCheck, tableCheckId)); } // end METADATA table sanity check return sawProblems; @@ -194,10 +199,11 @@ public static void main(String[] args) throws Exception { boolean sawProblems; try (Scope scope = span.makeCurrent()) { - sawProblems = checkMetadataAndRootTableEntries(AccumuloTable.ROOT.tableName(), opts); + sawProblems = checkMetadataAndRootTableEntries(AccumuloTable.ROOT.tableName(), opts, + System.out::println, System.out::println); System.out.println(); - sawProblems = - checkMetadataAndRootTableEntries(AccumuloTable.METADATA.tableName(), opts) || sawProblems; + sawProblems = checkMetadataAndRootTableEntries(AccumuloTable.METADATA.tableName(), opts, + System.out::println, System.out::println) || sawProblems; if (sawProblems) { throw new IllegalStateException(); } diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java b/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java index 8564a8065fb..0cfa61bf0dd 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java @@ -21,6 +21,7 @@ import java.util.Iterator; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.data.Range; @@ -55,14 +56,15 @@ public static void main(String[] args) throws Exception { Span span = TraceUtil.startSpan(FindOfflineTablets.class, "main"); try (Scope scope = span.makeCurrent()) { ServerContext context = opts.getServerContext(); - findOffline(context, null, false, false); + findOffline(context, null, false, false, System.out::println, System.out::println); } finally { span.end(); } } public static int findOffline(ServerContext context, String tableName, boolean skipZkScan, - boolean skipRootScan) throws TableNotFoundException { + boolean skipRootScan, Consumer printInfoMethod, Consumer printProblemMethod) + throws TableNotFoundException { final AtomicBoolean scanning = new AtomicBoolean(false); @@ -87,8 +89,8 @@ public void update(LiveTServerSet current, Set deleted, int offline = 0; if (!skipZkScan) { - System.out.println("Scanning zookeeper"); - if ((offline = checkTablets(context, zooScanner, tservers)) > 0) { + printInfoMethod.accept("Scanning zookeeper"); + if ((offline = checkTablets(context, zooScanner, tservers, printProblemMethod)) > 0) { return offline; } } @@ -98,10 +100,10 @@ public void update(LiveTServerSet current, Set deleted, } if (!skipRootScan) { - System.out.println("Scanning " + AccumuloTable.ROOT.tableName()); + printInfoMethod.accept("Scanning " + AccumuloTable.ROOT.tableName()); Iterator rootScanner = new MetaDataTableScanner(context, TabletsSection.getRange(), AccumuloTable.ROOT.tableName()); - if ((offline = checkTablets(context, rootScanner, tservers)) > 0) { + if ((offline = checkTablets(context, rootScanner, tservers, printProblemMethod)) > 0) { return offline; } } @@ -110,7 +112,7 @@ public void update(LiveTServerSet current, Set deleted, return 0; } - System.out.println("Scanning " + AccumuloTable.METADATA.tableName()); + printInfoMethod.accept("Scanning " + AccumuloTable.METADATA.tableName()); Range range = TabletsSection.getRange(); if (tableName != null) { @@ -120,12 +122,12 @@ public void update(LiveTServerSet current, Set deleted, try (MetaDataTableScanner metaScanner = new MetaDataTableScanner(context, range, AccumuloTable.METADATA.tableName())) { - return checkTablets(context, metaScanner, tservers); + return checkTablets(context, metaScanner, tservers, printProblemMethod); } } private static int checkTablets(ServerContext context, Iterator scanner, - LiveTServerSet tservers) { + LiveTServerSet tservers, Consumer printProblemMethod) { int offline = 0; while (scanner.hasNext() && !System.out.checkError()) { @@ -134,8 +136,8 @@ private static int checkTablets(ServerContext context, Iterator processing; private final AtomicReference exceptionRef; + private final Consumer printInfoMethod; + private final Consumer printProblemMethod; CheckFileTask(Map cache, VolumeManager fs, AtomicInteger missing, BatchWriter writer, - Key key, Path map, Set processing, AtomicReference exceptionRef) { + Key key, Path map, Set processing, AtomicReference exceptionRef, + Consumer printInfoMethod, Consumer printProblemMethod) { this.cache = cache; this.fs = fs; this.missing = missing; @@ -85,6 +89,8 @@ private static class CheckFileTask implements Runnable { this.path = map; this.processing = processing; this.exceptionRef = exceptionRef; + this.printInfoMethod = printInfoMethod; + this.printProblemMethod = printProblemMethod; } @Override @@ -101,9 +107,9 @@ public void run() { m.putDelete(key.getColumnFamily(), key.getColumnQualifier()); if (writer != null) { writer.addMutation(m); - System.out.println("Reference " + path + " removed from " + key.getRow()); + printInfoMethod.accept("Reference " + path + " removed from " + key.getRow()); } else { - System.out.println("File " + path + " is missing"); + printProblemMethod.accept("File " + path + " is missing"); } } } catch (Exception e) { @@ -117,15 +123,15 @@ public void run() { } } - private static int checkTable(ServerContext context, String tableName, Range range, boolean fix) - throws Exception { + private static int checkTable(ServerContext context, String tableName, Range range, boolean fix, + Consumer printInfoMethod, Consumer printProblemMethod) throws Exception { Map cache = new LRUMap<>(100000); Set processing = new HashSet<>(); ExecutorService threadPool = ThreadPools.getServerThreadPools() .getPoolBuilder(UTILITY_CHECK_FILE_TASKS).numCoreThreads(16).build(); - System.out.printf("Scanning : %s %s\n", tableName, range); + printInfoMethod.accept(String.format("Scanning : %s %s\n", tableName, range)); VolumeManager fs = context.getVolumeManager(); Scanner metadata = context.createScanner(tableName, Authorizations.EMPTY); @@ -161,8 +167,8 @@ private static int checkTable(ServerContext context, String tableName, Range ran processing.add(map); } - threadPool.execute( - new CheckFileTask(cache, fs, missing, writer, key, map, processing, exceptionRef)); + threadPool.execute(new CheckFileTask(cache, fs, missing, writer, key, map, processing, + exceptionRef, printInfoMethod, printProblemMethod)); } threadPool.shutdown(); @@ -181,33 +187,42 @@ private static int checkTable(ServerContext context, String tableName, Range ran writer.close(); } - System.out.printf("Scan finished, missing files: %d, total files: %d\n", missing.get(), count); + String msg = + String.format("Scan finished, missing files: %d, total files: %d\n", missing.get(), count); + if (missing.get() == 0) { + printInfoMethod.accept(msg); + } else { + printProblemMethod.accept(msg); + } return missing.get(); } - static int checkAllTables(ServerContext context, boolean fix) throws Exception { - int missing = - checkTable(context, AccumuloTable.ROOT.tableName(), TabletsSection.getRange(), fix); + static int checkAllTables(ServerContext context, boolean fix, Consumer printInfoMethod, + Consumer printProblemMethod) throws Exception { + int missing = checkTable(context, AccumuloTable.ROOT.tableName(), TabletsSection.getRange(), + fix, printInfoMethod, printProblemMethod); if (missing == 0) { - return checkTable(context, AccumuloTable.METADATA.tableName(), TabletsSection.getRange(), - fix); + return checkTable(context, AccumuloTable.METADATA.tableName(), TabletsSection.getRange(), fix, + printInfoMethod, printProblemMethod); } else { return missing; } } - public static int checkTable(ServerContext context, String tableName, boolean fix) - throws Exception { + public static int checkTable(ServerContext context, String tableName, boolean fix, + Consumer printInfoMethod, Consumer printProblemMethod) throws Exception { if (tableName.equals(AccumuloTable.ROOT.tableName())) { throw new IllegalArgumentException("Can not check root table"); } else if (tableName.equals(AccumuloTable.METADATA.tableName())) { - return checkTable(context, AccumuloTable.ROOT.tableName(), TabletsSection.getRange(), fix); + return checkTable(context, AccumuloTable.ROOT.tableName(), TabletsSection.getRange(), fix, + printInfoMethod, printProblemMethod); } else { TableId tableId = context.getTableId(tableName); Range range = new KeyExtent(tableId, null, null).toMetaRange(); - return checkTable(context, AccumuloTable.METADATA.tableName(), range, fix); + return checkTable(context, AccumuloTable.METADATA.tableName(), range, fix, printInfoMethod, + printProblemMethod); } } @@ -216,7 +231,7 @@ public static void main(String[] args) throws Exception { opts.parseArgs(RemoveEntriesForMissingFiles.class.getName(), args); Span span = TraceUtil.startSpan(RemoveEntriesForMissingFiles.class, "main"); try (Scope scope = span.makeCurrent()) { - checkAllTables(opts.getServerContext(), opts.fix); + checkAllTables(opts.getServerContext(), opts.fix, System.out::println, System.out::println); } finally { span.end(); } diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/CheckRunner.java b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/CheckRunner.java index 7da66dc156a..d1a1ababb4d 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/CheckRunner.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/CheckRunner.java @@ -21,8 +21,11 @@ import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.cli.ServerUtilOpts; import org.apache.accumulo.server.util.Admin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public interface CheckRunner { + Logger log = LoggerFactory.getLogger(CheckRunner.class); /** * Runs the check @@ -44,17 +47,15 @@ Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtilOpts op default void printRunning() { String running = "Running check " + getCheck(); - System.out.println(); - System.out.println("-".repeat(running.length())); - System.out.println(running); - System.out.println("-".repeat(running.length())); + log.trace("-".repeat(running.length())); + log.trace(running); + log.trace("-".repeat(running.length())); } default void printCompleted(Admin.CheckCommand.CheckStatus status) { String completed = "Check " + getCheck() + " completed with status " + status; - System.out.println(); - System.out.println("-".repeat(completed.length())); - System.out.println(completed); - System.out.println("-".repeat(completed.length())); + log.trace("-".repeat(completed.length())); + log.trace(completed); + log.trace("-".repeat(completed.length())); } } 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 4756f7f14fb..dd1b8d525d6 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 @@ -70,11 +70,12 @@ default Admin.CheckCommand.CheckStatus checkRequiredColumns(ServerContext contex Set requiredColFams; boolean missingReqCol = false; - System.out.printf("Scanning the %s for missing required columns...\n", scanning()); + log.trace("Scanning the {} for missing required columns...\n", scanning()); try (Scanner scanner = context.createScanner(tableName(), Authorizations.EMPTY)) { var is = new IteratorSetting(100, "tablets", WholeRowIterator.class); scanner.addScanIterator(is); scanner.setRange(MetadataSchema.TabletsSection.getRange()); + fetchRequiredColumns(scanner); for (var entry : scanner) { requiredColFQs = new HashSet<>(requiredColFQs()); requiredColFams = new HashSet<>(requiredColFams()); @@ -93,8 +94,7 @@ default Admin.CheckCommand.CheckStatus checkRequiredColumns(ServerContext contex } } if (!requiredColFQs.isEmpty() || !requiredColFams.isEmpty()) { - System.out.printf( - "Tablet %s is missing required columns: col FQs: %s, col fams: %s in the %s\n", + log.warn("Tablet {} is missing required columns: col FQs: {}, col fams: {} in the {}\n", entry.getKey().getRow(), requiredColFQs, requiredColFams, scanning()); status = Admin.CheckCommand.CheckStatus.FAILED; missingReqCol = true; @@ -103,7 +103,7 @@ default Admin.CheckCommand.CheckStatus checkRequiredColumns(ServerContext contex } if (!missingReqCol) { - System.out.printf("...The %s contains all required columns for all tablets\n", scanning()); + log.trace("...The {} contains all required columns for all tablets\n", scanning()); } return status; } @@ -114,11 +114,11 @@ default Admin.CheckCommand.CheckStatus checkRequiredColumns(ServerContext contex */ default Admin.CheckCommand.CheckStatus checkColumns(ServerContext context, Iterator> iter, - Admin.CheckCommand.CheckStatus status) - throws TableNotFoundException, InterruptedException, KeeperException { + Admin.CheckCommand.CheckStatus status) { boolean invalidCol = false; + MetadataConstraints mc = new MetadataConstraints(); - System.out.printf("Scanning the %s for invalid columns...\n", scanning()); + log.trace("Scanning the {} for invalid columns...\n", scanning()); while (iter.hasNext()) { var entry = iter.next(); Key key = entry.getKey(); @@ -127,22 +127,29 @@ default Admin.CheckCommand.CheckStatus checkColumns(ServerContext context, m.at().family(key.getColumnFamily()).qualifier(key.getColumnQualifier()) .visibility(key.getColumnVisibility()).timestamp(key.getTimestamp()) .put(entry.getValue()); - MetadataConstraints mc = new MetadataConstraints(); var violations = mc.check(new ConstraintEnv(context), m); if (!violations.isEmpty()) { - violations.forEach( - violationCode -> System.out.println(mc.getViolationDescription(violationCode))); + violations.forEach(violationCode -> log.warn(mc.getViolationDescription(violationCode))); status = Admin.CheckCommand.CheckStatus.FAILED; invalidCol = true; } } if (!invalidCol) { - System.out.printf("...All columns in the %s are valid\n", scanning()); + log.trace("...All columns in the {} are valid\n", scanning()); } return status; } + default void fetchRequiredColumns(Scanner scanner) { + for (var reqColFQ : requiredColFQs()) { + scanner.fetchColumn(reqColFQ.getColumnFamily(), reqColFQ.getColumnQualifier()); + } + for (var reqColFam : requiredColFams()) { + scanner.fetchColumnFamily(reqColFam); + } + } + /** * A {@link SystemEnvironment} whose only valid operation is * {@link ConstraintEnv#getServerContext()} diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/MetadataTableCheckRunner.java b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/MetadataTableCheckRunner.java index 87c1a10a374..6e9d8bbf25b 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/MetadataTableCheckRunner.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/MetadataTableCheckRunner.java @@ -65,22 +65,23 @@ public Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtil Admin.CheckCommand.CheckStatus status = Admin.CheckCommand.CheckStatus.OK; printRunning(); - System.out.println("\n********** Looking for offline tablets **********\n"); - if (FindOfflineTablets.findOffline(context, null, true, true) != 0) { + log.trace("********** Looking for offline tablets **********"); + if (FindOfflineTablets.findOffline(context, null, true, true, log::trace, log::warn) != 0) { status = Admin.CheckCommand.CheckStatus.FAILED; } else { - System.out.println("All good... No offline tablets found"); + log.trace("All good... No offline tablets found"); } - System.out.println("\n********** Checking some references **********\n"); - if (CheckForMetadataProblems.checkMetadataAndRootTableEntries(tableName(), opts)) { + log.trace("********** Checking some references **********"); + if (CheckForMetadataProblems.checkMetadataAndRootTableEntries(tableName(), opts, log::trace, + log::warn)) { status = Admin.CheckCommand.CheckStatus.FAILED; } - System.out.println("\n********** Looking for missing columns **********\n"); + log.trace("********** Looking for missing columns **********"); status = checkRequiredColumns(context, status); - System.out.println("\n********** Looking for invalid columns **********\n"); + log.trace("********** Looking for invalid columns **********"); try (Scanner scanner = context.createScanner(tableName(), Authorizations.EMPTY)) { status = checkColumns(context, scanner.stream().map(AbstractMap.SimpleImmutableEntry::new).iterator(), status); 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 bef1c8173a9..3b4cd1112b1 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 @@ -74,17 +74,18 @@ public Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtil Admin.CheckCommand.CheckStatus status = Admin.CheckCommand.CheckStatus.OK; printRunning(); - System.out.println("\n********** Looking for offline tablets **********\n"); - if (FindOfflineTablets.findOffline(context, AccumuloTable.ROOT.tableName(), false, true) != 0) { + log.trace("********** Looking for offline tablets **********"); + if (FindOfflineTablets.findOffline(context, AccumuloTable.ROOT.tableName(), false, true, + log::trace, log::warn) != 0) { status = Admin.CheckCommand.CheckStatus.FAILED; } else { - System.out.println("All good... No offline tablets found"); + log.trace("All good... No offline tablets found"); } - System.out.println("\n********** Looking for missing columns **********\n"); + log.trace("********** Looking for missing columns **********"); status = checkRequiredColumns(context, status); - System.out.println("\n********** Looking for invalid columns **********\n"); + log.trace("********** Looking for invalid columns **********"); final String path = context.getZooKeeperRoot() + RootTable.ZROOT_TABLET; final String json = new String(context.getZooReader().getData(path), UTF_8); final var rtm = new RootTabletMetadata(json); @@ -105,7 +106,7 @@ public Admin.CheckCommand.CheckStatus checkRequiredColumns(ServerContext context final Set requiredColFQs = new HashSet<>(requiredColFQs()); final Set requiredColFams = new HashSet<>(requiredColFams()); - System.out.printf("Scanning the %s for missing required columns...\n", scanning()); + log.trace("Scanning the {} for missing required columns...\n", scanning()); rtm.toKeyValues().forEach(e -> { var key = e.getKey(); rowsSeen.add(key.getRow()); @@ -118,17 +119,15 @@ public Admin.CheckCommand.CheckStatus checkRequiredColumns(ServerContext context if (rowsSeen.size() != 1) { status = Admin.CheckCommand.CheckStatus.FAILED; - System.out.println("Did not see one tablet for the root table!"); + log.warn("Did not see one tablet for the root table!"); } else { if (!requiredColFQs.isEmpty() || !requiredColFams.isEmpty()) { - System.out.printf( - "Tablet %s is missing required columns: col FQs: %s, col fams: %s in the %s\n", + log.warn("Tablet {} is missing required columns: col FQs: {}, col fams: {} in the {}\n", rowsSeen.stream().findFirst().orElseThrow(), requiredColFQs, requiredColFams, scanning()); status = Admin.CheckCommand.CheckStatus.FAILED; } else { - System.out.printf("...The %s contains all required columns for the root tablet\n", - scanning()); + log.trace("...The {} contains all required columns for the root tablet\n", scanning()); } } 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 f4e790ab2a4..fe7fde801b5 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 @@ -66,23 +66,24 @@ public Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtil Admin.CheckCommand.CheckStatus status = Admin.CheckCommand.CheckStatus.OK; printRunning(); - System.out.println("\n********** Looking for offline tablets **********\n"); - if (FindOfflineTablets.findOffline(context, AccumuloTable.METADATA.tableName(), true, false) - != 0) { + log.trace("********** Looking for offline tablets **********"); + if (FindOfflineTablets.findOffline(context, AccumuloTable.METADATA.tableName(), true, false, + log::trace, log::warn) != 0) { status = Admin.CheckCommand.CheckStatus.FAILED; } else { - System.out.println("All good... No offline tablets found"); + log.trace("All good... No offline tablets found"); } - System.out.println("\n********** Checking some references **********\n"); - if (CheckForMetadataProblems.checkMetadataAndRootTableEntries(tableName(), opts)) { + log.trace("********** Checking some references **********"); + if (CheckForMetadataProblems.checkMetadataAndRootTableEntries(tableName(), opts, log::trace, + log::warn)) { status = Admin.CheckCommand.CheckStatus.FAILED; } - System.out.println("\n********** Looking for missing columns **********\n"); + log.trace("********** Looking for missing columns **********"); status = checkRequiredColumns(context, status); - System.out.println("\n********** Looking for invalid columns **********\n"); + log.trace("********** Looking for invalid columns **********"); try (Scanner scanner = context.createScanner(tableName(), Authorizations.EMPTY)) { status = checkColumns(context, scanner.stream().map(AbstractMap.SimpleImmutableEntry::new).iterator(), status); 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 0a2e799b288..9e8467eb7bf 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,20 +18,9 @@ */ package org.apache.accumulo.server.util.checkCommand; -import java.util.List; -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.fate.zookeeper.ZooReaderWriter; -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 SystemConfigCheckRunner implements CheckRunner { private static final Admin.CheckCommand.Check check = Admin.CheckCommand.Check.SYSTEM_CONFIG; @@ -41,71 +30,10 @@ public Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtil boolean fixFiles) throws Exception { Admin.CheckCommand.CheckStatus status = Admin.CheckCommand.CheckStatus.OK; printRunning(); - - System.out.println("\n********** Checking some references **********\n"); - status = checkTableLocks(context, status); - printCompleted(status); return status; } - private static Admin.CheckCommand.CheckStatus checkTableLocks(ServerContext context, - Admin.CheckCommand.CheckStatus status) - throws InterruptedException, KeeperException, AccumuloException, AccumuloSecurityException { - final AdminUtil admin = new AdminUtil<>(true); - final String zkRoot = context.getZooKeeperRoot(); - final var zTableLocksPath = ServiceLock.path(zkRoot + Constants.ZTABLE_LOCKS); - final String fateZkPath = zkRoot + Constants.ZFATE; - final ZooReaderWriter zk = context.getZooReaderWriter(); - final ZooStore zs = new ZooStore<>(fateZkPath, zk); - - System.out.println("Ensuring table and namespace locks are valid..."); - - var tableIds = context.tableOperations().tableIdMap().values(); - var namespaceIds = context.namespaceOperations().namespaceIdMap().values(); - List lockedIds = zk.getChildren(zTableLocksPath.toString()); - boolean locksExist = !lockedIds.isEmpty(); - - if (locksExist) { - lockedIds.removeAll(tableIds); - lockedIds.removeAll(namespaceIds); - if (!lockedIds.isEmpty()) { - status = Admin.CheckCommand.CheckStatus.FAILED; - System.out - .println("...Some table and namespace locks are INVALID (the table/namespace DNE): " - + lockedIds); - } else { - System.out.println("...locks are valid"); - } - } else { - System.out.println("...no locks present"); - } - - System.out.println("Ensuring table and namespace locks are associated with a FATE op..."); - - if (locksExist) { - final var fateStatus = admin.getStatus(zs, zk, zTableLocksPath, null, null); - if (!fateStatus.getDanglingHeldLocks().isEmpty() - || !fateStatus.getDanglingWaitingLocks().isEmpty()) { - status = Admin.CheckCommand.CheckStatus.FAILED; - System.out.println("The following locks did not have an associated FATE operation\n"); - for (Map.Entry> entry : fateStatus.getDanglingHeldLocks().entrySet()) { - System.out.println("txid: " + entry.getKey() + " locked: " + entry.getValue()); - } - for (Map.Entry> entry : fateStatus.getDanglingWaitingLocks() - .entrySet()) { - System.out.println("txid: " + entry.getKey() + " locking: " + entry.getValue()); - } - } else { - System.out.println("...locks are valid"); - } - } else { - System.out.println("...no locks present"); - } - - return status; - } - @Override public Admin.CheckCommand.Check getCheck() { return check; diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/SystemFilesCheckRunner.java b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/SystemFilesCheckRunner.java index a7c97b6c79c..2240b94a5bb 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/SystemFilesCheckRunner.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/SystemFilesCheckRunner.java @@ -33,9 +33,9 @@ public Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtil Admin.CheckCommand.CheckStatus status = Admin.CheckCommand.CheckStatus.OK; printRunning(); - System.out.println("\n********** Looking for missing system files **********\n"); + log.trace("********** Looking for missing system files **********"); if (RemoveEntriesForMissingFiles.checkTable(context, AccumuloTable.METADATA.tableName(), - fixFiles) != 0) { + fixFiles, log::trace, log::warn) != 0) { status = Admin.CheckCommand.CheckStatus.FAILED; } 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 new file mode 100644 index 00000000000..9161c9867da --- /dev/null +++ b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/TableLocksCheckRunner.java @@ -0,0 +1,112 @@ +/* + * 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.List; +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.fate.zookeeper.ZooReaderWriter; +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; + + @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 some references **********"); + status = checkTableLocks(context, status); + + printCompleted(status); + return status; + } + + @Override + public Admin.CheckCommand.Check getCheck() { + return check; + } + + private static Admin.CheckCommand.CheckStatus checkTableLocks(ServerContext context, + Admin.CheckCommand.CheckStatus status) + throws InterruptedException, KeeperException, AccumuloException, AccumuloSecurityException { + final AdminUtil admin = new AdminUtil<>(true); + final String zkRoot = context.getZooKeeperRoot(); + final var zTableLocksPath = ServiceLock.path(zkRoot + Constants.ZTABLE_LOCKS); + final String fateZkPath = zkRoot + Constants.ZFATE; + final ZooReaderWriter zk = context.getZooReaderWriter(); + final ZooStore zs = new ZooStore<>(fateZkPath, zk); + + log.trace("Ensuring table and namespace locks are valid..."); + + var tableIds = context.tableOperations().tableIdMap().values(); + var namespaceIds = context.namespaceOperations().namespaceIdMap().values(); + List lockedIds = zk.getChildren(zTableLocksPath.toString()); + boolean locksExist = !lockedIds.isEmpty(); + + if (locksExist) { + lockedIds.removeAll(tableIds); + lockedIds.removeAll(namespaceIds); + if (!lockedIds.isEmpty()) { + status = Admin.CheckCommand.CheckStatus.FAILED; + log.warn("...Some table and namespace locks are INVALID (the table/namespace DNE): " + + lockedIds); + } else { + log.trace("...locks are valid"); + } + } else { + log.trace("...no locks present"); + } + + log.trace("Ensuring table and namespace locks are associated with a FATE op..."); + + if (locksExist) { + final var fateStatus = admin.getStatus(zs, zk, zTableLocksPath, null, null); + if (!fateStatus.getDanglingHeldLocks().isEmpty() + || !fateStatus.getDanglingWaitingLocks().isEmpty()) { + status = Admin.CheckCommand.CheckStatus.FAILED; + log.warn("The following locks did not have an associated FATE operation\n"); + for (Map.Entry> entry : fateStatus.getDanglingHeldLocks().entrySet()) { + log.warn("txid: " + entry.getKey() + " locked: " + entry.getValue()); + } + for (Map.Entry> entry : fateStatus.getDanglingWaitingLocks() + .entrySet()) { + log.warn("txid: " + entry.getKey() + " locking: " + entry.getValue()); + } + } else { + log.trace("...locks are valid"); + } + } else { + log.trace("...no locks present"); + } + + return status; + } +} diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/UserFilesCheckRunner.java b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/UserFilesCheckRunner.java index a5570b3c472..e89a1f372c0 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/UserFilesCheckRunner.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/UserFilesCheckRunner.java @@ -35,12 +35,13 @@ public Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtil Admin.CheckCommand.CheckStatus status = Admin.CheckCommand.CheckStatus.OK; printRunning(); - System.out.println("\n********** Looking for missing user files **********\n"); + log.trace("********** Looking for missing user files **********"); for (String tableName : context.tableOperations().list()) { var tableId = context.getTableId(tableName); if (!AccumuloTable.allTableIds().contains(context.getTableId(tableName))) { - System.out.printf("Checking table %s (%s) for missing files\n", tableName, tableId); - if (RemoveEntriesForMissingFiles.checkTable(context, tableName, fixFiles) != 0) { + log.trace("Checking table {} ({}) for missing files\n", tableName, tableId); + if (RemoveEntriesForMissingFiles.checkTable(context, tableName, fixFiles, log::trace, + log::warn) != 0) { status = Admin.CheckCommand.CheckStatus.FAILED; } } 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 73dde21e730..8e32f1c89a2 100644 --- a/test/src/main/java/org/apache/accumulo/test/AdminCheckIT.java +++ b/test/src/main/java/org/apache/accumulo/test/AdminCheckIT.java @@ -19,6 +19,7 @@ package org.apache.accumulo.test; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -30,6 +31,7 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.TreeMap; import java.util.function.Supplier; import java.util.regex.Pattern; @@ -51,6 +53,7 @@ import org.junit.jupiter.api.Test; import com.beust.jcommander.JCommander; +import com.google.common.collect.Sets; public class AdminCheckIT extends ConfigurableMacBase { private static final PrintStream ORIGINAL_OUT = System.out; @@ -149,6 +152,9 @@ public void testAdminCheckListAndRunInvalidArgs() throws Exception { @Test public void testAdminCheckRunNoCheckFailures() { // tests running the checks with none failing on run + Admin.CheckCommand.Check rootTableCheck = Admin.CheckCommand.Check.ROOT_TABLE; + Admin.CheckCommand.Check systemFilesCheck = Admin.CheckCommand.Check.SYSTEM_FILES; + Admin.CheckCommand.Check userFilesCheck = Admin.CheckCommand.Check.USER_FILES; boolean[] allChecksPass = new boolean[Admin.CheckCommand.Check.values().length]; Arrays.fill(allChecksPass, true); @@ -167,14 +173,15 @@ public void testAdminCheckRunNoCheckFailures() { String out3 = executeCheckCommand(new String[] {"check", "run", "-p", "[A-Z]+_[A-Z]+"}, allChecksPass); // run subset of checks - String out4 = executeCheckCommand( - new String[] {"check", "run", "ROOT_TABLE", "SYSTEM_FILES", "USER_FILES"}, allChecksPass); + String out4 = executeCheckCommand(new String[] {"check", "run", rootTableCheck.name(), + systemFilesCheck.name(), userFilesCheck.name()}, allChecksPass); // run same subset of checks but using a pattern to specify the checks (case shouldn't matter) String out5 = executeCheckCommand(new String[] {"check", "run", "-p", "ROOT_TABLE|.*files"}, allChecksPass); String expRunAllRunOrder = "Running dummy check SYSTEM_CONFIG\nDummy check SYSTEM_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" + "Running dummy check METADATA_TABLE\nDummy check METADATA_TABLE completed with status OK\n" @@ -186,10 +193,11 @@ 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|OKROOT_METADATA|OKROOT_TABLE|OK" + String expRunAllStatusInfo = "-SYSTEM_CONFIG|OKTABLE_LOCKS|OKROOT_METADATA|OKROOT_TABLE|OK" + "METADATA_TABLE|OKSYSTEM_FILES|OKUSER_FILES|OK-"; - String expRunSubStatusInfo = "-SYSTEM_CONFIG|FILTERED_OUTROOT_METADATA|FILTERED_OUT" - + "ROOT_TABLE|OKMETADATA_TABLE|FILTERED_OUTSYSTEM_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-"; assertTrue(out1.contains(expRunAllRunOrder)); assertTrue(out2.contains(expRunAllRunOrder)); @@ -197,6 +205,10 @@ public void testAdminCheckRunNoCheckFailures() { assertTrue(out4.contains(expRunSubRunOrder)); assertTrue(out5.contains(expRunSubRunOrder)); + assertNoOtherChecksRan(out4, true, rootTableCheck, systemFilesCheck, userFilesCheck); + assertNoOtherChecksRan(out5, true, rootTableCheck, systemFilesCheck, userFilesCheck); + // no need to check out1-3 above, those should run all + out1 = out1.replaceAll("\\s+", ""); out2 = out2.replaceAll("\\s+", ""); out3 = out3.replaceAll("\\s+", ""); @@ -214,9 +226,10 @@ public void testAdminCheckRunNoCheckFailures() { public void testAdminCheckRunWithCheckFailures() { // tests running checks with some failing - boolean[] rootTableFails = new boolean[] {true, true, false, true, true, true}; - boolean[] systemConfigFails = new boolean[] {false, true, true, true, true, true}; - boolean[] userFilesAndMetadataTableFails = new boolean[] {true, true, true, false, true, false}; + boolean[] rootTableFails = new boolean[] {true, true, true, false, true, true, true}; + boolean[] systemConfigFails = new boolean[] {false, true, true, true, true, true, true}; + boolean[] userFilesAndMetadataTableFails = + new boolean[] {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 @@ -238,6 +251,7 @@ public void testAdminCheckRunWithCheckFailures() { String expRunOrder1 = "Running dummy check SYSTEM_CONFIG\nDummy check SYSTEM_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"; String expRunOrder2 = @@ -252,19 +266,30 @@ public void testAdminCheckRunWithCheckFailures() { assertTrue(out3.contains(expRunOrder3And4)); 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); + 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); + assertNoOtherChecksRan(out4, true, Admin.CheckCommand.Check.SYSTEM_CONFIG, + Admin.CheckCommand.Check.ROOT_TABLE, Admin.CheckCommand.Check.USER_FILES); + out1 = out1.replaceAll("\\s+", ""); out2 = out2.replaceAll("\\s+", ""); out3 = out3.replaceAll("\\s+", ""); out4 = out4.replaceAll("\\s+", ""); - String expStatusInfo1 = "-SYSTEM_CONFIG|OKROOT_METADATA|OKROOT_TABLE|FAILED" + 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 expStatusInfo2 = "-SYSTEM_CONFIG|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|OKROOT_METADATA|FILTERED_OUTROOT_TABLE|OK" - + "METADATA_TABLE|FILTERED_OUTSYSTEM_FILES|FILTERED_OUTUSER_FILES|FAILED"; + String expStatusInfo3And4 = "-SYSTEM_CONFIG|OKTABLE_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)); @@ -278,8 +303,10 @@ public void testAdminCheckRunWithCheckFailures() { */ @Test - public void testSystemConfigCheck() throws Exception { + public void testPassingTableLocksCheck() throws Exception { + // Tests the TABLE_LOCKS check in the case where all checks pass String table = getUniqueNames(1)[0]; + Admin.CheckCommand.Check tableLocksCheck = Admin.CheckCommand.Check.TABLE_LOCKS; try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) { client.tableOperations().create(table); @@ -296,20 +323,22 @@ public void testSystemConfigCheck() throws Exception { slowCompaction.setIterators(List.of(is)); client.tableOperations().compact(table, slowCompaction); - var p = getCluster().exec(Admin.class, "check", "run", "system_config"); + 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 SYSTEM_CONFIG completed with status OK")); + assertTrue(out.contains("Check TABLE_LOCKS completed with status OK")); + assertNoOtherChecksRan(out, false, tableLocksCheck); } } @Test public void testPassingMetadataTableCheck() throws Exception { // Tests the METADATA_TABLE check in the case where all checks pass + Admin.CheckCommand.Check metaTableCheck = Admin.CheckCommand.Check.METADATA_TABLE; // no extra setup needed, just check the metadata table - var p = getCluster().exec(Admin.class, "check", "run", "metadata_table"); + var p = getCluster().exec(Admin.class, "check", "run", metaTableCheck.name()); assertEquals(0, p.getProcess().waitFor()); String out = p.readStdOut(); assertTrue(out.contains("Looking for offline tablets")); @@ -317,14 +346,16 @@ public void testPassingMetadataTableCheck() throws Exception { assertTrue(out.contains("Looking for missing columns")); assertTrue(out.contains("Looking for invalid columns")); assertTrue(out.contains("Check METADATA_TABLE completed with status OK")); + assertNoOtherChecksRan(out, false, metaTableCheck); } @Test public void testPassingRootTableCheck() throws Exception { // Tests the ROOT_TABLE check in the case where all checks pass + Admin.CheckCommand.Check rootTableCheck = Admin.CheckCommand.Check.ROOT_TABLE; // no extra setup needed, just check the root table - var p = getCluster().exec(Admin.class, "check", "run", "root_table"); + var p = getCluster().exec(Admin.class, "check", "run", rootTableCheck.name()); assertEquals(0, p.getProcess().waitFor()); String out = p.readStdOut(); assertTrue(out.contains("Looking for offline tablets")); @@ -332,36 +363,42 @@ public void testPassingRootTableCheck() throws Exception { assertTrue(out.contains("Looking for missing columns")); assertTrue(out.contains("Looking for invalid columns")); assertTrue(out.contains("Check ROOT_TABLE completed with status OK")); + assertNoOtherChecksRan(out, false, rootTableCheck); } @Test public void testPassingRootMetadataCheck() throws Exception { // Tests the ROOT_TABLE check in the case where all checks pass + Admin.CheckCommand.Check rootMetaCheck = Admin.CheckCommand.Check.ROOT_METADATA; // no extra setup needed, just check the root table metadata - var p = getCluster().exec(Admin.class, "check", "run", "root_metadata"); + var p = getCluster().exec(Admin.class, "check", "run", rootMetaCheck.name()); assertEquals(0, p.getProcess().waitFor()); String out = p.readStdOut(); assertTrue(out.contains("Looking for offline tablets")); assertTrue(out.contains("Looking for missing columns")); assertTrue(out.contains("Looking for invalid columns")); assertTrue(out.contains("Check ROOT_METADATA completed with status OK")); + assertNoOtherChecksRan(out, false, rootMetaCheck); } @Test public void testPassingSystemFilesCheck() throws Exception { // Tests the SYSTEM_FILES check in the case where it should pass + Admin.CheckCommand.Check sysFilesCheck = Admin.CheckCommand.Check.SYSTEM_FILES; // no extra setup needed, just run the check - var p = getCluster().exec(Admin.class, "check", "run", "system_files"); + 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()); + assertNoOtherChecksRan(out, false, sysFilesCheck); } @Test public void testPassingUserFilesCheck() throws Exception { // Tests the USER_FILES check in the case where it should pass + Admin.CheckCommand.Check userFilesCheck = Admin.CheckCommand.Check.USER_FILES; try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) { // create a table, insert some data, and flush so there's a file to check @@ -370,10 +407,11 @@ public void testPassingUserFilesCheck() throws Exception { ReadWriteIT.ingest(client, 10, 10, 10, 0, table); client.tableOperations().flush(table, null, null, true); - var p = getCluster().exec(Admin.class, "check", "run", "user_files"); + 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()); + assertNoOtherChecksRan(out, false, userFilesCheck); } } @@ -416,6 +454,19 @@ private Admin createMockAdmin(boolean[] checksPass) { return admin; } + /** + * Asserts that no checks (other than those provided) ran. + */ + private void assertNoOtherChecksRan(String out, boolean isDummyCheck, + Admin.CheckCommand.Check... checks) { + Set otherChecks = + Sets.difference(Set.of(Admin.CheckCommand.Check.values()), Set.of(checks)); + for (var check : otherChecks) { + assertFalse( + out.contains("Running " + (isDummyCheck ? "dummy " : "") + "check " + check.name())); + } + } + static abstract class DummyCheckRunner implements CheckRunner { private final boolean passes; @@ -447,6 +498,17 @@ public Admin.CheckCommand.Check getCheck() { } } + static class DummyTableLocksCheckRunner extends DummyCheckRunner { + public DummyTableLocksCheckRunner(boolean passes) { + super(passes); + } + + @Override + public Admin.CheckCommand.Check getCheck() { + return Admin.CheckCommand.Check.TABLE_LOCKS; + } + } + static class DummyRootMetadataCheckRunner extends DummyCheckRunner { public DummyRootMetadataCheckRunner(boolean passes) { super(passes); @@ -509,14 +571,15 @@ 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.ROOT_METADATA, - () -> new DummyRootMetadataCheckRunner(checksPass[1])); - this.checkRunners.put(Check.ROOT_TABLE, () -> new DummyRootTableCheckRunner(checksPass[2])); + () -> new DummyRootMetadataCheckRunner(checksPass[2])); + this.checkRunners.put(Check.ROOT_TABLE, () -> new DummyRootTableCheckRunner(checksPass[3])); this.checkRunners.put(Check.METADATA_TABLE, - () -> new DummyMetadataTableCheckRunner(checksPass[3])); + () -> new DummyMetadataTableCheckRunner(checksPass[4])); this.checkRunners.put(Check.SYSTEM_FILES, - () -> new DummySystemFilesCheckRunner(checksPass[4])); - this.checkRunners.put(Check.USER_FILES, () -> new DummyUserFilesCheckRunner(checksPass[5])); + () -> new DummySystemFilesCheckRunner(checksPass[5])); + this.checkRunners.put(Check.USER_FILES, () -> new DummyUserFilesCheckRunner(checksPass[6])); } @Override diff --git a/test/src/main/resources/log4j2-test.properties b/test/src/main/resources/log4j2-test.properties index 23da7cd1e89..fa8c7864ba9 100644 --- a/test/src/main/resources/log4j2-test.properties +++ b/test/src/main/resources/log4j2-test.properties @@ -142,5 +142,8 @@ logger.38.level = debug logger.39.name = org.apache.accumulo.manager.Manager logger.39.level = trace +logger.40.name = org.apache.accumulo.server.util.checkCommand.CheckRunner +logger.40.level = trace + rootLogger.level = debug rootLogger.appenderRef.console.ref = STDOUT