From eabd7e9137c1b4052060fe1a3edc273b103cec7f Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Sun, 2 Feb 2025 14:06:20 -0500 Subject: [PATCH] wip --- .../org/apache/accumulo/core/fate/Fate.java | 1 + .../manager/merge/FindMergeableRangeTask.java | 184 ++++++++++++++++++ .../manager/tableOps/merge/CountFiles.java | 17 +- .../manager/tableOps/merge/MergeInfo.java | 10 +- .../tableOps/merge/ReserveTablets.java | 10 +- .../manager/tableOps/merge/TableRangeOp.java | 3 +- .../tableOps/merge/UnreserveAndError.java | 3 +- .../tableOps/merge/VerifyMergeability.java | 101 ++++++++++ .../accumulo/test/ample/TestAmpleUtil.java | 13 ++ .../accumulo/test/fate/ManagerRepoIT.java | 46 ++++- 10 files changed, 376 insertions(+), 12 deletions(-) create mode 100644 server/manager/src/main/java/org/apache/accumulo/manager/merge/FindMergeableRangeTask.java create mode 100644 server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/VerifyMergeability.java diff --git a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java index 566dc5e9eed..c58dcfd5bb3 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java @@ -106,6 +106,7 @@ public enum FateOperation { NAMESPACE_RENAME(TFateOperation.NAMESPACE_RENAME), SHUTDOWN_TSERVER(null), SYSTEM_SPLIT(null), + SYSTEM_MERGE(null), TABLE_BULK_IMPORT2(TFateOperation.TABLE_BULK_IMPORT2), TABLE_CANCEL_COMPACT(TFateOperation.TABLE_CANCEL_COMPACT), TABLE_CLONE(TFateOperation.TABLE_CLONE), diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/merge/FindMergeableRangeTask.java b/server/manager/src/main/java/org/apache/accumulo/manager/merge/FindMergeableRangeTask.java new file mode 100644 index 00000000000..713a241df60 --- /dev/null +++ b/server/manager/src/main/java/org/apache/accumulo/manager/merge/FindMergeableRangeTask.java @@ -0,0 +1,184 @@ +/* + * 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.manager.merge; + +import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.FILES; +import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.MERGEABILITY; + +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; + +import org.apache.accumulo.core.clientImpl.thrift.TableOperation; +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.core.data.NamespaceId; +import org.apache.accumulo.core.data.TableId; +import org.apache.accumulo.core.fate.Fate.FateOperation; +import org.apache.accumulo.core.fate.FateInstanceType; +import org.apache.accumulo.core.metadata.schema.TabletMetadata; +import org.apache.accumulo.core.util.time.SteadyTime; +import org.apache.accumulo.manager.Manager; +import org.apache.accumulo.manager.tableOps.TraceRepo; +import org.apache.accumulo.manager.tableOps.merge.MergeInfo; +import org.apache.accumulo.manager.tableOps.merge.TableRangeOp; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.io.Text; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class FindMergeableRangeTask implements Runnable { + + private static final Logger log = LoggerFactory.getLogger(FindMergeableRangeTask.class); + private final Manager manager; + + public FindMergeableRangeTask(Manager manager) { + this.manager = Objects.requireNonNull(manager); + } + + @Override + public void run() { + var context = manager.getContext(); + Map tables = context.getTableIdToNameMap(); + + for (Entry table : tables.entrySet()) { + TableId tableId = table.getKey(); + String tableName = table.getValue(); + + long maxFileCount = + context.getTableConfiguration(tableId).getCount(Property.TABLE_MERGE_FILE_MAX); + long threshold = + context.getTableConfiguration(tableId).getAsBytes(Property.TABLE_SPLIT_THRESHOLD); + double mergeabilityThreshold = .25; + long maxTotalSize = (long) (threshold * mergeabilityThreshold); + + try { + NamespaceId namespaceId = context.getNamespaceId(tableId); + var type = FateInstanceType.fromTableId(tableId); + + try (var tablets = context.getAmple().readTablets().forTable(tableId) + .fetch(FILES, MERGEABILITY).checkConsistency().build()) { + + final MergeableRange current = + new MergeableRange(manager.getSteadyTime(), maxFileCount, maxTotalSize); + + for (var tabletMetadata : tablets) { + + if (!current.add(tabletMetadata)) { + submit(current, type, table, namespaceId); + current.resetAndAdd(tabletMetadata); + } + + // log.debug("{} found {} files in the merge range, maxFiles is {}", fateId, + // totalFiles, + // maxFiles); + // mergeable = false; + // } + } + + submit(current, type, table, namespaceId); + } + + } catch (Exception e) { + log.error("Failed to generate system merges for {}", tableName, e); + } + } + + } + + void submit(MergeableRange range, FateInstanceType type, Entry table, + NamespaceId namespaceId) { + if (range.tabletCount < 2) { + return; + } + + TableId tableId = table.getKey(); + String tableName = table.getValue(); + + String startRowStr = StringUtils.defaultIfBlank(range.startRow.toString(), "-inf"); + String endRowStr = StringUtils.defaultIfBlank(range.startRow.toString(), "+inf"); + log.debug("Creating merge op: {} from startRow: {} to endRow: {}", tableId, startRowStr, + endRowStr); + var fateId = manager.fate(type).startTransaction(); + String goalMessage = TableOperation.MERGE + " Merge table " + tableName + "(" + tableId + + ") splits from " + startRowStr + " to " + endRowStr; + + manager.fate(type).seedTransaction(FateOperation.SYSTEM_MERGE, fateId, + new TraceRepo<>(new TableRangeOp(MergeInfo.Operation.MERGE, namespaceId, tableId, + range.startRow, range.endRow)), + true, goalMessage); + } + + static class MergeableRange { + final SteadyTime currentTime; + final long maxFileCount; + final long maxTotalSize; + + Text startRow; + Text endRow; + int tabletCount; + long totalFileCount = 0; + long totalFileSize = 0; + + MergeableRange(SteadyTime currentTime, long maxFileCount, long maxTotalSize) { + this.currentTime = currentTime; + this.maxFileCount = maxFileCount; + this.maxTotalSize = maxTotalSize; + } + + boolean add(TabletMetadata tm) { + if (validate(tm)) { + tabletCount++; + if (startRow == null) { + startRow = tm.getEndRow(); + } else { + endRow = tm.getEndRow(); + } + totalFileCount += tm.getFiles().size(); + totalFileSize += tm.getFileSize(); + return true; + } + return false; + } + + private boolean validate(TabletMetadata tm) { + if (!tm.getTabletMergeability().isMergeable(currentTime)) { + return false; + } + + if (totalFileCount + tm.getFiles().size() > maxFileCount) { + return false; + } + + return totalFileSize + tm.getFileSize() <= maxTotalSize; + } + + void resetAndAdd(TabletMetadata tm) { + reset(); + add(tm); + } + + void reset() { + startRow = null; + endRow = null; + tabletCount = 0; + totalFileCount = 0; + totalFileSize = 0; + } + } +} diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/CountFiles.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/CountFiles.java index 4083eecc864..f108419a513 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/CountFiles.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/CountFiles.java @@ -19,6 +19,7 @@ package org.apache.accumulo.manager.tableOps.merge; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.FILES; +import static org.apache.accumulo.manager.tableOps.merge.MergeInfo.Operation.SYSTEM_MERGE; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.fate.FateId; @@ -28,6 +29,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.base.Preconditions; + public class CountFiles extends ManagerRepo { private static final Logger log = LoggerFactory.getLogger(CountFiles.class); private static final long serialVersionUID = 1L; @@ -39,6 +42,9 @@ public CountFiles(MergeInfo mergeInfo) { @Override public Repo call(FateId fateId, Manager env) throws Exception { + // SYSTEM_MERGE should be executing VerifyMergeability repo, which already + // will count files + Preconditions.checkState(data.op != SYSTEM_MERGE, "Unexpected op %s", SYSTEM_MERGE); var range = data.getReserveExtent(); @@ -75,10 +81,13 @@ public Repo call(FateId fateId, Manager env) throws Exception { if (totalFiles >= maxFiles) { return new UnreserveAndError(data, totalFiles, maxFiles); } else { - if (data.op == MergeInfo.Operation.MERGE) { - return new MergeTablets(data); - } else { - return new DeleteRows(data); + switch (data.op) { + case MERGE: + return new MergeTablets(data); + case DELETE: + return new DeleteRows(data); + default: + throw new IllegalStateException("Unknown op " + data.op); } } } diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/MergeInfo.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/MergeInfo.java index 0da5159b657..6b84e16e7f0 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/MergeInfo.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/MergeInfo.java @@ -19,6 +19,7 @@ package org.apache.accumulo.manager.tableOps.merge; import java.io.Serializable; +import java.util.Objects; import org.apache.accumulo.core.clientImpl.AcceptableThriftTableOperationException; import org.apache.accumulo.core.clientImpl.thrift.TableOperation; @@ -37,7 +38,11 @@ public class MergeInfo implements Serializable { private static final long serialVersionUID = 1L; public enum Operation { - MERGE, DELETE, + MERGE, SYSTEM_MERGE, DELETE; + + public boolean isMergeOp() { + return this == MERGE || this == SYSTEM_MERGE; + } } final TableId tableId; @@ -60,7 +65,7 @@ private MergeInfo(TableId tableId, NamespaceId namespaceId, byte[] startRow, byt this.namespaceId = namespaceId; this.startRow = startRow; this.endRow = endRow; - this.op = op; + this.op = Objects.requireNonNull(op); this.mergeRangeSet = mergeRange != null; if (mergeRange != null) { mergeStartRow = @@ -102,6 +107,7 @@ public KeyExtent getMergeExtent() { public KeyExtent getReserveExtent() { switch (op) { case MERGE: + case SYSTEM_MERGE: return getOriginalExtent(); case DELETE: { if (endRow == null) { diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/ReserveTablets.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/ReserveTablets.java index ffaa6adcd25..9a8a4c3dc09 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/ReserveTablets.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/ReserveTablets.java @@ -128,6 +128,14 @@ public long isReady(FateId fateId, Manager env) throws Exception { @Override public Repo call(FateId fateId, Manager environment) throws Exception { - return new CountFiles(data); + switch (data.op) { + case SYSTEM_MERGE: + return new VerifyMergeability(data); + case MERGE: + case DELETE: + return new CountFiles(data); + default: + throw new IllegalStateException("Unknown op " + data.op); + } } } diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOp.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOp.java index ddbbce5b7e4..ea63940db6a 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOp.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOp.java @@ -57,8 +57,7 @@ public TableRangeOp(MergeInfo.Operation op, NamespaceId namespaceId, TableId tab @Override public Repo call(FateId fateId, Manager env) throws Exception { - if (AccumuloTable.ROOT.tableId().equals(data.tableId) - && MergeInfo.Operation.MERGE.equals(data.op)) { + if (AccumuloTable.ROOT.tableId().equals(data.tableId) && data.op.isMergeOp()) { log.warn("Attempt to merge tablets for {} does nothing. It is not splittable.", AccumuloTable.ROOT.tableName()); } diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/UnreserveAndError.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/UnreserveAndError.java index 926e190c881..de9aa83b2ff 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/UnreserveAndError.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/UnreserveAndError.java @@ -45,8 +45,7 @@ public UnreserveAndError(MergeInfo mergeInfo, long totalFiles, long maxFiles) { public Repo call(FateId fateId, Manager environment) throws Exception { FinishTableRangeOp.removeOperationIds(log, mergeInfo, fateId, environment); throw new AcceptableThriftTableOperationException(mergeInfo.tableId.toString(), null, - mergeInfo.op == MergeInfo.Operation.MERGE ? TableOperation.MERGE - : TableOperation.DELETE_RANGE, + mergeInfo.op.isMergeOp() ? TableOperation.MERGE : TableOperation.DELETE_RANGE, TableOperationExceptionType.OTHER, "Aborted merge because it would produce a tablets with more files than the configured limit of " + maxFiles + ". Observed " + totalFiles + " files in the merge range."); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/VerifyMergeability.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/VerifyMergeability.java new file mode 100644 index 00000000000..59558296e73 --- /dev/null +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/VerifyMergeability.java @@ -0,0 +1,101 @@ +/* + * 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.manager.tableOps.merge; + +import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.FILES; +import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.MERGEABILITY; + +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.core.fate.FateId; +import org.apache.accumulo.core.fate.Repo; +import org.apache.accumulo.manager.Manager; +import org.apache.accumulo.manager.tableOps.ManagerRepo; +import org.apache.accumulo.manager.tableOps.merge.MergeInfo.Operation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Preconditions; + +public class VerifyMergeability extends ManagerRepo { + private static final Logger log = LoggerFactory.getLogger(VerifyMergeability.class); + private static final long serialVersionUID = 1L; + private final MergeInfo data; + + public VerifyMergeability(MergeInfo mergeInfo) { + this.data = mergeInfo; + Preconditions.checkArgument(data.op == Operation.SYSTEM_MERGE, "Must be a System Merge"); + } + + @Override + public Repo call(FateId fateId, Manager env) throws Exception { + + var range = data.getReserveExtent(); + + var currentTime = env.getSteadyTime(); + var context = env.getContext(); + var tableConf = context.getTableConfiguration(data.tableId); + var threshold = tableConf.getAsBytes(Property.TABLE_SPLIT_THRESHOLD); + + // max percentage of split threshold + double mergeabilityThreshold = .25; + long maxTotalSize = (long) (threshold * mergeabilityThreshold); + long totalFiles = 0; + + long maxFiles = env.getContext().getTableConfiguration(data.getOriginalExtent().tableId()) + .getCount(Property.TABLE_MERGE_FILE_MAX); + + try (var tablets = env.getContext().getAmple().readTablets().forTable(data.tableId) + .overlapping(range.prevEndRow(), range.endRow()).fetch(FILES, MERGEABILITY) + .checkConsistency().build()) { + + boolean mergeable = true; + long totalSize = 0; + + for (var tabletMetadata : tablets) { + if (!tabletMetadata.getTabletMergeability().isMergeable(currentTime)) { + mergeable = false; + break; + } + + totalFiles += tabletMetadata.getFiles().size(); + if (totalFiles >= maxFiles) { + log.debug("{} found {} files in the merge range, maxFiles is {}", fateId, totalFiles, + maxFiles); + mergeable = false; + break; + } + + totalSize += tabletMetadata.getFileSize(); + if (totalSize > maxTotalSize) { + mergeable = false; + break; + } + + } + + if (!mergeable) { + // TODO: update UnreserveAndError to pass reason, not always max files + return new UnreserveAndError(data, totalFiles, 0); + } + } + + return new MergeTablets(data); + } + +} diff --git a/test/src/main/java/org/apache/accumulo/test/ample/TestAmpleUtil.java b/test/src/main/java/org/apache/accumulo/test/ample/TestAmpleUtil.java index 2f5ec912fb7..8b4283f31db 100644 --- a/test/src/main/java/org/apache/accumulo/test/ample/TestAmpleUtil.java +++ b/test/src/main/java/org/apache/accumulo/test/ample/TestAmpleUtil.java @@ -20,6 +20,9 @@ import static org.apache.accumulo.test.ample.metadata.TestAmple.testAmpleServerContext; +import java.time.Duration; + +import org.apache.accumulo.core.util.time.SteadyTime; import org.apache.accumulo.manager.Manager; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.test.ample.metadata.TestAmple.TestServerAmpleImpl; @@ -35,4 +38,14 @@ public static Manager mockWithAmple(ServerContext context, TestServerAmpleImpl a return manager; } + public static Manager mockWithAmple(ServerContext context, TestServerAmpleImpl ample, + Duration currentTime) { + Manager manager = EasyMock.mock(Manager.class); + EasyMock.expect(manager.getContext()).andReturn(testAmpleServerContext(context, ample)) + .atLeastOnce(); + EasyMock.expect(manager.getSteadyTime()).andReturn(SteadyTime.from(currentTime)).anyTimes(); + EasyMock.replay(manager); + return manager; + } + } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/ManagerRepoIT.java b/test/src/main/java/org/apache/accumulo/test/fate/ManagerRepoIT.java index 617eba0ba16..29c6cf40108 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/ManagerRepoIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/ManagerRepoIT.java @@ -25,12 +25,14 @@ import static org.apache.accumulo.test.ample.metadata.TestAmple.not; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.assertNoCompactionMetadata; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import java.nio.file.Path; +import java.time.Duration; import java.util.Arrays; import java.util.HashSet; import java.util.List; @@ -77,6 +79,8 @@ import org.apache.accumulo.manager.tableOps.merge.MergeInfo.Operation; import org.apache.accumulo.manager.tableOps.merge.MergeTablets; import org.apache.accumulo.manager.tableOps.merge.ReserveTablets; +import org.apache.accumulo.manager.tableOps.merge.UnreserveAndError; +import org.apache.accumulo.manager.tableOps.merge.VerifyMergeability; import org.apache.accumulo.manager.tableOps.split.AllocateDirsAndEnsureOnline; import org.apache.accumulo.manager.tableOps.split.FindSplits; import org.apache.accumulo.manager.tableOps.split.PreSplit; @@ -107,7 +111,7 @@ public static void teardown() { } @ParameterizedTest - @EnumSource(MergeInfo.Operation.class) + @EnumSource(value = MergeInfo.Operation.class, names = {"MERGE", "DELETE"}) public void testNoWalsMergeRepos(MergeInfo.Operation operation) throws Exception { String[] tableNames = getUniqueNames(2); String metadataTable = tableNames[0] + operation; @@ -163,6 +167,46 @@ public void testNoWalsMergeRepos(MergeInfo.Operation operation) throws Exception } } + @Test + public void testVerifyMergeability() throws Exception { + String[] tableNames = getUniqueNames(2); + String metadataTable = tableNames[0]; + String userTable = tableNames[1]; + + try (ClientContext client = + (ClientContext) Accumulo.newClient().from(getClientProps()).build()) { + client.tableOperations().create(userTable); + TableId tableId = TableId.of(client.tableOperations().tableIdMap().get(userTable)); + + // Set up Test ample and manager + TestAmple.createMetadataTable(client, metadataTable); + TestServerAmpleImpl testAmple = (TestServerAmpleImpl) TestAmple + .create(getCluster().getServerContext(), Map.of(DataLevel.USER, metadataTable)); + testAmple.createMetadataFromExisting(client, tableId); + Manager manager = + mockWithAmple(getCluster().getServerContext(), testAmple, Duration.ofDays(1)); + + // Create a test operation and fate id for testing merge and delete rows + // and add operation to test metadata for the tablet + var fateId = FateId.from(FateInstanceType.USER, UUID.randomUUID()); + var opid = TabletOperationId.from(TabletOperationType.MERGING, fateId); + var extent = new KeyExtent(tableId, null, null); + + try (TabletsMutator tm = testAmple.mutateTablets()) { + tm.mutateTablet(extent).putOperation(opid).mutate(); + } + + // Build either MergeTablets or DeleteRows repo for testing no WALs, both should check this + // condition + final MergeInfo mergeInfo = new MergeInfo(tableId, + manager.getContext().getNamespaceId(tableId), null, null, Operation.SYSTEM_MERGE); + final ManagerRepo repo = new VerifyMergeability(mergeInfo); + + assertInstanceOf(UnreserveAndError.class, repo.call(fateId, manager)); + + } + } + @Test public void testSplitOffline() throws Exception { String[] tableNames = getUniqueNames(2);