From 97f2dc07b25832dc6660582131c8e9d61c5a1d57 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Tue, 6 Jan 2026 15:59:12 -0500 Subject: [PATCH 01/21] Prevents iterator conflicts: This commit adds checks when adding an iterator that the given iterator does not conflict with any existing iterators. Conflict meaning same name or same priority. Iterators can be added several ways, and previously only TableOperations.attachIterator and NamespaceOperations.attachIterator would check for conflicts. This commit adds iterator conflict checks to: - Scanner.addScanIterator - TableOperations.setProperty - TableOperations.modifyProperties - NewTableConfiguration.attachIterator Note that this does not add conflict checks to NamespaceOperations.setProperty or NamespaceOperations.modifyProperties, these will be done in another commit. This commit also accounts for the several ways in which conflicts can arise: - Iterators that are attached directly to a table (either through TableOperations.attachIterator, TableOperations.setProperty, or TableOperations.modifyProperties) - Iterators that are attached to a namespace, inherited by a table (either through NamespaceOperations.attachIterator, NamespaceOperations.setProperty, or NamespaceOperations.modifyProperties) - Conflicts with default table iterators (if the table has them) - Adding the exact iterator already present should not fail This commit also adds a new IteratorConflictsIT to test all of the above. Part of #6030 --- .../client/ClientSideIteratorScanner.java | 3 + .../accumulo/core/client/IsolatedScanner.java | 9 +- .../client/admin/NewTableConfiguration.java | 64 ++- .../client/mapred/AbstractInputFormat.java | 10 +- .../client/mapreduce/AbstractInputFormat.java | 9 +- .../core/client/rfile/RFileScanner.java | 1 + .../core/clientImpl/ClientContext.java | 40 +- .../core/clientImpl/OfflineScanner.java | 6 +- .../accumulo/core/clientImpl/ScannerImpl.java | 6 +- .../core/clientImpl/ScannerOptions.java | 18 +- .../clientImpl/TableOperationsHelper.java | 17 + .../core/clientImpl/TableOperationsImpl.java | 54 ++- .../clientImpl/TabletServerBatchDeleter.java | 7 +- .../clientImpl/TabletServerBatchReader.java | 12 +- .../iteratorsImpl/IteratorConfigUtil.java | 41 ++ .../metadata/MetadataLocationObtainer.java | 7 +- .../core/metadata/schema/TabletsMetadata.java | 3 +- .../admin/NewTableConfigurationTest.java | 5 +- .../core/clientImpl/ScannerImplTest.java | 16 +- .../core/clientImpl/ScannerOptionsTest.java | 8 +- .../TabletServerBatchReaderTest.java | 10 +- .../mapred/AccumuloRecordReader.java | 9 +- .../mapreduce/AccumuloRecordReader.java | 9 +- .../server/metadata/ServerAmpleImpl.java | 3 +- .../server/util/ManagerMetadataUtil.java | 6 +- .../server/util/MetadataTableUtil.java | 4 +- .../java/org/apache/accumulo/gc/GCRun.java | 3 +- .../manager/tableOps/bulkVer1/CopyFailed.java | 7 +- .../accumulo/test/ClientSideIteratorIT.java | 4 +- .../apache/accumulo/test/CloseScannerIT.java | 4 +- .../accumulo/test/ConditionalWriterIT.java | 7 +- .../apache/accumulo/test/IteratorEnvIT.java | 4 +- .../test/NewTableConfigurationIT.java | 20 +- .../org/apache/accumulo/test/SampleIT.java | 13 +- .../accumulo/test/ScanConsistencyIT.java | 5 +- .../accumulo/test/ScanFlushWithTimeIT.java | 4 +- .../test/functional/IteratorConflictsIT.java | 383 ++++++++++++++++++ .../test/functional/ManagerAssignmentIT.java | 3 +- .../test/functional/PerTableCryptoIT.java | 3 +- .../test/functional/ScanIteratorIT.java | 2 + .../test/functional/SplitRecoveryIT.java | 3 +- .../TabletStateChangeIteratorIT.java | 3 +- .../accumulo/test/shell/ShellServerIT.java | 1 + 43 files changed, 752 insertions(+), 94 deletions(-) create mode 100644 test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java index 0209e8deece..cfe77155018 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java +++ b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java @@ -173,6 +173,7 @@ public SortedKeyValueIterator deepCopy(final IteratorEnvironment env) * @param scanner the source scanner */ public ClientSideIteratorScanner(final Scanner scanner) { + super((setting) -> {}); smi = new ScannerTranslatorImpl(scanner, scanner.getSamplerConfiguration()); this.range = scanner.getRange(); this.size = scanner.getBatchSize(); @@ -188,6 +189,8 @@ public ClientSideIteratorScanner(final Scanner scanner) { var scannerImpl = (ScannerImpl) scanner; this.context = () -> scannerImpl.getClientContext(); this.tableId = () -> scannerImpl.getTableId(); + setIteratorValidator( + scannerImpl.getClientContext().getScanIteratorValidator(scannerImpl.getTableId())); } else { // These may never be used, so only fail if an attempt is made to use them. this.context = () -> { diff --git a/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java b/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java index 5a7328a5f26..54fbfda5893 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java +++ b/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java @@ -24,6 +24,7 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.Map.Entry; +import java.util.function.Consumer; import org.apache.accumulo.core.clientImpl.IsolationException; import org.apache.accumulo.core.clientImpl.ScannerOptions; @@ -220,11 +221,13 @@ public void clear() { private long readaheadThreshold; private final RowBufferFactory bufferFactory; - public IsolatedScanner(Scanner scanner) { - this(scanner, new MemoryRowBufferFactory()); + public IsolatedScanner(Scanner scanner, Consumer iteratorValidator) { + this(scanner, new MemoryRowBufferFactory(), iteratorValidator); } - public IsolatedScanner(Scanner scanner, RowBufferFactory bufferFactory) { + public IsolatedScanner(Scanner scanner, RowBufferFactory bufferFactory, + Consumer iteratorValidator) { + super(iteratorValidator); this.scanner = scanner; this.range = scanner.getRange(); this.retryTimeout = scanner.getTimeout(MILLISECONDS); diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java index 9aacc7ca18f..b6249df99d8 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java +++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java @@ -49,7 +49,6 @@ import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError; import org.apache.hadoop.io.Text; -import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSortedSet; /** @@ -73,8 +72,25 @@ public class NewTableConfiguration { private Map summarizerProps = Collections.emptyMap(); private Map localityProps = Collections.emptyMap(); private final Map iteratorProps = new HashMap<>(); + private final Map inheritedIteratorProps = new HashMap<>(); private SortedSet splitProps = Collections.emptySortedSet(); + /** + * Configures the {@link NewTableConfiguration} with iterators inherited from the parent + * namespace. This is used internally in table creation - no need to call directly. + * + * @param props the parent namespace config + */ + public void configureInheritedIteratorProps(Map props) { + for (var prop : props.entrySet()) { + var propKey = prop.getKey(); + var propVal = prop.getValue(); + if (IteratorConfigUtil.isIterProp(propKey, propVal)) { + inheritedIteratorProps.put(propKey, propVal); + } + } + } + private void checkDisjoint(Map props, Map derivedProps, String kind) { checkArgument(Collections.disjoint(props.keySet(), derivedProps.keySet()), @@ -185,7 +201,7 @@ public NewTableConfiguration setProperties(Map props) { * * @return the current properties configured */ - public Map getProperties() { + public Map getProperties() throws AccumuloException { Map propertyMap = new HashMap<>(); propertyMap.putAll(summarizerProps); @@ -199,29 +215,51 @@ public Map getProperties() { // check the properties for conflicts with default iterators var defaultIterSettings = IteratorConfigUtil.getInitialTableIteratorSettings(); // if a default prop already exists, don't want to consider that a conflict - var noDefaultsPropMap = new HashMap<>(propertyMap); - noDefaultsPropMap.entrySet().removeIf(entry -> initTableProps.get(entry.getKey()) != null - && initTableProps.get(entry.getKey()).equals(entry.getValue())); - defaultIterSettings.forEach((setting, scopes) -> { + for (var defaultIterSetting : defaultIterSettings.entrySet()) { + var setting = defaultIterSetting.getKey(); + var scopes = defaultIterSetting.getValue(); try { - TableOperationsHelper.checkIteratorConflicts(noDefaultsPropMap, setting, scopes); + TableOperationsHelper.checkIteratorConflicts(propertyMap, setting, scopes); } catch (AccumuloException e) { - throw new IllegalStateException(String.format( + throw new AccumuloException(String.format( "conflict with default table iterator: scopes: %s setting: %s", scopes, setting), e); } - }); + } // check the properties for conflicts with default properties (non-iterator) var nonIterDefaults = IteratorConfigUtil.getInitialTableProperties(); nonIterDefaults.keySet().removeAll(IteratorConfigUtil.getInitialTableIterators().keySet()); - nonIterDefaults.forEach((dk, dv) -> { + for (var nonIterDefault : nonIterDefaults.entrySet()) { + var dk = nonIterDefault.getKey(); + var dv = nonIterDefault.getValue(); var valInPropMap = propertyMap.get(dk); - Preconditions.checkState(valInPropMap == null || valInPropMap.equals(dv), String.format( - "conflict for property %s : %s (default val) != %s (set val)", dk, dv, valInPropMap)); - }); + if (valInPropMap != null && !valInPropMap.equals(dv)) { + throw new AccumuloException(String.format( + "conflict for property %s : %s (default val) != %s (set val)", dk, dv, valInPropMap)); + } + } propertyMap.putAll(initTableProps); } + + // check for conflicts between attached iterators and namespace-inherited iterators + for (var iterProp : iteratorProps.entrySet()) { + var iterPropKey = iterProp.getKey(); + var iterPropVal = iterProp.getValue(); + if (IteratorConfigUtil.isNonOptionIterProp(iterPropKey, iterPropVal)) { + var iterPropKeyParts = iterPropKey.split("\\."); + var iterPropValParts = iterPropVal.split(","); + String iterName = iterPropKeyParts[iterPropKeyParts.length - 1]; + IteratorScope iterScope = + IteratorScope.valueOf(iterPropKeyParts[iterPropKeyParts.length - 2]); + Map opts = IteratorConfigUtil.gatherOpts(iterPropKey, iteratorProps); + var is = new IteratorSetting(Integer.parseInt(iterPropValParts[0]), iterName, + iterPropValParts[1], opts); + TableOperationsHelper.checkIteratorConflicts(inheritedIteratorProps, is, + EnumSet.of(iterScope)); + } + } + return Collections.unmodifiableMap(propertyMap); } diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java index 088a0655c4d..f84a84d032e 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java @@ -494,16 +494,18 @@ public void initialize(InputSplit inSplit, JobConf job) throws IOException { Scanner scanner; try { + var iteratorValidator = client.getScanIteratorValidator(table); if (isOffline) { - scanner = - new OfflineScanner(client, TableId.of(baseSplit.getTableId()), authorizations); + scanner = new OfflineScanner(client, TableId.of(baseSplit.getTableId()), authorizations, + iteratorValidator); } else { - scanner = new ScannerImpl(client, TableId.of(baseSplit.getTableId()), authorizations); + scanner = new ScannerImpl(client, TableId.of(baseSplit.getTableId()), authorizations, + iteratorValidator); } if (isIsolated) { log.info("Creating isolated scanner"); @SuppressWarnings("resource") - var wrapped = new IsolatedScanner(scanner); + var wrapped = new IsolatedScanner(scanner, iteratorValidator); scanner = wrapped; } if (usesLocalIterators) { diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java index 432471334b3..05db0c2e301 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java @@ -508,17 +508,20 @@ public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IO } try { + var iteratorValidator = client.getScanIteratorValidator(table); if (isOffline) { - scanner = new OfflineScanner(client, TableId.of(split.getTableId()), authorizations); + scanner = new OfflineScanner(client, TableId.of(split.getTableId()), authorizations, + iteratorValidator); } else { // Not using public API to create scanner so that we can use table ID // Table ID is used in case of renames during M/R job - scanner = new ScannerImpl(client, TableId.of(split.getTableId()), authorizations); + scanner = new ScannerImpl(client, TableId.of(split.getTableId()), authorizations, + iteratorValidator); } if (isIsolated) { log.info("Creating isolated scanner"); @SuppressWarnings("resource") - var wrapped = new IsolatedScanner(scanner); + var wrapped = new IsolatedScanner(scanner, iteratorValidator); scanner = wrapped; } if (usesLocalIterators) { diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java index 9266a53e510..88a7576d93a 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java +++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java @@ -134,6 +134,7 @@ static class Opts { } RFileScanner(Opts opts) { + super((setting) -> {}); if (!opts.auths.equals(Authorizations.EMPTY) && !opts.useSystemIterators) { throw new IllegalArgumentException( "Set authorizations and specified not to use system iterators"); diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java index e5b059a80f7..947d282a52f 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java @@ -34,6 +34,7 @@ import java.nio.file.Path; import java.util.Collection; import java.util.Collections; +import java.util.EnumSet; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -44,6 +45,7 @@ import java.util.concurrent.Future; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; +import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -59,6 +61,7 @@ import org.apache.accumulo.core.client.ConditionalWriter; import org.apache.accumulo.core.client.ConditionalWriterConfig; import org.apache.accumulo.core.client.Durability; +import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.MultiTableBatchWriter; import org.apache.accumulo.core.client.NamespaceNotFoundException; import org.apache.accumulo.core.client.Scanner; @@ -83,6 +86,7 @@ import org.apache.accumulo.core.fate.zookeeper.ZooCacheFactory; import org.apache.accumulo.core.fate.zookeeper.ZooReader; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.iterators.IteratorUtil; import org.apache.accumulo.core.manager.state.tables.TableState; import org.apache.accumulo.core.metadata.RootTable; import org.apache.accumulo.core.metadata.schema.Ample; @@ -711,7 +715,7 @@ public BatchScanner createBatchScanner(String tableName, Authorizations authoriz ensureOpen(); checkArgument(authorizations != null, "authorizations is null"); return new TabletServerBatchReader(this, requireNotOffline(getTableId(tableName), tableName), - tableName, authorizations, numQueryThreads); + tableName, authorizations, numQueryThreads, getScanIteratorValidator(tableName)); } @Override @@ -737,7 +741,8 @@ public BatchDeleter createBatchDeleter(String tableName, Authorizations authoriz ensureOpen(); checkArgument(authorizations != null, "authorizations is null"); return new TabletServerBatchDeleter(this, requireNotOffline(getTableId(tableName), tableName), - tableName, authorizations, numQueryThreads, config.merge(getBatchWriterConfig())); + tableName, authorizations, numQueryThreads, config.merge(getBatchWriterConfig()), + getScanIteratorValidator(tableName)); } @Override @@ -796,8 +801,8 @@ public Scanner createScanner(String tableName, Authorizations authorizations) throws TableNotFoundException { ensureOpen(); checkArgument(authorizations != null, "authorizations is null"); - Scanner scanner = - new ScannerImpl(this, requireNotOffline(getTableId(tableName), tableName), authorizations); + Scanner scanner = new ScannerImpl(this, requireNotOffline(getTableId(tableName), tableName), + authorizations, getScanIteratorValidator(tableName)); Integer batchSize = ClientProperty.SCANNER_BATCH_SIZE.getInteger(getProperties()); if (batchSize != null) { scanner.setBatchSize(batchSize); @@ -812,6 +817,33 @@ public Scanner createScanner(String tableName) return createScanner(tableName, auths); } + private Consumer getScanIteratorValidator(Callable tableNameGetter) { + return (givenIter) -> { + try { + tableOperations().checkIteratorConflicts(tableNameGetter.call(), givenIter, + EnumSet.of(IteratorUtil.IteratorScope.scan)); + } catch (Exception e) { + throw new IllegalArgumentException(e); + } + }; + } + + /** + * @see #getScanIteratorValidator(String) + */ + public Consumer getScanIteratorValidator(TableId tableId) { + return getScanIteratorValidator(() -> getTableName(tableId)); + } + + /** + * Returns an IteratorSetting Consumer which checks if the given IteratorSetting conflicts with + * iterators on the given table. Invoking the Consumer will throw an + * {@link IllegalArgumentException} if there is a conflict + */ + public Consumer getScanIteratorValidator(String tableName) { + return getScanIteratorValidator(() -> tableName); + } + @Override public String whoami() { ensureOpen(); diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineScanner.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineScanner.java index 3cef0428763..d1ff801d2d9 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineScanner.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineScanner.java @@ -22,8 +22,10 @@ import java.util.Iterator; import java.util.Map.Entry; +import java.util.function.Consumer; import org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Range; @@ -41,7 +43,9 @@ public class OfflineScanner extends ScannerOptions implements Scanner { private final Authorizations authorizations; private final Text tableId; - public OfflineScanner(ClientContext context, TableId tableId, Authorizations authorizations) { + public OfflineScanner(ClientContext context, TableId tableId, Authorizations authorizations, + Consumer iteratorValidator) { + super(iteratorValidator); checkArgument(context != null, "context is null"); checkArgument(tableId != null, "tableId is null"); checkArgument(authorizations != null, "authorizations is null"); diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerImpl.java index b4dfaa6258d..910796de518 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerImpl.java @@ -26,8 +26,10 @@ import java.util.LinkedHashMap; import java.util.Map; import java.util.Map.Entry; +import java.util.function.Consumer; import org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Range; @@ -109,7 +111,9 @@ private synchronized void ensureOpen() { } } - public ScannerImpl(ClientContext context, TableId tableId, Authorizations authorizations) { + public ScannerImpl(ClientContext context, TableId tableId, Authorizations authorizations, + Consumer iteratorValidator) { + super(iteratorValidator); checkArgument(context != null, "context is null"); checkArgument(tableId != null, "tableId is null"); checkArgument(authorizations != null, "authorizations is null"); diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerOptions.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerOptions.java index 2b044f6fd84..126b34ae3bc 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerOptions.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerOptions.java @@ -34,6 +34,7 @@ import java.util.SortedSet; import java.util.TreeSet; import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.ScannerBase; @@ -67,15 +68,30 @@ public class ScannerOptions implements ScannerBase { private ConsistencyLevel consistencyLevel = ConsistencyLevel.IMMEDIATE; - protected ScannerOptions() {} + private Consumer iteratorValidator; + + protected ScannerOptions(Consumer iteratorValidator) { + checkArgument(iteratorValidator != null, "iteratorValidator is null"); + this.iteratorValidator = iteratorValidator; + } public ScannerOptions(ScannerOptions so) { setOptions(this, so); } + public void setIteratorValidator(Consumer iteratorValidator) { + checkArgument(iteratorValidator != null, "iteratorValidator is null"); + this.iteratorValidator = iteratorValidator; + } + @Override public synchronized void addScanIterator(IteratorSetting si) { checkArgument(si != null, "si is null"); + + // validate given iterator does not conflict with those already added to the table + iteratorValidator.accept(si); + + // validate given iterator does not conflict with those already added to the scanner if (serverSideIteratorList.isEmpty()) { serverSideIteratorList = new ArrayList<>(); } diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsHelper.java index 030bb958806..901aeb94746 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsHelper.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsHelper.java @@ -144,7 +144,13 @@ public static void checkIteratorConflicts(Map props, IteratorSett String.format("%s%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase()); String nameStr = String.format("%s.%s", scopeStr, setting.getName()); String optStr = String.format("%s.opt.", nameStr); + String valStr = String.format("%s,%s", setting.getPriority(), setting.getIteratorClass()); Map optionConflicts = new TreeMap<>(); + // skip if the setting is present in the map... not a conflict if exactly the same + if (props.containsKey(nameStr) && props.get(nameStr).equals(valStr) + && containsSameIterOpts(props, setting, optStr)) { + continue; + } for (Entry property : props.entrySet()) { if (property.getKey().startsWith(scopeStr)) { if (property.getKey().equals(nameStr)) { @@ -180,6 +186,17 @@ public static void checkIteratorConflicts(Map props, IteratorSett } } + private static boolean containsSameIterOpts(Map props, IteratorSetting setting, + String optStr) { + for (var opt : setting.getOptions().entrySet()) { + final String optKey = optStr + opt.getKey(); + if (!props.containsKey(optKey) || !props.get(optKey).equals(opt.getValue())) { + return false; + } + } + return true; + } + @Override public void checkIteratorConflicts(String tableName, IteratorSetting setting, EnumSet scopes) throws AccumuloException, TableNotFoundException { diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java index 17727b2b8f1..669103a0eb3 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java @@ -116,6 +116,7 @@ import org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; +import org.apache.accumulo.core.iteratorsImpl.IteratorConfigUtil; import org.apache.accumulo.core.manager.state.tables.TableState; import org.apache.accumulo.core.manager.thrift.FateOperation; import org.apache.accumulo.core.manager.thrift.FateService; @@ -231,6 +232,19 @@ public void create(String tableName, NewTableConfiguration ntc) NEW_TABLE_NAME.validate(tableName); checkArgument(ntc != null, "ntc is null"); + final String[] nsAndTable; + try { + if ((nsAndTable = tableName.split("\\" + Namespace.SEPARATOR)).length == 2) { + ntc.configureInheritedIteratorProps( + context.namespaceOperations().getNamespaceProperties(nsAndTable[0])); + } else { + ntc.configureInheritedIteratorProps( + context.namespaceOperations().getNamespaceProperties(Namespace.DEFAULT.name())); + } + } catch (NamespaceNotFoundException e) { + throw new AccumuloException(e); + } + List args = new ArrayList<>(); args.add(ByteBuffer.wrap(tableName.getBytes(UTF_8))); args.add(ByteBuffer.wrap(ntc.getTimeType().name().getBytes(UTF_8))); @@ -1008,20 +1022,50 @@ public void setProperty(final String tableName, final String property, final Str checkArgument(value != null, "value is null"); try { + checkIteratorConflicts(Map.copyOf(this.getConfiguration(tableName)), property, value); + setPropertyNoChecks(tableName, property, value); checkLocalityGroups(tableName, property); - } catch (TableNotFoundException e) { + } catch (TableNotFoundException | IllegalArgumentException e) { throw new AccumuloException(e); } } + private void checkIteratorConflicts(Map props, String property, String value) + throws AccumuloException, TableNotFoundException, IllegalArgumentException { + if (props.containsKey(property) && props.get(property).equals(value)) { + // setting a property that already exists (i.e., no change) + return; + } + if (IteratorConfigUtil.isNonOptionIterProp(property, value)) { + String[] iterPropParts = property.split("\\."); + IteratorScope scope = IteratorScope.valueOf(iterPropParts[2]); + String iterName = iterPropParts[3]; + String[] priorityAndClass; + if ((priorityAndClass = value.split(",")).length == 2) { + // given a single property, the only way for the property to be equivalent to an existing + // iterator is if the existing iterator has no options (opts are set as separate props) + IteratorSetting givenIter = new IteratorSetting(Integer.parseInt(priorityAndClass[0]), + iterName, priorityAndClass[1]); + checkIteratorConflicts(props, givenIter, EnumSet.of(scope)); + } + } + } + private Map tryToModifyProperties(String tableName, final Consumer> mapMutator) throws AccumuloException, AccumuloSecurityException, IllegalArgumentException, ConcurrentModificationException { final TVersionedProperties vProperties = ThriftClientTypes.CLIENT.execute(context, client -> client .getVersionedTableProperties(TraceUtil.traceInfo(), context.rpcCreds(), tableName)); + final Map configBeforeMut; + try { + configBeforeMut = getConfiguration(tableName); + } catch (TableNotFoundException e) { + throw new AccumuloException(e); + } + mapMutator.accept(vProperties.getProperties()); // A reference to the map was passed to the user, maybe they still have the reference and are @@ -1030,6 +1074,14 @@ private Map tryToModifyProperties(String tableName, // from here on the code is assured to always be dealing with the same map. vProperties.setProperties(Map.copyOf(vProperties.getProperties())); + try { + for (var property : vProperties.getProperties().entrySet()) { + checkIteratorConflicts(configBeforeMut, property.getKey(), property.getValue()); + } + } catch (TableNotFoundException e) { + throw new AccumuloException(e); + } + try { // Send to server ThriftClientTypes.MANAGER.executeVoid(context, diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchDeleter.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchDeleter.java index 02be6e3904f..f37af60511b 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchDeleter.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchDeleter.java @@ -20,6 +20,7 @@ import java.util.Iterator; import java.util.Map.Entry; +import java.util.function.Consumer; import org.apache.accumulo.core.client.BatchDeleter; import org.apache.accumulo.core.client.BatchWriter; @@ -41,8 +42,10 @@ public class TabletServerBatchDeleter extends TabletServerBatchReader implements private final BatchWriterConfig bwConfig; public TabletServerBatchDeleter(ClientContext context, TableId tableId, String tableName, - Authorizations authorizations, int numQueryThreads, BatchWriterConfig bwConfig) { - super(context, BatchDeleter.class, tableId, tableName, authorizations, numQueryThreads); + Authorizations authorizations, int numQueryThreads, BatchWriterConfig bwConfig, + Consumer iteratorValidator) { + super(context, BatchDeleter.class, tableId, tableName, authorizations, numQueryThreads, + iteratorValidator); this.context = context; this.tableId = tableId; this.bwConfig = bwConfig; diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java index 8b149da57fa..a28c82139b7 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java @@ -29,8 +29,10 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; import org.apache.accumulo.core.client.BatchScanner; +import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.TableId; @@ -57,12 +59,16 @@ public class TabletServerBatchReader extends ScannerOptions implements BatchScan private ArrayList ranges = null; public TabletServerBatchReader(ClientContext context, TableId tableId, String tableName, - Authorizations authorizations, int numQueryThreads) { - this(context, BatchScanner.class, tableId, tableName, authorizations, numQueryThreads); + Authorizations authorizations, int numQueryThreads, + Consumer iteratorValidator) { + this(context, BatchScanner.class, tableId, tableName, authorizations, numQueryThreads, + iteratorValidator); } protected TabletServerBatchReader(ClientContext context, Class scopeClass, TableId tableId, - String tableName, Authorizations authorizations, int numQueryThreads) { + String tableName, Authorizations authorizations, int numQueryThreads, + Consumer iteratorValidator) { + super(iteratorValidator); checkArgument(context != null, "context is null"); checkArgument(tableId != null, "tableId is null"); checkArgument(authorizations != null, "authorizations is null"); diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java index 59224deb9b9..cda8ed96493 100644 --- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.EnumSet; @@ -31,6 +32,7 @@ import java.util.Map.Entry; import java.util.Set; import java.util.TreeMap; +import java.util.stream.Collectors; import org.apache.accumulo.core.classloader.ClassLoaderUtil; import org.apache.accumulo.core.client.IteratorSetting; @@ -57,6 +59,17 @@ public class IteratorConfigUtil { public static final Comparator ITER_INFO_COMPARATOR = Comparator.comparingInt(IterInfo::getPriority); + private static final String ITERATOR_PROP_REGEX = + ("^" + Property.TABLE_ITERATOR_PREFIX.getKey() + "(" + Arrays.stream(IteratorScope.values()) + .map(scope -> scope.name().toLowerCase()).collect(Collectors.joining(".|")) + ".)") + .replace(".", "\\.") + "[^.]+$"; + private static final String ITERATOR_PROP_VAL_REGEX = "^[0-9]+,[^,]+$"; + private static final String ITERATOR_PROP_OPT_REGEX = + ("^" + Property.TABLE_ITERATOR_PREFIX.getKey() + "(" + + Arrays.stream(IteratorScope.values()).map(scope -> scope.name().toLowerCase()) + .collect(Collectors.joining(".|")) + + ".)").replace(".", "\\.") + "[^.]+\\.opt\\.[^.]+$"; + /** * Fetch the correct configuration key prefix for the given scope. Throws an * IllegalArgumentException if no property exists for the given scope. @@ -273,4 +286,32 @@ private static Class> loadClass(boolean useAcc log.trace("Iterator class {} loaded from classpath", iterInfo.className); return clazz; } + + /** + * Returns true if the property is an iterator property not including iterator option properties + */ + public static boolean isNonOptionIterProp(String propKey, String propVal) { + return propKey.matches(IteratorConfigUtil.ITERATOR_PROP_REGEX) + && propVal.matches(IteratorConfigUtil.ITERATOR_PROP_VAL_REGEX); + } + + public static boolean isIterProp(String propKey, String propVal) { + return isNonOptionIterProp(propKey, propVal) || ITERATOR_PROP_OPT_REGEX.matches(propKey); + } + + /** + * returns a map of the options associated with the given iterator property key. Options of the + * iterator are obtained by searching the given map + */ + public static Map gatherOpts(String iterPropKey, Map map) { + Map opts = new HashMap<>(); + for (var iteratorProp : map.entrySet()) { + if (ITERATOR_PROP_OPT_REGEX.matches(iteratorProp.getKey()) + && iteratorProp.getKey().contains(iterPropKey)) { + String[] parts = iteratorProp.getKey().split("\\."); + opts.put(parts[parts.length - 1], iteratorProp.getValue()); + } + } + return opts; + } } diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java index ce198a7e85c..4dc8c82fa46 100644 --- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java +++ b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java @@ -31,6 +31,7 @@ import java.util.SortedSet; import java.util.TreeMap; import java.util.TreeSet; +import java.util.function.Consumer; import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.AccumuloException; @@ -160,6 +161,10 @@ private void decodeRows(TreeMap encodedResults, TreeMap re } private static class SettableScannerOptions extends ScannerOptions { + public SettableScannerOptions(Consumer iteratorValidator) { + super(iteratorValidator); + } + public ScannerOptions setColumns(SortedSet locCols) { this.fetchedColumns = locCols; // see comment in lookupTablet about why iterator is used @@ -186,7 +191,7 @@ public List lookupTablets(ClientContext context, String tserver, }; ScannerOptions opts = null; - try (SettableScannerOptions unsetOpts = new SettableScannerOptions()) { + try (SettableScannerOptions unsetOpts = new SettableScannerOptions((setting) -> {})) { opts = unsetOpts.setColumns(locCols); } diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java index 41189e8d663..597c55ff056 100644 --- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java +++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java @@ -197,7 +197,8 @@ private TabletsMetadata buildNonRoot(AccumuloClient client) { String resolvedTable = table == null ? level.metaTable() : table; Scanner scanner = - new IsolatedScanner(client.createScanner(resolvedTable, Authorizations.EMPTY)); + new IsolatedScanner(client.createScanner(resolvedTable, Authorizations.EMPTY), + ((ClientContext) client).getScanIteratorValidator(resolvedTable)); scanner.setRange(range); boolean extentsPresent = extentsToFetch != null; diff --git a/core/src/test/java/org/apache/accumulo/core/client/admin/NewTableConfigurationTest.java b/core/src/test/java/org/apache/accumulo/core/client/admin/NewTableConfigurationTest.java index 89930fb37ac..d940a21d8e1 100644 --- a/core/src/test/java/org/apache/accumulo/core/client/admin/NewTableConfigurationTest.java +++ b/core/src/test/java/org/apache/accumulo/core/client/admin/NewTableConfigurationTest.java @@ -31,6 +31,7 @@ import java.util.SortedSet; import java.util.TreeSet; +import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.sample.SamplerConfiguration; import org.apache.accumulo.core.client.summary.Summarizer; @@ -124,7 +125,7 @@ public void populateOptions() { * Verify enableSampling returns */ @Test - public void testEnableSampling() { + public void testEnableSampling() throws AccumuloException { SamplerConfiguration sha1SamplerConfig = new SamplerConfiguration("com.mysampler"); sha1SamplerConfig.setOptions(options); NewTableConfiguration ntcSample2 = @@ -138,7 +139,7 @@ public void testEnableSampling() { * Verify enableSummarization returns SummarizerConfiguration with the expected class name(s). */ @Test - public void testEnableSummarization() { + public void testEnableSummarization() throws AccumuloException { SummarizerConfiguration summarizerConfig1 = SummarizerConfiguration .builder("com.test.summarizer").setPropertyId("s1").addOption("opt1", "v1").build(); NewTableConfiguration ntcSummarization1 = diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerImplTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerImplTest.java index 676f00c593c..a7f774a302e 100644 --- a/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerImplTest.java +++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerImplTest.java @@ -34,11 +34,15 @@ public class ScannerImplTest { @BeforeEach public void setup() { context = EasyMock.createMock(ClientContext.class); + EasyMock.expect(context.getScanIteratorValidator(EasyMock.anyString())) + .andReturn(iteratorSetting -> {}); + EasyMock.replay(context); } @Test public void testValidReadaheadValues() { - try (var s = new ScannerImpl(context, TableId.of("foo"), Authorizations.EMPTY)) { + try (var s = new ScannerImpl(context, TableId.of("foo"), Authorizations.EMPTY, + context.getScanIteratorValidator("bar"))) { s.setReadaheadThreshold(0); s.setReadaheadThreshold(10); s.setReadaheadThreshold(Long.MAX_VALUE); @@ -49,7 +53,8 @@ public void testValidReadaheadValues() { @Test public void testInValidReadaheadValues() { - try (var s = new ScannerImpl(context, TableId.of("foo"), Authorizations.EMPTY)) { + try (var s = new ScannerImpl(context, TableId.of("foo"), Authorizations.EMPTY, + context.getScanIteratorValidator("bar"))) { assertThrows(IllegalArgumentException.class, () -> s.setReadaheadThreshold(-1)); } } @@ -57,15 +62,16 @@ public void testInValidReadaheadValues() { @Test public void testGetAuthorizations() { Authorizations expected = new Authorizations("a,b"); - try (var s = new ScannerImpl(context, TableId.of("foo"), expected)) { + try (var s = new ScannerImpl(context, TableId.of("foo"), expected, + context.getScanIteratorValidator("bar"))) { assertEquals(expected, s.getAuthorizations()); } } @Test public void testNullAuthorizationsFails() { - assertThrows(IllegalArgumentException.class, - () -> new ScannerImpl(context, TableId.of("foo"), null)); + assertThrows(IllegalArgumentException.class, () -> new ScannerImpl(context, TableId.of("foo"), + null, context.getScanIteratorValidator("bar"))); } } diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerOptionsTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerOptionsTest.java index 5c484acc0df..c3f56e3911d 100644 --- a/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerOptionsTest.java +++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerOptionsTest.java @@ -40,7 +40,7 @@ public class ScannerOptionsTest { */ @Test public void testAddRemoveIterator() { - try (ScannerOptions options = new ScannerOptions()) { + try (ScannerOptions options = new ScannerOptions((setting) -> {})) { options.addScanIterator(new IteratorSetting(1, "NAME", WholeRowIterator.class)); assertEquals(1, options.serverSideIteratorList.size()); options.removeScanIterator("NAME"); @@ -50,7 +50,7 @@ public void testAddRemoveIterator() { @Test public void testIteratorConflict() { - try (ScannerOptions options = new ScannerOptions()) { + try (ScannerOptions options = new ScannerOptions((setting) -> {})) { options.addScanIterator(new IteratorSetting(1, "NAME", DebugIterator.class)); assertThrows(IllegalArgumentException.class, () -> options.addScanIterator(new IteratorSetting(2, "NAME", DebugIterator.class))); @@ -61,7 +61,7 @@ public void testIteratorConflict() { @Test public void testFetchColumn() { - try (ScannerOptions options = new ScannerOptions()) { + try (ScannerOptions options = new ScannerOptions((setting) -> {})) { assertEquals(0, options.getFetchedColumns().size()); IteratorSetting.Column col = new IteratorSetting.Column(new Text("family"), new Text("qualifier")); @@ -76,7 +76,7 @@ public void testFetchColumn() { @Test public void testFetchNullColumn() { - try (ScannerOptions options = new ScannerOptions()) { + try (ScannerOptions options = new ScannerOptions((setting) -> {})) { // Require a non-null instance of Column assertThrows(IllegalArgumentException.class, () -> options.fetchColumn(null)); } diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderTest.java index cfc47322f5e..60558839bdc 100644 --- a/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderTest.java +++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderTest.java @@ -37,21 +37,23 @@ public class TabletServerBatchReaderTest { public void setup() { context = EasyMock.createMock(ClientContext.class); EasyMock.expect(context.threadPools()).andReturn(ThreadPools.getServerThreadPools()); + EasyMock.expect(context.getScanIteratorValidator(EasyMock.anyString())) + .andReturn(iteratorSetting -> {}); EasyMock.replay(context); } @Test public void testGetAuthorizations() { Authorizations expected = new Authorizations("a,b"); - try (BatchScanner s = - new TabletServerBatchReader(context, TableId.of("foo"), "fooName", expected, 1)) { + try (BatchScanner s = new TabletServerBatchReader(context, TableId.of("foo"), "fooName", + expected, 1, context.getScanIteratorValidator("fooName"))) { assertEquals(expected, s.getAuthorizations()); } } @Test public void testNullAuthorizationsFails() { - assertThrows(IllegalArgumentException.class, - () -> new TabletServerBatchReader(context, TableId.of("foo"), "fooName", null, 1)); + assertThrows(IllegalArgumentException.class, () -> new TabletServerBatchReader(context, + TableId.of("foo"), "fooName", null, 1, context.getScanIteratorValidator("fooName"))); } } diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java index 2325c086a93..4ec077d050b 100644 --- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java +++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java @@ -184,19 +184,22 @@ public void initialize(InputSplit inSplit, JobConf job) throws IOException { } Scanner scanner; + var iteratorValidator = context.getScanIteratorValidator(table); try { if (isOffline) { - scanner = new OfflineScanner(context, TableId.of(baseSplit.getTableId()), authorizations); + scanner = new OfflineScanner(context, TableId.of(baseSplit.getTableId()), authorizations, + iteratorValidator); } else { - scanner = new ScannerImpl(context, TableId.of(baseSplit.getTableId()), authorizations); + scanner = new ScannerImpl(context, TableId.of(baseSplit.getTableId()), authorizations, + iteratorValidator); scanner.setConsistencyLevel(cl == null ? ConsistencyLevel.IMMEDIATE : cl); log.info("Using consistency level: {}", scanner.getConsistencyLevel()); } if (isIsolated) { log.info("Creating isolated scanner"); @SuppressWarnings("resource") - var wrapped = new IsolatedScanner(scanner); + var wrapped = new IsolatedScanner(scanner, iteratorValidator); scanner = wrapped; } if (usesLocalIterators) { diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java index 66445e62cdf..296670ad377 100644 --- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java +++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java @@ -203,19 +203,22 @@ public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IO } try { + var iteratorValidator = ((ClientContext) client).getScanIteratorValidator(table); if (isOffline) { - scanner = new OfflineScanner(context, TableId.of(split.getTableId()), authorizations); + scanner = new OfflineScanner(context, TableId.of(split.getTableId()), authorizations, + iteratorValidator); } else { // Not using public API to create scanner so that we can use table ID // Table ID is used in case of renames during M/R job - scanner = new ScannerImpl(context, TableId.of(split.getTableId()), authorizations); + scanner = new ScannerImpl(context, TableId.of(split.getTableId()), authorizations, + iteratorValidator); scanner.setConsistencyLevel(cl == null ? ConsistencyLevel.IMMEDIATE : cl); log.info("Using consistency level: {}", scanner.getConsistencyLevel()); } if (isIsolated) { log.info("Creating isolated scanner"); @SuppressWarnings("resource") - var wrapped = new IsolatedScanner(scanner); + var wrapped = new IsolatedScanner(scanner, iteratorValidator); scanner = wrapped; } if (usesLocalIterators) { diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java index 22cb7a941fb..a68301c1c54 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java +++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java @@ -191,7 +191,8 @@ public void removeBulkLoadEntries(TableId tableId, long tid, Text firstSplit, Te Preconditions.checkArgument(DataLevel.of(tableId) == DataLevel.USER); try ( Scanner mscanner = - new IsolatedScanner(context.createScanner(MetadataTable.NAME, Authorizations.EMPTY)); + new IsolatedScanner(context.createScanner(MetadataTable.NAME, Authorizations.EMPTY), + context.getScanIteratorValidator(MetadataTable.NAME)); BatchWriter bw = context.createBatchWriter(MetadataTable.NAME)) { mscanner.setRange(new KeyExtent(tableId, lastSplit, firstSplit).toMetaRange()); mscanner.fetchColumnFamily(BulkFileColumnFamily.NAME); diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java index c0b5e27fa65..3f0c9629d1a 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java @@ -132,7 +132,8 @@ private static KeyExtent fixSplit(ServerContext context, TableId tableId, Text m Key prevRowKey = new Key(new Text(TabletsSection.encodeRow(tableId, metadataPrevEndRow))); try (ScannerImpl scanner2 = - new ScannerImpl(context, Ample.DataLevel.of(tableId).metaTableId(), Authorizations.EMPTY)) { + new ScannerImpl(context, Ample.DataLevel.of(tableId).metaTableId(), Authorizations.EMPTY, + context.getScanIteratorValidator(Ample.DataLevel.of(tableId).metaTable()))) { scanner2.setRange(new Range(prevRowKey, prevRowKey.followingKey(PartialKey.ROW))); if (scanner2.iterator().hasNext()) { @@ -146,7 +147,8 @@ private static KeyExtent fixSplit(ServerContext context, TableId tableId, Text m Key rowKey = new Key(metadataEntry); try (Scanner scanner3 = new ScannerImpl(context, Ample.DataLevel.of(tableId).metaTableId(), - Authorizations.EMPTY)) { + Authorizations.EMPTY, + context.getScanIteratorValidator(Ample.DataLevel.of(tableId).metaTable()))) { scanner3.fetchColumnFamily(DataFileColumnFamily.NAME); scanner3.setRange(new Range(rowKey, rowKey.followingKey(PartialKey.ROW))); diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java index 60b16e55738..a07c16deb03 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java @@ -311,7 +311,9 @@ public static void splitDatafiles(Text midRow, double splitRatio, public static void deleteTable(TableId tableId, boolean insertDeletes, ServerContext context, ServiceLock lock) throws AccumuloException { - try (Scanner ms = new ScannerImpl(context, MetadataTable.ID, Authorizations.EMPTY); + try ( + Scanner ms = new ScannerImpl(context, MetadataTable.ID, Authorizations.EMPTY, + context.getScanIteratorValidator(MetadataTable.NAME)); BatchWriter bw = new BatchWriterImpl(context, MetadataTable.ID, new BatchWriterConfig().setMaxMemory(1000000) .setMaxLatency(120000L, TimeUnit.MILLISECONDS).setMaxWriteThreads(2))) { diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java b/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java index 22275c9e119..62e1b29cf3f 100644 --- a/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java +++ b/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java @@ -174,7 +174,8 @@ public Stream getBlipPaths() throws TableNotFoundException { int blipPrefixLen = MetadataSchema.BlipSection.getRowPrefix().length(); var scanner = - new IsolatedScanner(context.createScanner(level.metaTable(), Authorizations.EMPTY)); + new IsolatedScanner(context.createScanner(level.metaTable(), Authorizations.EMPTY), + context.getScanIteratorValidator(level.metaTable())); scanner.setRange(MetadataSchema.BlipSection.getRange()); return scanner.stream() .map(entry -> entry.getKey().getRow().toString().substring(blipPrefixLen)) diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java index b53d54edacf..389b2a2c4d3 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java @@ -27,7 +27,6 @@ import java.util.Set; import org.apache.accumulo.core.Constants; -import org.apache.accumulo.core.client.AccumuloClient; import org.apache.accumulo.core.client.IsolatedScanner; import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.data.Key; @@ -44,6 +43,7 @@ import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.manager.Manager; import org.apache.accumulo.manager.tableOps.ManagerRepo; +import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.fs.VolumeManager; import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection; import org.apache.accumulo.server.zookeeper.DistributedWorkQueue; @@ -120,9 +120,10 @@ public Repo call(long tid, Manager manager) throws Exception { */ // determine which failed files were loaded - AccumuloClient client = manager.getContext(); + ServerContext client = manager.getContext(); try (Scanner mscanner = - new IsolatedScanner(client.createScanner(MetadataTable.NAME, Authorizations.EMPTY))) { + new IsolatedScanner(client.createScanner(MetadataTable.NAME, Authorizations.EMPTY), + client.getScanIteratorValidator(MetadataTable.NAME))) { mscanner.setRange(new KeyExtent(tableId, null, null).toMetaRange()); mscanner.fetchColumnFamily(BulkFileColumnFamily.NAME); diff --git a/test/src/main/java/org/apache/accumulo/test/ClientSideIteratorIT.java b/test/src/main/java/org/apache/accumulo/test/ClientSideIteratorIT.java index 824284ee981..fc33e2e9696 100644 --- a/test/src/main/java/org/apache/accumulo/test/ClientSideIteratorIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ClientSideIteratorIT.java @@ -217,8 +217,8 @@ private void runPluginEnvTest(Set expected) throws Exception { // accumulo config from iterators also. client.tableOperations().offline(tableName, true); var context = (ClientContext) client; - try (OfflineScanner offlineScanner = - new OfflineScanner(context, context.getTableId(tableName), Authorizations.EMPTY)) { + try (OfflineScanner offlineScanner = new OfflineScanner(context, context.getTableId(tableName), + Authorizations.EMPTY, context.getScanIteratorValidator(tableName))) { initCalled.set(false); offlineScanner.addScanIterator(new IteratorSetting(100, "filter", TestPropFilter.class)); assertEquals(expected, offlineScanner.stream().map(e -> e.getKey().getRowData().toString()) diff --git a/test/src/main/java/org/apache/accumulo/test/CloseScannerIT.java b/test/src/main/java/org/apache/accumulo/test/CloseScannerIT.java index 6a4c4f68121..0532fd25692 100644 --- a/test/src/main/java/org/apache/accumulo/test/CloseScannerIT.java +++ b/test/src/main/java/org/apache/accumulo/test/CloseScannerIT.java @@ -28,6 +28,7 @@ import org.apache.accumulo.core.client.IsolatedScanner; import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.client.ScannerBase; +import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.security.Authorizations; @@ -126,7 +127,8 @@ private static Scanner createScanner(AccumuloClient client, String tableName, in throws Exception { Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY); if (i % 2 == 0) { - scanner = new IsolatedScanner(scanner); + scanner = new IsolatedScanner(scanner, + ((ClientContext) client).getScanIteratorValidator(tableName)); } return scanner; } diff --git a/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java b/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java index bccbc0ced25..563c0715b8e 100644 --- a/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java @@ -69,7 +69,9 @@ import org.apache.accumulo.core.client.TableOfflineException; import org.apache.accumulo.core.client.admin.NewTableConfiguration; import org.apache.accumulo.core.client.security.tokens.PasswordToken; +import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.clientImpl.ConditionalWriterImpl; +import org.apache.accumulo.core.clientImpl.Namespace; import org.apache.accumulo.core.conf.ClientProperty; import org.apache.accumulo.core.data.ArrayByteSequence; import org.apache.accumulo.core.data.ByteSequence; @@ -88,6 +90,7 @@ import org.apache.accumulo.core.iterators.user.VersioningIterator; import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.security.ColumnVisibility; +import org.apache.accumulo.core.security.NamespacePermission; import org.apache.accumulo.core.security.SystemPermission; import org.apache.accumulo.core.security.TablePermission; import org.apache.accumulo.core.util.FastFormat; @@ -265,6 +268,7 @@ public void testFields() throws Exception { client1.securityOperations().changeUserAuthorizations(user, auths); client1.securityOperations().grantSystemPermission(user, SystemPermission.CREATE_TABLE); + client1.securityOperations().grantNamespacePermission(user, Namespace.DEFAULT.name(), NamespacePermission.ALTER_NAMESPACE); try (AccumuloClient client2 = Accumulo.newClient().from(client1.properties()).as(user, user1.getToken()).build()) { @@ -1195,7 +1199,8 @@ public MutatorTask(String tableName, AccumuloClient client, ArrayList client.tableOperations() + var exception = assertThrows(AccumuloException.class, () -> client.tableOperations() .create(table, new NewTableConfiguration().attachIterator(iterator1))); - assertTrue(exception.getMessage().contains("iterator priority conflict")); + assertTrue(exception.getMessage().contains("conflict with default table iterator")); // add an iterator with same name as the default iterator var iterator2 = new IteratorSetting(10, "vers", "foo.bar"); - exception = assertThrows(IllegalStateException.class, () -> client.tableOperations() - .create(table, new NewTableConfiguration().attachIterator(iterator2))); - assertTrue(exception.getMessage().contains("iterator name conflict")); + exception = assertThrows(AccumuloException.class, () -> client.tableOperations().create(table, + new NewTableConfiguration().attachIterator(iterator2))); + assertTrue(exception.getMessage().contains("conflict with default table iterator")); // try to attach the exact default iterators, should not present a conflict as they are // equivalent to what would be added IteratorConfigUtil.getInitialTableIteratorSettings().forEach((setting, scopes) -> { @@ -561,17 +561,17 @@ public void testConflictsWithDefaults() throws Exception { for (IteratorScope iterScope : IteratorScope.values()) { props.put(Property.TABLE_ITERATOR_PREFIX + iterScope.name() + ".foo", "20,foo.bar"); } - exception = assertThrows(IllegalStateException.class, () -> client.tableOperations() + exception = assertThrows(AccumuloException.class, () -> client.tableOperations() .create(table2, new NewTableConfiguration().setProperties(props))); - assertTrue(exception.getMessage().contains("iterator priority conflict")); + assertTrue(exception.getMessage().contains("conflict with default table iterator")); props.clear(); // add an iterator with same name as the default iterator for (IteratorScope iterScope : IteratorScope.values()) { props.put(Property.TABLE_ITERATOR_PREFIX + iterScope.name() + ".vers", "10,foo.bar"); } - exception = assertThrows(IllegalStateException.class, () -> client.tableOperations() + exception = assertThrows(AccumuloException.class, () -> client.tableOperations() .create(table2, new NewTableConfiguration().setProperties(props))); - assertTrue(exception.getMessage().contains("iterator name conflict")); + assertTrue(exception.getMessage().contains("conflict with default table iterator")); props.clear(); // try to attach the exact default iterators, should not present a conflict as they are // equivalent to what would be added @@ -583,7 +583,7 @@ public void testConflictsWithDefaults() throws Exception { */ // setting a value different from default should throw props.put(Property.TABLE_CONSTRAINT_PREFIX + "1", "foo"); - exception = assertThrows(IllegalStateException.class, () -> client.tableOperations() + exception = assertThrows(AccumuloException.class, () -> client.tableOperations() .create(table3, new NewTableConfiguration().setProperties(props))); assertTrue(exception.getMessage() .contains("conflict for property " + Property.TABLE_CONSTRAINT_PREFIX + "1")); diff --git a/test/src/main/java/org/apache/accumulo/test/SampleIT.java b/test/src/main/java/org/apache/accumulo/test/SampleIT.java index 369444ac0a5..bb30fd2605b 100644 --- a/test/src/main/java/org/apache/accumulo/test/SampleIT.java +++ b/test/src/main/java/org/apache/accumulo/test/SampleIT.java @@ -137,7 +137,8 @@ public void testBasic() throws Exception { Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY); Scanner isoScanner = - new IsolatedScanner(client.createScanner(tableName, Authorizations.EMPTY)); + new IsolatedScanner(client.createScanner(tableName, Authorizations.EMPTY), + ((ClientContext) client).getScanIteratorValidator(tableName)); Scanner csiScanner = new ClientSideIteratorScanner(client.createScanner(tableName, Authorizations.EMPTY)); scanner.setSamplerConfiguration(SC1); @@ -210,8 +211,8 @@ private Scanner newOfflineScanner(AccumuloClient client, String tableName, Strin client.tableOperations().clone(tableName, clone, false, em, es); client.tableOperations().offline(clone, true); TableId cloneID = TableId.of(client.tableOperations().tableIdMap().get(clone)); - OfflineScanner oScanner = - new OfflineScanner((ClientContext) client, cloneID, Authorizations.EMPTY); + OfflineScanner oScanner = new OfflineScanner((ClientContext) client, cloneID, + Authorizations.EMPTY, ((ClientContext) client).getScanIteratorValidator(clone)); if (sc != null) { oScanner.setSamplerConfiguration(sc); } @@ -315,7 +316,8 @@ public void testIterator() throws Exception { Scanner oScanner = null; try { scanner = client.createScanner(tableName, Authorizations.EMPTY); - isoScanner = new IsolatedScanner(client.createScanner(tableName, Authorizations.EMPTY)); + isoScanner = new IsolatedScanner(client.createScanner(tableName, Authorizations.EMPTY), + ((ClientContext) client).getScanIteratorValidator(tableName)); csiScanner = new ClientSideIteratorScanner(client.createScanner(tableName, Authorizations.EMPTY)); bScanner = client.createBatchScanner(tableName, Authorizations.EMPTY, 2); @@ -414,7 +416,8 @@ public void testSampleNotPresent() throws Exception { } Scanner scanner = client.createScanner(tableName); - Scanner isoScanner = new IsolatedScanner(client.createScanner(tableName)); + Scanner isoScanner = new IsolatedScanner(client.createScanner(tableName), + ((ClientContext) client).getScanIteratorValidator(tableName)); isoScanner.setBatchSize(10); Scanner csiScanner = new ClientSideIteratorScanner(client.createScanner(tableName)); try (BatchScanner bScanner = client.createBatchScanner(tableName)) { diff --git a/test/src/main/java/org/apache/accumulo/test/ScanConsistencyIT.java b/test/src/main/java/org/apache/accumulo/test/ScanConsistencyIT.java index 0c03de99b02..9f1f90a2a30 100644 --- a/test/src/main/java/org/apache/accumulo/test/ScanConsistencyIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ScanConsistencyIT.java @@ -50,6 +50,7 @@ import org.apache.accumulo.core.client.admin.CompactionConfig; import org.apache.accumulo.core.client.rfile.RFile; import org.apache.accumulo.core.client.rfile.RFileWriter; +import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Range; @@ -353,7 +354,9 @@ private static ScanStats batchScanData(TestContext tctx, Range range) throws Exc private static ScanStats scanData(TestContext tctx, Random random, Range range, boolean scanIsolated) throws Exception { try (ExpectedScanData expectedScanData = tctx.dataTracker.beginScan(); - Scanner scanner = scanIsolated ? new IsolatedScanner(tctx.client.createScanner(tctx.table)) + Scanner scanner = scanIsolated + ? new IsolatedScanner(tctx.client.createScanner(tctx.table), + ((ClientContext) tctx.client).getScanIteratorValidator(tctx.table)) : tctx.client.createScanner(tctx.table)) { Set expected = expectedScanData.getExpectedData(range).collect(Collectors.toSet()); diff --git a/test/src/main/java/org/apache/accumulo/test/ScanFlushWithTimeIT.java b/test/src/main/java/org/apache/accumulo/test/ScanFlushWithTimeIT.java index 7c88ee40b34..16c9f96c9bb 100644 --- a/test/src/main/java/org/apache/accumulo/test/ScanFlushWithTimeIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ScanFlushWithTimeIT.java @@ -34,6 +34,7 @@ import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.client.ScannerBase; +import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.security.Authorizations; @@ -86,7 +87,8 @@ public void test() throws Exception { testScanner(s, 1200); log.info("IsolatedScanner"); - IsolatedScanner is = new IsolatedScanner(s); + IsolatedScanner is = + new IsolatedScanner(s, ((ClientContext) c).getScanIteratorValidator(tableName)); is.setReadaheadThreshold(1); // buffers an entire row testScanner(is, 2200); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java new file mode 100644 index 00000000000..bf4248a87f6 --- /dev/null +++ b/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java @@ -0,0 +1,383 @@ +/* + * 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.test.functional; + +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.EnumSet; +import java.util.List; +import java.util.function.Consumer; + +import org.apache.accumulo.core.client.Accumulo; +import org.apache.accumulo.core.client.AccumuloClient; +import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.IteratorSetting; +import org.apache.accumulo.core.client.Scanner; +import org.apache.accumulo.core.client.admin.NamespaceOperations; +import org.apache.accumulo.core.client.admin.NewTableConfiguration; +import org.apache.accumulo.core.client.admin.TableOperations; +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.core.iterators.IteratorUtil; +import org.apache.accumulo.core.iteratorsImpl.IteratorConfigUtil; +import org.apache.accumulo.harness.SharedMiniClusterBase; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.function.Executable; + +/** + * Tests that iterator conflicts are detected and cause exceptions. Iterators can be added multiple + * ways. This test ensures that: + *

+ * - {@link Scanner#addScanIterator(IteratorSetting)} + *

+ * - {@link TableOperations#setProperty(String, String, String)} + *

+ * - {@link TableOperations#modifyProperties(String, Consumer)} + *

+ * - {@link NewTableConfiguration#attachIterator(IteratorSetting, EnumSet)} + *

+ * - {@link TableOperations#attachIterator(String, IteratorSetting, EnumSet)} + *

+ * All fail when conflicts arise from: + *

+ * - Iterators attached directly to a table + *

+ * - Iterators attached to a namespace, inherited by a table + *

+ * - Default table iterators, but should not fail if {@link NewTableConfiguration#withoutDefaults()} + * is specified + *

+ * - Adding the exact iterator already present should not fail + *

+ */ +public class IteratorConflictsIT extends SharedMiniClusterBase { + private static TableOperations tops; + private static NamespaceOperations nops; + private static AccumuloClient client; + private static final IteratorSetting iter1 = new IteratorSetting(99, "iter1name", "foo"); + private static final String iter1Key = Property.TABLE_ITERATOR_PREFIX + + IteratorUtil.IteratorScope.scan.name().toLowerCase() + "." + iter1.getName(); + private static final String iter1Val = "99,foo"; + private static final IteratorSetting iter1PrioConflict = + new IteratorSetting(99, "othername", "foo"); + private static final IteratorSetting iter1NameConflict = + new IteratorSetting(101, iter1.getName(), "foo"); + private static final String iter1PrioConflictKey = Property.TABLE_ITERATOR_PREFIX + + IteratorUtil.IteratorScope.scan.name().toLowerCase() + ".othername"; + private static final String iter1PrioConflictVal = "99,foo"; + private static final String iter1NameConflictKey = Property.TABLE_ITERATOR_PREFIX + + IteratorUtil.IteratorScope.scan.name().toLowerCase() + "." + iter1.getName(); + private static final String iter1NameConflictVal = "101,foo"; + private static final IteratorSetting defaultIterPrioConflict = + new IteratorSetting(20, "bar", "foo"); + private static final IteratorSetting defaultIterNameConflict = + new IteratorSetting(101, "vers", "foo"); + private static final IteratorSetting defaultTableIter = + IteratorConfigUtil.getInitialTableIteratorSettings().keySet().iterator().next(); + private static final String defaultIterPrioConflictKey = Property.TABLE_ITERATOR_PREFIX + + IteratorUtil.IteratorScope.scan.name().toLowerCase() + ".foo"; + private static final String defaultIterPrioConflictVal = defaultTableIter.getPriority() + ",bar"; + private static final String defaultIterNameConflictKey = Property.TABLE_ITERATOR_PREFIX + + IteratorUtil.IteratorScope.scan.name().toLowerCase() + "." + defaultTableIter.getName(); + private static final String defaultIterNameConflictVal = "99,bar"; + private static final String defaultIterKey = Property.TABLE_ITERATOR_PREFIX.getKey() + + IteratorUtil.IteratorScope.scan.name().toLowerCase() + "." + defaultTableIter.getName(); + private static final String defaultIterVal = + defaultTableIter.getPriority() + "," + defaultTableIter.getIteratorClass(); + + @BeforeAll + public static void startup() throws Exception { + SharedMiniClusterBase.startMiniCluster(); + client = Accumulo.newClient().from(getClientProps()).build(); + tops = client.tableOperations(); + nops = client.namespaceOperations(); + } + + @AfterAll + public static void shutdown() throws Exception { + client.close(); + SharedMiniClusterBase.stopMiniCluster(); + } + + @Test + public void testTableIterConflict() throws Exception { + final String[] tableNames = getUniqueNames(4); + String table1 = tableNames[0]; + String table2 = tableNames[1]; + String table3 = tableNames[2]; + String table4 = tableNames[3]; + for (String table : tableNames) { + tops.create(table); + } + + // testing Scanner.addScanIterator + try (var scanner = client.createScanner(table1)) { + testTableIterConflict(table1, IllegalArgumentException.class, + () -> scanner.addScanIterator(iter1PrioConflict), + () -> scanner.addScanIterator(iter1NameConflict)); + } + + // testing TableOperations.setProperty + testTableIterConflict(table2, AccumuloException.class, + () -> tops.setProperty(table2, iter1PrioConflictKey, iter1PrioConflictVal), + () -> tops.setProperty(table2, iter1NameConflictKey, iter1NameConflictVal)); + + // testing TableOperations.modifyProperties + testTableIterConflict(table3, AccumuloException.class, + () -> tops.modifyProperties(table3, + props -> props.put(iter1PrioConflictKey, iter1PrioConflictVal)), + () -> tops.modifyProperties(table3, + props -> props.put(iter1NameConflictKey, iter1NameConflictVal))); + + // NewTableConfiguration.attachIterator is not applicable for this test + // Attaching the iterator to the table requires the table to exist, but testing + // NewTableConfiguration.attachIterator requires that the table does not exist + + // testing TableOperations.attachIterator + testTableIterConflict(table4, AccumuloException.class, + () -> tops.attachIterator(table4, iter1PrioConflict), + () -> tops.attachIterator(table4, iter1NameConflict)); + } + + private void testTableIterConflict(String table, Class exceptionClass, + Executable iterPrioConflictExec, Executable iterNameConflictExec) throws Exception { + tops.attachIterator(table, iter1); + var e = assertThrows(exceptionClass, iterPrioConflictExec); + assertTrue(e.getMessage().contains("iterator priority conflict") + && e.getMessage().contains(iter1.getName())); + e = assertThrows(exceptionClass, iterNameConflictExec); + assertTrue(e.getMessage().contains("iterator name conflict") + && e.getMessage().contains(iter1.getName())); + } + + @Test + public void testNamespaceIterConflict() throws Exception { + final String[] names = getUniqueNames(10); + String ns1 = names[0]; + String table1 = ns1 + "." + names[1]; + String ns2 = names[2]; + String table2 = ns2 + "." + names[3]; + String ns3 = names[4]; + String table3 = ns3 + "." + names[5]; + String ns4 = names[6]; + String table4 = ns4 + "." + names[7]; + String ns5 = names[8]; + String table5 = ns5 + "." + names[9]; + for (String ns : List.of(ns1, ns2, ns3, ns4, ns5)) { + nops.create(ns); + } + // don't create table4 + for (String table : List.of(table1, table2, table3, table5)) { + tops.create(table); + } + + // testing Scanner.addScanIterator + try (var scanner = client.createScanner(table1)) { + testNamespaceIterConflict(ns1, IllegalArgumentException.class, + () -> scanner.addScanIterator(iter1PrioConflict), + () -> scanner.addScanIterator(iter1NameConflict)); + } + + // testing TableOperations.setProperty + testNamespaceIterConflict(ns2, AccumuloException.class, + () -> tops.setProperty(table2, iter1PrioConflictKey, iter1PrioConflictVal), + () -> tops.setProperty(table2, iter1NameConflictKey, iter1NameConflictVal)); + + // testing TableOperations.modifyProperties + testNamespaceIterConflict(ns3, AccumuloException.class, + () -> tops.modifyProperties(table3, + props -> props.put(iter1PrioConflictKey, iter1PrioConflictVal)), + () -> tops.modifyProperties(table3, + props -> props.put(iter1NameConflictKey, iter1NameConflictVal))); + + // testing NewTableConfiguration.attachIterator + testNamespaceIterConflict(ns4, AccumuloException.class, + () -> tops.create(table4, new NewTableConfiguration().attachIterator(iter1PrioConflict)), + () -> tops.create(table4, new NewTableConfiguration().attachIterator(iter1NameConflict))); + + // testing TableOperations.attachIterator + testNamespaceIterConflict(ns5, AccumuloException.class, + () -> tops.attachIterator(table5, iter1PrioConflict), + () -> tops.attachIterator(table5, iter1NameConflict)); + } + + private void testNamespaceIterConflict(String ns, Class exceptionClass, + Executable iterPrioConflictExec, Executable iterNameConflictExec) throws Exception { + nops.attachIterator(ns, iter1); + + var e = assertThrows(exceptionClass, iterPrioConflictExec); + assertTrue(e.getMessage().contains("iterator priority conflict") + && e.getMessage().contains(iter1.getName())); + e = assertThrows(exceptionClass, iterNameConflictExec); + assertTrue(e.getMessage().contains("iterator name conflict") + && e.getMessage().contains(iter1.getName())); + } + + @Test + public void testDefaultIterConflict() throws Throwable { + final String[] tables = getUniqueNames(11); + String defaultsTable1 = tables[0]; + String noDefaultsTable1 = tables[1]; + String defaultsTable2 = tables[2]; + String noDefaultsTable2 = tables[3]; + String defaultsTable3 = tables[4]; + String noDefaultsTable3 = tables[5]; + String defaultsTable4 = tables[6]; + String noDefaultsTable4 = tables[7]; + String noDefaultsTable5 = tables[9]; + String defaultsTable5 = tables[8]; + String noDefaultsTable6 = tables[10]; + // don't create defaultsTable4 + for (String defaultsTable : List.of(defaultsTable1, defaultsTable2, defaultsTable3, + defaultsTable5)) { + tops.create(defaultsTable); + } + // don't create noDefaultsTable4 or noDefaultsTable5 + for (String noDefaultsTable : List.of(noDefaultsTable1, noDefaultsTable2, noDefaultsTable3, + noDefaultsTable6)) { + tops.create(noDefaultsTable, new NewTableConfiguration().withoutDefaults()); + } + + // testing Scanner.addScanIterator + try (var defaultsScanner = client.createScanner(defaultsTable1); + var noDefaultsScanner = client.createScanner(noDefaultsTable1)) { + testDefaultIterConflict(IllegalArgumentException.class, + () -> defaultsScanner.addScanIterator(defaultIterPrioConflict), + () -> defaultsScanner.addScanIterator(defaultIterNameConflict), + () -> noDefaultsScanner.addScanIterator(defaultIterPrioConflict), + () -> noDefaultsScanner.addScanIterator(defaultIterNameConflict)); + } + + // testing TableOperations.setProperty + testDefaultIterConflict(AccumuloException.class, + () -> tops.setProperty(defaultsTable2, defaultIterPrioConflictKey, + defaultIterPrioConflictVal), + () -> tops.setProperty(defaultsTable2, defaultIterNameConflictKey, + defaultIterNameConflictVal), + () -> tops.setProperty(noDefaultsTable2, defaultIterPrioConflictKey, + defaultIterPrioConflictVal), + () -> tops.setProperty(noDefaultsTable2, defaultIterNameConflictKey, + defaultIterNameConflictVal)); + + // testing TableOperations.modifyProperties + testDefaultIterConflict(AccumuloException.class, + () -> tops.modifyProperties(defaultsTable3, + props -> props.put(defaultIterPrioConflictKey, defaultIterPrioConflictVal)), + () -> tops.modifyProperties(defaultsTable3, + props -> props.put(defaultIterNameConflictKey, defaultIterNameConflictVal)), + () -> tops.modifyProperties(noDefaultsTable3, + props -> props.put(defaultIterPrioConflictKey, defaultIterPrioConflictVal)), + () -> tops.modifyProperties(noDefaultsTable3, + props -> props.put(defaultIterNameConflictKey, defaultIterNameConflictVal))); + + // testing NewTableConfiguration.attachIterator + testDefaultIterConflict(AccumuloException.class, + () -> tops.create(defaultsTable4, + new NewTableConfiguration().attachIterator(defaultIterPrioConflict)), + () -> tops.create(defaultsTable4, + new NewTableConfiguration().attachIterator(defaultIterNameConflict)), + () -> tops.create(noDefaultsTable4, + new NewTableConfiguration().attachIterator(defaultIterPrioConflict).withoutDefaults()), + () -> tops.create(noDefaultsTable5, + new NewTableConfiguration().attachIterator(defaultIterNameConflict).withoutDefaults())); + + // testing TableOperations.attachIterator + testDefaultIterConflict(AccumuloException.class, + () -> tops.attachIterator(defaultsTable5, defaultIterPrioConflict), + () -> tops.attachIterator(defaultsTable5, defaultIterNameConflict), + () -> tops.attachIterator(noDefaultsTable6, defaultIterPrioConflict), + () -> tops.attachIterator(noDefaultsTable6, defaultIterNameConflict)); + } + + private void testDefaultIterConflict(Class exceptionClass, + Executable defaultsTableOp1, Executable defaultsTableOp2, Executable noDefaultsTableOp1, + Executable noDefaultsTableOp2) throws Throwable { + var e = assertThrows(exceptionClass, defaultsTableOp1); + // exception message different depending on operation, just checking essential info + assertTrue( + e.getMessage().contains("VersioningIterator") && e.getMessage().contains("conflict")); + + e = assertThrows(exceptionClass, defaultsTableOp2); + // exception message different depending on operation, just checking essential info + assertTrue( + e.getMessage().contains("VersioningIterator") && e.getMessage().contains("conflict")); + + noDefaultsTableOp1.execute(); // should NOT fail + + noDefaultsTableOp2.execute(); // should NOT fail + } + + @Test + public void testSameIterNoConflict() throws Throwable { + final String[] names = getUniqueNames(7); + final String table1 = names[0]; + final String table2 = names[1]; + final String table3 = names[2]; + final String ns = names[3]; + final String table4 = ns + "." + names[4]; + final String table5 = names[5]; + final String table6 = names[6]; + // don't create table4 or table5 + for (String table : List.of(table1, table2, table3, table6)) { + tops.create(table); + tops.attachIterator(table, iter1); + } + + // testing Scanner.addScanIterator + try (var scanner = client.createScanner(table1)) { + testSameIterNoConflict(() -> scanner.addScanIterator(iter1), + () -> scanner.addScanIterator(defaultTableIter)); + } + + // testing TableOperations.setProperty + // note that this is not technically the exact same iterator since the default iterator has + // options (which are separate properties), but this call has no effect on the + // property map/iterators, so this call should not throw + testSameIterNoConflict(() -> tops.setProperty(table2, iter1Key, iter1Val), + () -> tops.setProperty(table2, defaultIterKey, defaultIterVal)); + + // testing TableOperations.modifyProperties + // note that this is not technically the exact same iterator since the default iterator has + // options (which are separate properties), but this call has no effect on the + // property map/iterators, so this call should not throw + testSameIterNoConflict( + () -> tops.modifyProperties(table3, props -> props.put(iter1Key, iter1Val)), + () -> tops.modifyProperties(table3, props -> props.put(defaultIterKey, defaultIterVal))); + + // testing NewTableConfiguration.attachIterator + nops.create(ns); + nops.attachIterator(ns, iter1); + testSameIterNoConflict( + () -> tops.create(table4, new NewTableConfiguration().attachIterator(iter1)), + () -> tops.create(table5, new NewTableConfiguration().attachIterator(defaultTableIter))); + + // testing TableOperations.attachIterator + testSameIterNoConflict(() -> tops.attachIterator(table6, iter1), + () -> tops.attachIterator(table6, defaultTableIter)); + } + + private void testSameIterNoConflict(Executable addIter1Executable, + Executable addDefaultIterExecutable) throws Throwable { + // should be able to add same exact iterator + addIter1Executable.execute(); + addDefaultIterExecutable.execute(); + } +} diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java index 50827f3dacf..b1ae0a2fad2 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java @@ -142,7 +142,8 @@ public void testShutdownOnlyTServerWithUserTable() throws Exception { Runnable task = () -> { while (true) { - try (var scanner = new IsolatedScanner(client.createScanner(tableName))) { + try (var scanner = new IsolatedScanner(client.createScanner(tableName), + ((ClientContext) client).getScanIteratorValidator(tableName))) { // TODO maybe do not close scanner? The following limit was placed on the stream to // avoid reading all the data possibly leaving a scan session active on the tserver AtomicInteger count = new AtomicInteger(0); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/PerTableCryptoIT.java b/test/src/main/java/org/apache/accumulo/test/functional/PerTableCryptoIT.java index 53306cf33d0..368cebc3312 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/PerTableCryptoIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/PerTableCryptoIT.java @@ -163,7 +163,8 @@ public void testOfflineIterator() throws Exception { TableId tableId = TableId.of(c.tableOperations().tableIdMap().get(tableName)); c.tableOperations().offline(tableName, true); - try (var oScanner = new OfflineScanner((ClientContext) c, tableId, Authorizations.EMPTY)) { + try (var oScanner = new OfflineScanner((ClientContext) c, tableId, Authorizations.EMPTY, + ((ClientContext) c).getScanIteratorValidator(tableName))) { long count = oScanner.stream().count(); assertEquals(count, 100_000); } diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ScanIteratorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ScanIteratorIT.java index 53fb54dc98c..647c4dfca5a 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/ScanIteratorIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/ScanIteratorIT.java @@ -93,6 +93,8 @@ public void setup() throws Exception { accumuloClient.securityOperations().grantTablePermission(user, tableName, TablePermission.READ); accumuloClient.securityOperations().grantTablePermission(user, tableName, TablePermission.WRITE); + accumuloClient.securityOperations().grantTablePermission(user, tableName, + TablePermission.ALTER_TABLE); accumuloClient.securityOperations().changeUserAuthorizations(user, AuthsIterator.AUTHS); } diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java index 7b5b59accc4..50e4ada2d1c 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java @@ -254,7 +254,8 @@ private void splitPartiallyAndRecover(ServerContext context, KeyExtent extent, K private void ensureTabletHasNoUnexpectedMetadataEntries(ServerContext context, KeyExtent extent, SortedMap expectedMapFiles) throws Exception { - try (Scanner scanner = new ScannerImpl(context, MetadataTable.ID, Authorizations.EMPTY)) { + try (Scanner scanner = new ScannerImpl(context, MetadataTable.ID, Authorizations.EMPTY, + context.getScanIteratorValidator(MetadataTable.NAME))) { scanner.setRange(extent.toMetaRange()); HashSet expectedColumns = new HashSet<>(); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java index 64e167c902a..ad3c55d04cc 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java @@ -278,7 +278,8 @@ private void copyTable(AccumuloClient client, String source, String copy) List mutations = new ArrayList<>(); try (Scanner scanner = client.createScanner(source, Authorizations.EMPTY)) { - RowIterator rows = new RowIterator(new IsolatedScanner(scanner)); + RowIterator rows = new RowIterator( + new IsolatedScanner(scanner, ((ClientContext) client).getScanIteratorValidator(source))); while (rows.hasNext()) { Iterator> row = rows.next(); diff --git a/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java index 00e7cfdc033..a7d0713f730 100644 --- a/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java +++ b/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java @@ -357,6 +357,7 @@ public void user() throws Exception { String perms = ts.exec("userpermissions -u xyzzy", true); assertTrue(perms.contains("Table permissions (" + MetadataTable.NAME + "): Table.READ")); ts.exec("grant -u xyzzy -s System.CREATE_TABLE", true); + ts.exec("grant -u xyzzy -ns \"\" Namespace.ALTER_NAMESPACE", true); perms = ts.exec("userpermissions -u xyzzy", true); assertTrue(perms.contains("")); ts.exec("grant -u " + getPrincipal() + " -t " + MetadataTable.NAME + " Table.WRITE", true); From 06c1800b8b6c6ae85cc61723090b80a95cf812fe Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Wed, 7 Jan 2026 14:24:33 -0500 Subject: [PATCH 02/21] Prevents iterator conflicts p2: Adds conflict checks to: - NamespaceOperations.attachIterator (was previously only checking for conflicts with iterators in the namespace, now also checks for conflicts with iterators in the tables of the namespace) - NamespaceOperations.setProperty (check conflicts with namespace iterators and all tables in the namespace) - NamespaceOperations.modifyProperties (check conflicts with namespace iterators and all tables in the namespace) New tests to IteratorConflictsIT to test the above --- .../clientImpl/NamespaceOperationsHelper.java | 42 +--- .../clientImpl/NamespaceOperationsImpl.java | 10 + .../clientImpl/TableOperationsHelper.java | 58 +---- .../core/clientImpl/TableOperationsImpl.java | 27 +-- .../iteratorsImpl/IteratorConfigUtil.java | 145 ++++++++++++ .../accumulo/test/ConditionalWriterIT.java | 3 +- .../test/functional/IteratorConflictsIT.java | 218 ++++++++++++++---- 7 files changed, 340 insertions(+), 163 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsHelper.java index d05e16afc6c..360f05df11e 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsHelper.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsHelper.java @@ -32,6 +32,7 @@ import org.apache.accumulo.core.client.admin.NamespaceOperations; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; +import org.apache.accumulo.core.iteratorsImpl.IteratorConfigUtil; public abstract class NamespaceOperationsHelper implements NamespaceOperations { @@ -146,45 +147,8 @@ public void checkIteratorConflicts(String namespace, IteratorSetting setting, if (!exists(namespace)) { throw new NamespaceNotFoundException(null, namespace, null); } - for (IteratorScope scope : scopes) { - String scopeStr = - String.format("%s%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase()); - String nameStr = String.format("%s.%s", scopeStr, setting.getName()); - String optStr = String.format("%s.opt.", nameStr); - Map optionConflicts = new TreeMap<>(); - for (Entry property : this.getProperties(namespace)) { - if (property.getKey().startsWith(scopeStr)) { - if (property.getKey().equals(nameStr)) { - throw new AccumuloException(new IllegalArgumentException("iterator name conflict for " - + setting.getName() + ": " + property.getKey() + "=" + property.getValue())); - } - if (property.getKey().startsWith(optStr)) { - optionConflicts.put(property.getKey(), property.getValue()); - } - if (property.getKey().contains(".opt.")) { - continue; - } - String[] parts = property.getValue().split(","); - if (parts.length != 2) { - throw new AccumuloException("Bad value for existing iterator setting: " - + property.getKey() + "=" + property.getValue()); - } - try { - if (Integer.parseInt(parts[0]) == setting.getPriority()) { - throw new AccumuloException(new IllegalArgumentException( - "iterator priority conflict: " + property.getKey() + "=" + property.getValue())); - } - } catch (NumberFormatException e) { - throw new AccumuloException("Bad value for existing iterator setting: " - + property.getKey() + "=" + property.getValue()); - } - } - } - if (!optionConflicts.isEmpty()) { - throw new AccumuloException(new IllegalArgumentException( - "iterator options conflict for " + setting.getName() + ": " + optionConflicts)); - } - } + var props = this.getNamespaceProperties(namespace); + IteratorConfigUtil.checkIteratorConflicts(props, setting, scopes); } @Override diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java index 451496ed9c5..6c7b870a180 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java @@ -57,6 +57,7 @@ import org.apache.accumulo.core.data.constraints.Constraint; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; +import org.apache.accumulo.core.iteratorsImpl.IteratorConfigUtil; import org.apache.accumulo.core.manager.thrift.FateOperation; import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; import org.apache.accumulo.core.trace.TraceUtil; @@ -186,6 +187,8 @@ public void setProperty(final String namespace, final String property, final Str checkArgument(value != null, "value is null"); try { + IteratorConfigUtil.checkIteratorConflicts(tableOps, this, namespace, property, value); + ThriftClientTypes.MANAGER.executeVoidTableCommand(context, client -> client.setNamespaceProperty(TraceUtil.traceInfo(), context.rpcCreds(), namespace, property, value)); @@ -215,6 +218,11 @@ private Map tryToModifyProperties(final String namespace, // from here on the code is assured to always be dealing with the same map. vProperties.setProperties(Map.copyOf(vProperties.getProperties())); + for (var property : vProperties.getProperties().entrySet()) { + IteratorConfigUtil.checkIteratorConflicts(tableOps, this, namespace, property.getKey(), + property.getValue()); + } + try { // Send to server ThriftClientTypes.MANAGER.executeVoidTableCommand(context, @@ -374,6 +382,8 @@ public void attachIterator(String namespace, IteratorSetting setting, throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException { // testClassLoad validates the namespace name testClassLoad(namespace, setting.getIteratorClass(), SortedKeyValueIterator.class.getName()); + IteratorConfigUtil.checkIteratorConflictsWithTablesInNamespace(tableOps, namespace, setting, + scopes); super.attachIterator(namespace, setting, scopes); } diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsHelper.java index 901aeb94746..bf138ffe7f8 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsHelper.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsHelper.java @@ -35,6 +35,7 @@ import org.apache.accumulo.core.client.admin.TableOperations; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; +import org.apache.accumulo.core.iteratorsImpl.IteratorConfigUtil; public abstract class TableOperationsHelper implements TableOperations { @@ -139,62 +140,7 @@ public static void checkIteratorConflicts(Map props, IteratorSett EnumSet scopes) throws AccumuloException { checkArgument(setting != null, "setting is null"); checkArgument(scopes != null, "scopes is null"); - for (IteratorScope scope : scopes) { - String scopeStr = - String.format("%s%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase()); - String nameStr = String.format("%s.%s", scopeStr, setting.getName()); - String optStr = String.format("%s.opt.", nameStr); - String valStr = String.format("%s,%s", setting.getPriority(), setting.getIteratorClass()); - Map optionConflicts = new TreeMap<>(); - // skip if the setting is present in the map... not a conflict if exactly the same - if (props.containsKey(nameStr) && props.get(nameStr).equals(valStr) - && containsSameIterOpts(props, setting, optStr)) { - continue; - } - for (Entry property : props.entrySet()) { - if (property.getKey().startsWith(scopeStr)) { - if (property.getKey().equals(nameStr)) { - throw new AccumuloException(new IllegalArgumentException("iterator name conflict for " - + setting.getName() + ": " + property.getKey() + "=" + property.getValue())); - } - if (property.getKey().startsWith(optStr)) { - optionConflicts.put(property.getKey(), property.getValue()); - } - if (property.getKey().contains(".opt.")) { - continue; - } - String[] parts = property.getValue().split(","); - if (parts.length != 2) { - throw new AccumuloException("Bad value for existing iterator setting: " - + property.getKey() + "=" + property.getValue()); - } - try { - if (Integer.parseInt(parts[0]) == setting.getPriority()) { - throw new AccumuloException(new IllegalArgumentException( - "iterator priority conflict: " + property.getKey() + "=" + property.getValue())); - } - } catch (NumberFormatException e) { - throw new AccumuloException("Bad value for existing iterator setting: " - + property.getKey() + "=" + property.getValue()); - } - } - } - if (!optionConflicts.isEmpty()) { - throw new AccumuloException(new IllegalArgumentException( - "iterator options conflict for " + setting.getName() + ": " + optionConflicts)); - } - } - } - - private static boolean containsSameIterOpts(Map props, IteratorSetting setting, - String optStr) { - for (var opt : setting.getOptions().entrySet()) { - final String optKey = optStr + opt.getKey(); - if (!props.containsKey(optKey) || !props.get(optKey).equals(opt.getValue())) { - return false; - } - } - return true; + IteratorConfigUtil.checkIteratorConflicts(props, setting, scopes); } @Override diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java index 669103a0eb3..fb53f88e84f 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java @@ -1022,7 +1022,8 @@ public void setProperty(final String tableName, final String property, final Str checkArgument(value != null, "value is null"); try { - checkIteratorConflicts(Map.copyOf(this.getConfiguration(tableName)), property, value); + IteratorConfigUtil.checkIteratorConflicts(Map.copyOf(this.getConfiguration(tableName)), + property, value); setPropertyNoChecks(tableName, property, value); @@ -1032,27 +1033,6 @@ public void setProperty(final String tableName, final String property, final Str } } - private void checkIteratorConflicts(Map props, String property, String value) - throws AccumuloException, TableNotFoundException, IllegalArgumentException { - if (props.containsKey(property) && props.get(property).equals(value)) { - // setting a property that already exists (i.e., no change) - return; - } - if (IteratorConfigUtil.isNonOptionIterProp(property, value)) { - String[] iterPropParts = property.split("\\."); - IteratorScope scope = IteratorScope.valueOf(iterPropParts[2]); - String iterName = iterPropParts[3]; - String[] priorityAndClass; - if ((priorityAndClass = value.split(",")).length == 2) { - // given a single property, the only way for the property to be equivalent to an existing - // iterator is if the existing iterator has no options (opts are set as separate props) - IteratorSetting givenIter = new IteratorSetting(Integer.parseInt(priorityAndClass[0]), - iterName, priorityAndClass[1]); - checkIteratorConflicts(props, givenIter, EnumSet.of(scope)); - } - } - } - private Map tryToModifyProperties(String tableName, final Consumer> mapMutator) throws AccumuloException, AccumuloSecurityException, IllegalArgumentException, ConcurrentModificationException { @@ -1076,7 +1056,8 @@ private Map tryToModifyProperties(String tableName, try { for (var property : vProperties.getProperties().entrySet()) { - checkIteratorConflicts(configBeforeMut, property.getKey(), property.getValue()); + IteratorConfigUtil.checkIteratorConflicts(configBeforeMut, property.getKey(), + property.getValue()); } } catch (TableNotFoundException e) { throw new AccumuloException(e); diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java index cda8ed96493..e4f347f355e 100644 --- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java @@ -35,7 +35,15 @@ import java.util.stream.Collectors; import org.apache.accumulo.core.classloader.ClassLoaderUtil; +import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.IteratorSetting; +import org.apache.accumulo.core.client.NamespaceNotFoundException; +import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.client.admin.TableOperations; +import org.apache.accumulo.core.clientImpl.Namespace; +import org.apache.accumulo.core.clientImpl.NamespaceOperationsHelper; +import org.apache.accumulo.core.clientImpl.TableOperationsHelper; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Key; @@ -287,6 +295,143 @@ private static Class> loadClass(boolean useAcc return clazz; } + public static void checkIteratorConflicts(Map props, String property, String value) + throws AccumuloException, TableNotFoundException, IllegalArgumentException { + if (props.containsKey(property) && props.get(property).equals(value)) { + // setting a property that already exists (i.e., no change) + return; + } + if (IteratorConfigUtil.isNonOptionIterProp(property, value)) { + String[] iterPropParts = property.split("\\."); + IteratorScope scope = IteratorScope.valueOf(iterPropParts[2]); + String iterName = iterPropParts[3]; + String[] priorityAndClass; + if ((priorityAndClass = value.split(",")).length == 2) { + // given a single property, the only way for the property to be equivalent to an existing + // iterator is if the existing iterator has no options (opts are set as separate props) + IteratorSetting givenIter = new IteratorSetting(Integer.parseInt(priorityAndClass[0]), + iterName, priorityAndClass[1]); + TableOperationsHelper.checkIteratorConflicts(props, givenIter, EnumSet.of(scope)); + } + } + } + + public static void checkIteratorConflicts(TableOperations tableOps, NamespaceOperationsHelper noh, + String namespace, String property, String value) + throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException { + var props = noh.getNamespaceProperties(namespace); + if (props.containsKey(property) && props.get(property).equals(value)) { + // setting a property that already exists (i.e., no change) + return; + } + + // checking for conflicts in the namespace + if (IteratorConfigUtil.isNonOptionIterProp(property, value)) { + String[] iterPropParts = property.split("\\."); + IteratorScope scope = IteratorScope.valueOf(iterPropParts[2]); + String iterName = iterPropParts[3]; + String[] priorityAndClass; + if ((priorityAndClass = value.split(",")).length == 2) { + // given a single property, the only way for the property to be equivalent to an existing + // iterator is if the existing iterator has no options (opts are set as separate props) + IteratorSetting givenIter = new IteratorSetting(Integer.parseInt(priorityAndClass[0]), + iterName, priorityAndClass[1]); + noh.checkIteratorConflicts(namespace, givenIter, EnumSet.of(scope)); + } + } + + // checking for conflicts for the tables in the namespace + checkIteratorConflictsWithTablesInNamespace(tableOps, namespace, property, value); + } + + public static void checkIteratorConflictsWithTablesInNamespace(TableOperations tableOps, + String namespace, IteratorSetting is, EnumSet scopes) + throws AccumuloException { + var tablesInNamespace = tableOps.list().stream() + .filter(t -> t.startsWith(namespace + Namespace.SEPARATOR)).collect(Collectors.toSet()); + try { + for (var table : tablesInNamespace) { + IteratorConfigUtil.checkIteratorConflicts(tableOps.getTableProperties(table), is, scopes); + } + } catch (TableNotFoundException | IllegalArgumentException e) { + throw new AccumuloException(e); + } + } + + public static void checkIteratorConflictsWithTablesInNamespace(TableOperations tableOps, + String namespace, String property, String value) throws AccumuloException { + var tablesInNamespace = tableOps.list().stream() + .filter(t -> t.startsWith(namespace + Namespace.SEPARATOR)).collect(Collectors.toSet()); + try { + for (var table : tablesInNamespace) { + IteratorConfigUtil.checkIteratorConflicts(tableOps.getTableProperties(table), property, + value); + } + } catch (TableNotFoundException | IllegalArgumentException e) { + throw new AccumuloException(e); + } + } + + public static void checkIteratorConflicts(Map props, IteratorSetting setting, + EnumSet scopes) throws AccumuloException { + for (IteratorScope scope : scopes) { + String scopeStr = + String.format("%s%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase()); + String nameStr = String.format("%s.%s", scopeStr, setting.getName()); + String optStr = String.format("%s.opt.", nameStr); + String valStr = String.format("%s,%s", setting.getPriority(), setting.getIteratorClass()); + Map optionConflicts = new TreeMap<>(); + // skip if the setting is present in the map... not a conflict if exactly the same + if (props.containsKey(nameStr) && props.get(nameStr).equals(valStr) + && IteratorConfigUtil.containsSameIterOpts(props, setting, optStr)) { + continue; + } + for (Entry property : props.entrySet()) { + if (property.getKey().startsWith(scopeStr)) { + if (property.getKey().equals(nameStr)) { + throw new AccumuloException(new IllegalArgumentException("iterator name conflict for " + + setting.getName() + ": " + property.getKey() + "=" + property.getValue())); + } + if (property.getKey().startsWith(optStr)) { + optionConflicts.put(property.getKey(), property.getValue()); + } + if (property.getKey().contains(".opt.")) { + continue; + } + String[] parts = property.getValue().split(","); + if (parts.length != 2) { + throw new AccumuloException("Bad value for existing iterator setting: " + + property.getKey() + "=" + property.getValue()); + } + try { + if (Integer.parseInt(parts[0]) == setting.getPriority()) { + throw new AccumuloException(new IllegalArgumentException( + "iterator priority conflict: " + property.getKey() + "=" + property.getValue())); + } + } catch (NumberFormatException e) { + throw new AccumuloException("Bad value for existing iterator setting: " + + property.getKey() + "=" + property.getValue()); + } + } + } + if (!optionConflicts.isEmpty()) { + throw new AccumuloException(new IllegalArgumentException( + "iterator options conflict for " + setting.getName() + ": " + optionConflicts)); + } + } + } + + public static boolean containsSameIterOpts(Map props, IteratorSetting setting, + String optStr) { + for (var opt : setting.getOptions().entrySet()) { + final String optKey = optStr + opt.getKey(); + if (!props.containsKey(optKey) || !props.get(optKey).equals(opt.getValue())) { + return false; + } + } + return true; + } + /** * Returns true if the property is an iterator property not including iterator option properties */ diff --git a/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java b/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java index 563c0715b8e..6e9fd1498df 100644 --- a/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java @@ -268,7 +268,8 @@ public void testFields() throws Exception { client1.securityOperations().changeUserAuthorizations(user, auths); client1.securityOperations().grantSystemPermission(user, SystemPermission.CREATE_TABLE); - client1.securityOperations().grantNamespacePermission(user, Namespace.DEFAULT.name(), NamespacePermission.ALTER_NAMESPACE); + client1.securityOperations().grantNamespacePermission(user, Namespace.DEFAULT.name(), + NamespacePermission.ALTER_NAMESPACE); try (AccumuloClient client2 = Accumulo.newClient().from(client1.properties()).as(user, user1.getToken()).build()) { diff --git a/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java index bf4248a87f6..559f44a2c95 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java @@ -56,6 +56,14 @@ *

* - {@link TableOperations#attachIterator(String, IteratorSetting, EnumSet)} *

+ * - {@link NamespaceOperations#attachIterator(String, IteratorSetting, EnumSet)} + *

+ *

+ * - {@link NamespaceOperations#setProperty(String, String, String)} + *

+ *

+ * - {@link NamespaceOperations#modifyProperties(String, Consumer)} + *

* All fail when conflicts arise from: *

* - Iterators attached directly to a table @@ -119,12 +127,21 @@ public static void shutdown() throws Exception { @Test public void testTableIterConflict() throws Exception { - final String[] tableNames = getUniqueNames(4); + final String[] tableNames = getUniqueNames(10); String table1 = tableNames[0]; String table2 = tableNames[1]; String table3 = tableNames[2]; String table4 = tableNames[3]; - for (String table : tableNames) { + String ns5 = tableNames[4]; + String table5 = ns5 + "." + tableNames[5]; + String ns6 = tableNames[6]; + String table6 = ns6 + "." + tableNames[7]; + String ns7 = tableNames[8]; + String table7 = ns7 + "." + tableNames[9]; + for (String ns : List.of(ns5, ns6, ns7)) { + nops.create(ns); + } + for (String table : List.of(table1, table2, table3, table4, table5, table6, table7)) { tops.create(table); } @@ -155,6 +172,23 @@ public void testTableIterConflict() throws Exception { testTableIterConflict(table4, AccumuloException.class, () -> tops.attachIterator(table4, iter1PrioConflict), () -> tops.attachIterator(table4, iter1NameConflict)); + + // testing NamespaceOperations.attachIterator + testTableIterConflict(table5, AccumuloException.class, + () -> nops.attachIterator(ns5, iter1PrioConflict), + () -> nops.attachIterator(ns5, iter1NameConflict)); + + // testing NamespaceOperations.setProperty + testTableIterConflict(table6, AccumuloException.class, + () -> nops.setProperty(ns6, iter1PrioConflictKey, iter1PrioConflictVal), + () -> nops.setProperty(ns6, iter1NameConflictKey, iter1NameConflictVal)); + + // testing NamespaceOperations.modifyProperties + testTableIterConflict(table7, AccumuloException.class, + () -> nops.modifyProperties(ns7, + props -> props.put(iter1PrioConflictKey, iter1PrioConflictVal)), + () -> nops.modifyProperties(ns7, + props -> props.put(iter1NameConflictKey, iter1NameConflictVal))); } private void testTableIterConflict(String table, Class exceptionClass, @@ -170,7 +204,7 @@ private void testTableIterConflict(String table, Class @Test public void testNamespaceIterConflict() throws Exception { - final String[] names = getUniqueNames(10); + final String[] names = getUniqueNames(16); String ns1 = names[0]; String table1 = ns1 + "." + names[1]; String ns2 = names[2]; @@ -181,11 +215,17 @@ public void testNamespaceIterConflict() throws Exception { String table4 = ns4 + "." + names[7]; String ns5 = names[8]; String table5 = ns5 + "." + names[9]; - for (String ns : List.of(ns1, ns2, ns3, ns4, ns5)) { + String ns6 = names[10]; + String table6 = ns5 + "." + names[11]; + String ns7 = names[12]; + String table7 = ns5 + "." + names[13]; + String ns8 = names[14]; + String table8 = ns5 + "." + names[15]; + for (String ns : List.of(ns1, ns2, ns3, ns4, ns5, ns6, ns7, ns8)) { nops.create(ns); } // don't create table4 - for (String table : List.of(table1, table2, table3, table5)) { + for (String table : List.of(table1, table2, table3, table5, table6, table7, table8)) { tops.create(table); } @@ -217,6 +257,23 @@ public void testNamespaceIterConflict() throws Exception { testNamespaceIterConflict(ns5, AccumuloException.class, () -> tops.attachIterator(table5, iter1PrioConflict), () -> tops.attachIterator(table5, iter1NameConflict)); + + // testing NamespaceOperations.attachIterator + testNamespaceIterConflict(ns6, AccumuloException.class, + () -> nops.attachIterator(ns6, iter1PrioConflict), + () -> nops.attachIterator(ns6, iter1NameConflict)); + + // testing NamespaceOperations.setProperty + testNamespaceIterConflict(ns6, AccumuloException.class, + () -> nops.setProperty(ns6, iter1PrioConflictKey, iter1PrioConflictVal), + () -> nops.setProperty(ns6, iter1NameConflictKey, iter1NameConflictVal)); + + // testing NamespaceOperations.modifyProperties + testNamespaceIterConflict(ns6, AccumuloException.class, + () -> nops.modifyProperties(ns6, + props -> props.put(iter1PrioConflictKey, iter1PrioConflictVal)), + () -> nops.modifyProperties(ns6, + props -> props.put(iter1NameConflictKey, iter1NameConflictVal))); } private void testNamespaceIterConflict(String ns, Class exceptionClass, @@ -233,30 +290,13 @@ private void testNamespaceIterConflict(String ns, Class @Test public void testDefaultIterConflict() throws Throwable { - final String[] tables = getUniqueNames(11); - String defaultsTable1 = tables[0]; - String noDefaultsTable1 = tables[1]; - String defaultsTable2 = tables[2]; - String noDefaultsTable2 = tables[3]; - String defaultsTable3 = tables[4]; - String noDefaultsTable3 = tables[5]; - String defaultsTable4 = tables[6]; - String noDefaultsTable4 = tables[7]; - String noDefaultsTable5 = tables[9]; - String defaultsTable5 = tables[8]; - String noDefaultsTable6 = tables[10]; - // don't create defaultsTable4 - for (String defaultsTable : List.of(defaultsTable1, defaultsTable2, defaultsTable3, - defaultsTable5)) { - tops.create(defaultsTable); - } - // don't create noDefaultsTable4 or noDefaultsTable5 - for (String noDefaultsTable : List.of(noDefaultsTable1, noDefaultsTable2, noDefaultsTable3, - noDefaultsTable6)) { - tops.create(noDefaultsTable, new NewTableConfiguration().withoutDefaults()); - } + final String[] tables = getUniqueNames(23); // testing Scanner.addScanIterator + String defaultsTable1 = tables[0]; + tops.create(defaultsTable1); + String noDefaultsTable1 = tables[1]; + tops.create(noDefaultsTable1, new NewTableConfiguration().withoutDefaults()); try (var defaultsScanner = client.createScanner(defaultsTable1); var noDefaultsScanner = client.createScanner(noDefaultsTable1)) { testDefaultIterConflict(IllegalArgumentException.class, @@ -267,6 +307,10 @@ public void testDefaultIterConflict() throws Throwable { } // testing TableOperations.setProperty + String defaultsTable2 = tables[2]; + tops.create(defaultsTable2); + String noDefaultsTable2 = tables[3]; + tops.create(noDefaultsTable2, new NewTableConfiguration().withoutDefaults()); testDefaultIterConflict(AccumuloException.class, () -> tops.setProperty(defaultsTable2, defaultIterPrioConflictKey, defaultIterPrioConflictVal), @@ -278,6 +322,10 @@ public void testDefaultIterConflict() throws Throwable { defaultIterNameConflictVal)); // testing TableOperations.modifyProperties + String defaultsTable3 = tables[4]; + tops.create(defaultsTable3); + String noDefaultsTable3 = tables[5]; + tops.create(noDefaultsTable3, new NewTableConfiguration().withoutDefaults()); testDefaultIterConflict(AccumuloException.class, () -> tops.modifyProperties(defaultsTable3, props -> props.put(defaultIterPrioConflictKey, defaultIterPrioConflictVal)), @@ -289,6 +337,9 @@ public void testDefaultIterConflict() throws Throwable { props -> props.put(defaultIterNameConflictKey, defaultIterNameConflictVal))); // testing NewTableConfiguration.attachIterator + String defaultsTable4 = tables[6]; + String noDefaultsTable4 = tables[7]; + String noDefaultsTable5 = tables[8]; testDefaultIterConflict(AccumuloException.class, () -> tops.create(defaultsTable4, new NewTableConfiguration().attachIterator(defaultIterPrioConflict)), @@ -300,11 +351,60 @@ public void testDefaultIterConflict() throws Throwable { new NewTableConfiguration().attachIterator(defaultIterNameConflict).withoutDefaults())); // testing TableOperations.attachIterator + String defaultsTable6 = tables[9]; + tops.create(defaultsTable6); + String noDefaultsTable6 = tables[10]; + tops.create(noDefaultsTable6, new NewTableConfiguration().withoutDefaults()); testDefaultIterConflict(AccumuloException.class, - () -> tops.attachIterator(defaultsTable5, defaultIterPrioConflict), - () -> tops.attachIterator(defaultsTable5, defaultIterNameConflict), + () -> tops.attachIterator(defaultsTable6, defaultIterPrioConflict), + () -> tops.attachIterator(defaultsTable6, defaultIterNameConflict), () -> tops.attachIterator(noDefaultsTable6, defaultIterPrioConflict), () -> tops.attachIterator(noDefaultsTable6, defaultIterNameConflict)); + + // testing NamespaceOperations.attachIterator + String ns7 = tables[11]; + nops.create(ns7); + String defaultsTable7 = ns7 + "." + tables[12]; + tops.create(defaultsTable7); + String ns8 = tables[13]; + nops.create(ns8); + String noDefaultsTable8 = ns8 + "." + tables[14]; + tops.create(noDefaultsTable8, new NewTableConfiguration().withoutDefaults()); + testDefaultIterConflict(AccumuloException.class, + () -> nops.attachIterator(ns7, defaultIterPrioConflict), + () -> nops.attachIterator(ns7, defaultIterNameConflict), + () -> nops.attachIterator(ns8, defaultIterPrioConflict), + () -> nops.attachIterator(ns8, defaultIterNameConflict)); + + // testing NamespaceOperations.setProperty + String ns9 = tables[15]; + nops.create(ns9); + String defaultsTable9 = ns9 + "." + tables[16]; + tops.create(defaultsTable9); + String ns10 = tables[17]; + nops.create(ns10); + String noDefaultsTable10 = ns10 + "." + tables[18]; + tops.create(noDefaultsTable10, new NewTableConfiguration().withoutDefaults()); + testDefaultIterConflict(AccumuloException.class, + () -> nops.setProperty(ns9, defaultIterPrioConflictKey, defaultIterPrioConflictVal), + () -> nops.setProperty(ns9, defaultIterNameConflictKey, defaultIterNameConflictVal), + () -> nops.setProperty(ns10, defaultIterPrioConflictKey, defaultIterPrioConflictVal), + () -> nops.setProperty(ns10, defaultIterNameConflictKey, defaultIterNameConflictVal)); + + // testing NamespaceOperations.modifyProperties + String ns11 = tables[19]; + nops.create(ns11); + String defaultsTable11 = ns11 + "." + tables[20]; + tops.create(defaultsTable11); + String ns12 = tables[21]; + nops.create(ns12); + String noDefaultsTable12 = ns12 + "." + tables[22]; + tops.create(noDefaultsTable12, new NewTableConfiguration().withoutDefaults()); + testDefaultIterConflict(AccumuloException.class, + () -> nops.setProperty(ns11, defaultIterPrioConflictKey, defaultIterPrioConflictVal), + () -> nops.setProperty(ns11, defaultIterNameConflictKey, defaultIterNameConflictVal), + () -> nops.setProperty(ns12, defaultIterPrioConflictKey, defaultIterPrioConflictVal), + () -> nops.setProperty(ns12, defaultIterNameConflictKey, defaultIterNameConflictVal)); } private void testDefaultIterConflict(Class exceptionClass, @@ -327,21 +427,12 @@ private void testDefaultIterConflict(Class exceptionCla @Test public void testSameIterNoConflict() throws Throwable { - final String[] names = getUniqueNames(7); - final String table1 = names[0]; - final String table2 = names[1]; - final String table3 = names[2]; - final String ns = names[3]; - final String table4 = ns + "." + names[4]; - final String table5 = names[5]; - final String table6 = names[6]; - // don't create table4 or table5 - for (String table : List.of(table1, table2, table3, table6)) { - tops.create(table); - tops.attachIterator(table, iter1); - } + final String[] names = getUniqueNames(13); // testing Scanner.addScanIterator + final String table1 = names[0]; + tops.create(table1); + tops.attachIterator(table1, iter1); try (var scanner = client.createScanner(table1)) { testSameIterNoConflict(() -> scanner.addScanIterator(iter1), () -> scanner.addScanIterator(defaultTableIter)); @@ -351,6 +442,9 @@ public void testSameIterNoConflict() throws Throwable { // note that this is not technically the exact same iterator since the default iterator has // options (which are separate properties), but this call has no effect on the // property map/iterators, so this call should not throw + final String table2 = names[1]; + tops.create(table2); + tops.attachIterator(table2, iter1); testSameIterNoConflict(() -> tops.setProperty(table2, iter1Key, iter1Val), () -> tops.setProperty(table2, defaultIterKey, defaultIterVal)); @@ -358,20 +452,56 @@ public void testSameIterNoConflict() throws Throwable { // note that this is not technically the exact same iterator since the default iterator has // options (which are separate properties), but this call has no effect on the // property map/iterators, so this call should not throw + final String table3 = names[2]; + tops.create(table3); + tops.attachIterator(table3, iter1); testSameIterNoConflict( () -> tops.modifyProperties(table3, props -> props.put(iter1Key, iter1Val)), () -> tops.modifyProperties(table3, props -> props.put(defaultIterKey, defaultIterVal))); // testing NewTableConfiguration.attachIterator - nops.create(ns); - nops.attachIterator(ns, iter1); + final String ns1 = names[3]; + final String table4 = ns1 + "." + names[4]; + final String table5 = names[5]; + nops.create(ns1); + nops.attachIterator(ns1, iter1); testSameIterNoConflict( () -> tops.create(table4, new NewTableConfiguration().attachIterator(iter1)), () -> tops.create(table5, new NewTableConfiguration().attachIterator(defaultTableIter))); // testing TableOperations.attachIterator + final String table6 = names[6]; + tops.create(table6); + tops.attachIterator(table6, iter1); testSameIterNoConflict(() -> tops.attachIterator(table6, iter1), () -> tops.attachIterator(table6, defaultTableIter)); + + // testing NamespaceOperations.attachIterator + final String ns2 = names[7]; + final String table7 = ns2 + "." + names[8]; + nops.create(ns2); + tops.create(table7); + tops.attachIterator(table7, iter1); + testSameIterNoConflict(() -> nops.attachIterator(ns2, iter1), + () -> nops.attachIterator(ns2, defaultTableIter)); + + // testing NamespaceOperations.setProperty + final String ns3 = names[9]; + final String table8 = ns3 + "." + names[10]; + nops.create(ns3); + tops.create(table8); + tops.attachIterator(table8, iter1); + testSameIterNoConflict(() -> nops.setProperty(ns3, iter1Key, iter1Val), + () -> nops.setProperty(ns3, defaultIterKey, defaultIterVal)); + + // testing NamespaceOperations.modifyProperties + final String ns4 = names[11]; + final String table9 = ns4 + "." + names[12]; + nops.create(ns4); + tops.create(table9); + tops.attachIterator(table9, iter1); + testSameIterNoConflict(() -> nops.setProperty(ns4, iter1Key, iter1Val), + () -> nops.setProperty(ns4, defaultIterKey, defaultIterVal)); } private void testSameIterNoConflict(Executable addIter1Executable, From 1e040090133179647fbf88a1f2e77f19a71dcb4b Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Wed, 7 Jan 2026 16:49:34 -0500 Subject: [PATCH 03/21] fix build --- .../apache/accumulo/test/functional/IteratorConflictsIT.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java index 559f44a2c95..e79d9981893 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java @@ -58,10 +58,8 @@ *

* - {@link NamespaceOperations#attachIterator(String, IteratorSetting, EnumSet)} *

- *

* - {@link NamespaceOperations#setProperty(String, String, String)} *

- *

* - {@link NamespaceOperations#modifyProperties(String, Consumer)} *

* All fail when conflicts arise from: @@ -74,7 +72,6 @@ * is specified *

* - Adding the exact iterator already present should not fail - *

*/ public class IteratorConflictsIT extends SharedMiniClusterBase { private static TableOperations tops; From cb2eccb02a46330efdc73a027c6806f22c58e781 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Tue, 13 Jan 2026 13:39:30 -0500 Subject: [PATCH 04/21] Prevents interator conflicts p3: - Moves the iterator conflict check for create table from client side to server side. - Checking if iterators added to scanner conflict with those already set on the table moved from client side to server side. - Adds iterator conflict checks to CloneConfiguration.Builder.setPropertiesToSet. This check is done server side. - Adds testing to IteratorConflictsIT for CloneConfiguration.Builder.setPropertiesToSet --- .../client/ClientSideIteratorScanner.java | 3 - .../accumulo/core/client/IsolatedScanner.java | 9 +- .../client/admin/NewTableConfiguration.java | 46 +-- .../client/mapred/AbstractInputFormat.java | 10 +- .../client/mapreduce/AbstractInputFormat.java | 9 +- .../core/client/rfile/RFileScanner.java | 1 - .../core/clientImpl/ClientContext.java | 40 +-- .../core/clientImpl/OfflineScanner.java | 6 +- .../accumulo/core/clientImpl/ScannerImpl.java | 6 +- .../core/clientImpl/ScannerOptions.java | 18 +- .../core/clientImpl/TableOperationsImpl.java | 14 - .../clientImpl/TabletServerBatchDeleter.java | 7 +- .../clientImpl/TabletServerBatchReader.java | 12 +- .../iteratorsImpl/IteratorConfigUtil.java | 33 +- .../metadata/MetadataLocationObtainer.java | 7 +- .../core/metadata/schema/TabletsMetadata.java | 3 +- .../core/clientImpl/ScannerImplTest.java | 16 +- .../core/clientImpl/ScannerOptionsTest.java | 8 +- .../TabletServerBatchReaderTest.java | 10 +- .../mapred/AccumuloRecordReader.java | 9 +- .../mapreduce/AccumuloRecordReader.java | 9 +- .../server/metadata/ServerAmpleImpl.java | 3 +- .../server/util/ManagerMetadataUtil.java | 6 +- .../server/util/MetadataTableUtil.java | 4 +- .../java/org/apache/accumulo/gc/GCRun.java | 3 +- .../accumulo/manager/FateServiceHandler.java | 68 ++++- .../manager/tableOps/bulkVer1/CopyFailed.java | 3 +- .../tserver/tablet/ScanDataSource.java | 22 ++ .../accumulo/test/ClientSideIteratorIT.java | 4 +- .../apache/accumulo/test/CloseScannerIT.java | 4 +- .../accumulo/test/ConditionalWriterIT.java | 8 +- .../apache/accumulo/test/IteratorEnvIT.java | 4 +- .../test/NewTableConfigurationIT.java | 12 +- .../org/apache/accumulo/test/SampleIT.java | 13 +- .../accumulo/test/ScanConsistencyIT.java | 5 +- .../accumulo/test/ScanFlushWithTimeIT.java | 4 +- .../test/functional/IteratorConflictsIT.java | 284 ++++++++++++------ .../test/functional/ManagerAssignmentIT.java | 3 +- .../test/functional/PerTableCryptoIT.java | 3 +- .../test/functional/ScanIteratorIT.java | 2 - .../test/functional/SplitRecoveryIT.java | 3 +- .../TabletStateChangeIteratorIT.java | 3 +- .../accumulo/test/shell/ShellServerIT.java | 1 - 43 files changed, 381 insertions(+), 357 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java index c45ed6fccb1..48fb306c86c 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java +++ b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java @@ -173,7 +173,6 @@ public SortedKeyValueIterator deepCopy(final IteratorEnvironment env) * @param scanner the source scanner */ public ClientSideIteratorScanner(final Scanner scanner) { - super((setting) -> {}); smi = new ScannerTranslatorImpl(scanner, scanner.getSamplerConfiguration()); this.range = scanner.getRange(); this.size = scanner.getBatchSize(); @@ -189,8 +188,6 @@ public ClientSideIteratorScanner(final Scanner scanner) { var scannerImpl = (ScannerImpl) scanner; this.context = () -> scannerImpl.getClientContext(); this.tableId = () -> scannerImpl.getTableId(); - setIteratorValidator( - scannerImpl.getClientContext().getScanIteratorValidator(scannerImpl.getTableId())); } else { // These may never be used, so only fail if an attempt is made to use them. this.context = () -> { diff --git a/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java b/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java index 54fbfda5893..5a7328a5f26 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java +++ b/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java @@ -24,7 +24,6 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.Map.Entry; -import java.util.function.Consumer; import org.apache.accumulo.core.clientImpl.IsolationException; import org.apache.accumulo.core.clientImpl.ScannerOptions; @@ -221,13 +220,11 @@ public void clear() { private long readaheadThreshold; private final RowBufferFactory bufferFactory; - public IsolatedScanner(Scanner scanner, Consumer iteratorValidator) { - this(scanner, new MemoryRowBufferFactory(), iteratorValidator); + public IsolatedScanner(Scanner scanner) { + this(scanner, new MemoryRowBufferFactory()); } - public IsolatedScanner(Scanner scanner, RowBufferFactory bufferFactory, - Consumer iteratorValidator) { - super(iteratorValidator); + public IsolatedScanner(Scanner scanner, RowBufferFactory bufferFactory) { this.scanner = scanner; this.range = scanner.getRange(); this.retryTimeout = scanner.getTimeout(MILLISECONDS); diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java index b6249df99d8..4acde47a041 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java +++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java @@ -49,6 +49,7 @@ import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError; import org.apache.hadoop.io.Text; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSortedSet; /** @@ -72,25 +73,8 @@ public class NewTableConfiguration { private Map summarizerProps = Collections.emptyMap(); private Map localityProps = Collections.emptyMap(); private final Map iteratorProps = new HashMap<>(); - private final Map inheritedIteratorProps = new HashMap<>(); private SortedSet splitProps = Collections.emptySortedSet(); - /** - * Configures the {@link NewTableConfiguration} with iterators inherited from the parent - * namespace. This is used internally in table creation - no need to call directly. - * - * @param props the parent namespace config - */ - public void configureInheritedIteratorProps(Map props) { - for (var prop : props.entrySet()) { - var propKey = prop.getKey(); - var propVal = prop.getValue(); - if (IteratorConfigUtil.isIterProp(propKey, propVal)) { - inheritedIteratorProps.put(propKey, propVal); - } - } - } - private void checkDisjoint(Map props, Map derivedProps, String kind) { checkArgument(Collections.disjoint(props.keySet(), derivedProps.keySet()), @@ -201,7 +185,7 @@ public NewTableConfiguration setProperties(Map props) { * * @return the current properties configured */ - public Map getProperties() throws AccumuloException { + public Map getProperties() { Map propertyMap = new HashMap<>(); propertyMap.putAll(summarizerProps); @@ -221,7 +205,7 @@ public Map getProperties() throws AccumuloException { try { TableOperationsHelper.checkIteratorConflicts(propertyMap, setting, scopes); } catch (AccumuloException e) { - throw new AccumuloException(String.format( + throw new IllegalArgumentException(String.format( "conflict with default table iterator: scopes: %s setting: %s", scopes, setting), e); } } @@ -233,33 +217,13 @@ public Map getProperties() throws AccumuloException { var dk = nonIterDefault.getKey(); var dv = nonIterDefault.getValue(); var valInPropMap = propertyMap.get(dk); - if (valInPropMap != null && !valInPropMap.equals(dv)) { - throw new AccumuloException(String.format( - "conflict for property %s : %s (default val) != %s (set val)", dk, dv, valInPropMap)); - } + Preconditions.checkArgument(valInPropMap == null || valInPropMap.equals(dv), String.format( + "conflict for property %s : %s (default val) != %s (set val)", dk, dv, valInPropMap)); } propertyMap.putAll(initTableProps); } - // check for conflicts between attached iterators and namespace-inherited iterators - for (var iterProp : iteratorProps.entrySet()) { - var iterPropKey = iterProp.getKey(); - var iterPropVal = iterProp.getValue(); - if (IteratorConfigUtil.isNonOptionIterProp(iterPropKey, iterPropVal)) { - var iterPropKeyParts = iterPropKey.split("\\."); - var iterPropValParts = iterPropVal.split(","); - String iterName = iterPropKeyParts[iterPropKeyParts.length - 1]; - IteratorScope iterScope = - IteratorScope.valueOf(iterPropKeyParts[iterPropKeyParts.length - 2]); - Map opts = IteratorConfigUtil.gatherOpts(iterPropKey, iteratorProps); - var is = new IteratorSetting(Integer.parseInt(iterPropValParts[0]), iterName, - iterPropValParts[1], opts); - TableOperationsHelper.checkIteratorConflicts(inheritedIteratorProps, is, - EnumSet.of(iterScope)); - } - } - return Collections.unmodifiableMap(propertyMap); } diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java index f84a84d032e..088a0655c4d 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java @@ -494,18 +494,16 @@ public void initialize(InputSplit inSplit, JobConf job) throws IOException { Scanner scanner; try { - var iteratorValidator = client.getScanIteratorValidator(table); if (isOffline) { - scanner = new OfflineScanner(client, TableId.of(baseSplit.getTableId()), authorizations, - iteratorValidator); + scanner = + new OfflineScanner(client, TableId.of(baseSplit.getTableId()), authorizations); } else { - scanner = new ScannerImpl(client, TableId.of(baseSplit.getTableId()), authorizations, - iteratorValidator); + scanner = new ScannerImpl(client, TableId.of(baseSplit.getTableId()), authorizations); } if (isIsolated) { log.info("Creating isolated scanner"); @SuppressWarnings("resource") - var wrapped = new IsolatedScanner(scanner, iteratorValidator); + var wrapped = new IsolatedScanner(scanner); scanner = wrapped; } if (usesLocalIterators) { diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java index 05db0c2e301..432471334b3 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java @@ -508,20 +508,17 @@ public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IO } try { - var iteratorValidator = client.getScanIteratorValidator(table); if (isOffline) { - scanner = new OfflineScanner(client, TableId.of(split.getTableId()), authorizations, - iteratorValidator); + scanner = new OfflineScanner(client, TableId.of(split.getTableId()), authorizations); } else { // Not using public API to create scanner so that we can use table ID // Table ID is used in case of renames during M/R job - scanner = new ScannerImpl(client, TableId.of(split.getTableId()), authorizations, - iteratorValidator); + scanner = new ScannerImpl(client, TableId.of(split.getTableId()), authorizations); } if (isIsolated) { log.info("Creating isolated scanner"); @SuppressWarnings("resource") - var wrapped = new IsolatedScanner(scanner, iteratorValidator); + var wrapped = new IsolatedScanner(scanner); scanner = wrapped; } if (usesLocalIterators) { diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java index e12d335f330..49e424b4563 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java +++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java @@ -135,7 +135,6 @@ static class Opts { } RFileScanner(Opts opts) { - super((setting) -> {}); if (!opts.auths.equals(Authorizations.EMPTY) && !opts.useSystemIterators) { throw new IllegalArgumentException( "Set authorizations and specified not to use system iterators"); diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java index 947d282a52f..e5b059a80f7 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java @@ -34,7 +34,6 @@ import java.nio.file.Path; import java.util.Collection; import java.util.Collections; -import java.util.EnumSet; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -45,7 +44,6 @@ import java.util.concurrent.Future; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; -import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -61,7 +59,6 @@ import org.apache.accumulo.core.client.ConditionalWriter; import org.apache.accumulo.core.client.ConditionalWriterConfig; import org.apache.accumulo.core.client.Durability; -import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.MultiTableBatchWriter; import org.apache.accumulo.core.client.NamespaceNotFoundException; import org.apache.accumulo.core.client.Scanner; @@ -86,7 +83,6 @@ import org.apache.accumulo.core.fate.zookeeper.ZooCacheFactory; import org.apache.accumulo.core.fate.zookeeper.ZooReader; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; -import org.apache.accumulo.core.iterators.IteratorUtil; import org.apache.accumulo.core.manager.state.tables.TableState; import org.apache.accumulo.core.metadata.RootTable; import org.apache.accumulo.core.metadata.schema.Ample; @@ -715,7 +711,7 @@ public BatchScanner createBatchScanner(String tableName, Authorizations authoriz ensureOpen(); checkArgument(authorizations != null, "authorizations is null"); return new TabletServerBatchReader(this, requireNotOffline(getTableId(tableName), tableName), - tableName, authorizations, numQueryThreads, getScanIteratorValidator(tableName)); + tableName, authorizations, numQueryThreads); } @Override @@ -741,8 +737,7 @@ public BatchDeleter createBatchDeleter(String tableName, Authorizations authoriz ensureOpen(); checkArgument(authorizations != null, "authorizations is null"); return new TabletServerBatchDeleter(this, requireNotOffline(getTableId(tableName), tableName), - tableName, authorizations, numQueryThreads, config.merge(getBatchWriterConfig()), - getScanIteratorValidator(tableName)); + tableName, authorizations, numQueryThreads, config.merge(getBatchWriterConfig())); } @Override @@ -801,8 +796,8 @@ public Scanner createScanner(String tableName, Authorizations authorizations) throws TableNotFoundException { ensureOpen(); checkArgument(authorizations != null, "authorizations is null"); - Scanner scanner = new ScannerImpl(this, requireNotOffline(getTableId(tableName), tableName), - authorizations, getScanIteratorValidator(tableName)); + Scanner scanner = + new ScannerImpl(this, requireNotOffline(getTableId(tableName), tableName), authorizations); Integer batchSize = ClientProperty.SCANNER_BATCH_SIZE.getInteger(getProperties()); if (batchSize != null) { scanner.setBatchSize(batchSize); @@ -817,33 +812,6 @@ public Scanner createScanner(String tableName) return createScanner(tableName, auths); } - private Consumer getScanIteratorValidator(Callable tableNameGetter) { - return (givenIter) -> { - try { - tableOperations().checkIteratorConflicts(tableNameGetter.call(), givenIter, - EnumSet.of(IteratorUtil.IteratorScope.scan)); - } catch (Exception e) { - throw new IllegalArgumentException(e); - } - }; - } - - /** - * @see #getScanIteratorValidator(String) - */ - public Consumer getScanIteratorValidator(TableId tableId) { - return getScanIteratorValidator(() -> getTableName(tableId)); - } - - /** - * Returns an IteratorSetting Consumer which checks if the given IteratorSetting conflicts with - * iterators on the given table. Invoking the Consumer will throw an - * {@link IllegalArgumentException} if there is a conflict - */ - public Consumer getScanIteratorValidator(String tableName) { - return getScanIteratorValidator(() -> tableName); - } - @Override public String whoami() { ensureOpen(); diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineScanner.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineScanner.java index d1ff801d2d9..3cef0428763 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineScanner.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineScanner.java @@ -22,10 +22,8 @@ import java.util.Iterator; import java.util.Map.Entry; -import java.util.function.Consumer; import org.apache.accumulo.core.Constants; -import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Range; @@ -43,9 +41,7 @@ public class OfflineScanner extends ScannerOptions implements Scanner { private final Authorizations authorizations; private final Text tableId; - public OfflineScanner(ClientContext context, TableId tableId, Authorizations authorizations, - Consumer iteratorValidator) { - super(iteratorValidator); + public OfflineScanner(ClientContext context, TableId tableId, Authorizations authorizations) { checkArgument(context != null, "context is null"); checkArgument(tableId != null, "tableId is null"); checkArgument(authorizations != null, "authorizations is null"); diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerImpl.java index 910796de518..b4dfaa6258d 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerImpl.java @@ -26,10 +26,8 @@ import java.util.LinkedHashMap; import java.util.Map; import java.util.Map.Entry; -import java.util.function.Consumer; import org.apache.accumulo.core.Constants; -import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Range; @@ -111,9 +109,7 @@ private synchronized void ensureOpen() { } } - public ScannerImpl(ClientContext context, TableId tableId, Authorizations authorizations, - Consumer iteratorValidator) { - super(iteratorValidator); + public ScannerImpl(ClientContext context, TableId tableId, Authorizations authorizations) { checkArgument(context != null, "context is null"); checkArgument(tableId != null, "tableId is null"); checkArgument(authorizations != null, "authorizations is null"); diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerOptions.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerOptions.java index 126b34ae3bc..2b044f6fd84 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerOptions.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerOptions.java @@ -34,7 +34,6 @@ import java.util.SortedSet; import java.util.TreeSet; import java.util.concurrent.TimeUnit; -import java.util.function.Consumer; import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.ScannerBase; @@ -68,30 +67,15 @@ public class ScannerOptions implements ScannerBase { private ConsistencyLevel consistencyLevel = ConsistencyLevel.IMMEDIATE; - private Consumer iteratorValidator; - - protected ScannerOptions(Consumer iteratorValidator) { - checkArgument(iteratorValidator != null, "iteratorValidator is null"); - this.iteratorValidator = iteratorValidator; - } + protected ScannerOptions() {} public ScannerOptions(ScannerOptions so) { setOptions(this, so); } - public void setIteratorValidator(Consumer iteratorValidator) { - checkArgument(iteratorValidator != null, "iteratorValidator is null"); - this.iteratorValidator = iteratorValidator; - } - @Override public synchronized void addScanIterator(IteratorSetting si) { checkArgument(si != null, "si is null"); - - // validate given iterator does not conflict with those already added to the table - iteratorValidator.accept(si); - - // validate given iterator does not conflict with those already added to the scanner if (serverSideIteratorList.isEmpty()) { serverSideIteratorList = new ArrayList<>(); } diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java index fb53f88e84f..5a2c77cac15 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java @@ -231,20 +231,6 @@ public void create(String tableName, NewTableConfiguration ntc) throws AccumuloException, AccumuloSecurityException, TableExistsException { NEW_TABLE_NAME.validate(tableName); checkArgument(ntc != null, "ntc is null"); - - final String[] nsAndTable; - try { - if ((nsAndTable = tableName.split("\\" + Namespace.SEPARATOR)).length == 2) { - ntc.configureInheritedIteratorProps( - context.namespaceOperations().getNamespaceProperties(nsAndTable[0])); - } else { - ntc.configureInheritedIteratorProps( - context.namespaceOperations().getNamespaceProperties(Namespace.DEFAULT.name())); - } - } catch (NamespaceNotFoundException e) { - throw new AccumuloException(e); - } - List args = new ArrayList<>(); args.add(ByteBuffer.wrap(tableName.getBytes(UTF_8))); args.add(ByteBuffer.wrap(ntc.getTimeType().name().getBytes(UTF_8))); diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchDeleter.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchDeleter.java index f37af60511b..02be6e3904f 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchDeleter.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchDeleter.java @@ -20,7 +20,6 @@ import java.util.Iterator; import java.util.Map.Entry; -import java.util.function.Consumer; import org.apache.accumulo.core.client.BatchDeleter; import org.apache.accumulo.core.client.BatchWriter; @@ -42,10 +41,8 @@ public class TabletServerBatchDeleter extends TabletServerBatchReader implements private final BatchWriterConfig bwConfig; public TabletServerBatchDeleter(ClientContext context, TableId tableId, String tableName, - Authorizations authorizations, int numQueryThreads, BatchWriterConfig bwConfig, - Consumer iteratorValidator) { - super(context, BatchDeleter.class, tableId, tableName, authorizations, numQueryThreads, - iteratorValidator); + Authorizations authorizations, int numQueryThreads, BatchWriterConfig bwConfig) { + super(context, BatchDeleter.class, tableId, tableName, authorizations, numQueryThreads); this.context = context; this.tableId = tableId; this.bwConfig = bwConfig; diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java index a28c82139b7..8b149da57fa 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java @@ -29,10 +29,8 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Consumer; import org.apache.accumulo.core.client.BatchScanner; -import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.TableId; @@ -59,16 +57,12 @@ public class TabletServerBatchReader extends ScannerOptions implements BatchScan private ArrayList ranges = null; public TabletServerBatchReader(ClientContext context, TableId tableId, String tableName, - Authorizations authorizations, int numQueryThreads, - Consumer iteratorValidator) { - this(context, BatchScanner.class, tableId, tableName, authorizations, numQueryThreads, - iteratorValidator); + Authorizations authorizations, int numQueryThreads) { + this(context, BatchScanner.class, tableId, tableName, authorizations, numQueryThreads); } protected TabletServerBatchReader(ClientContext context, Class scopeClass, TableId tableId, - String tableName, Authorizations authorizations, int numQueryThreads, - Consumer iteratorValidator) { - super(iteratorValidator); + String tableName, Authorizations authorizations, int numQueryThreads) { checkArgument(context != null, "context is null"); checkArgument(tableId != null, "tableId is null"); checkArgument(authorizations != null, "authorizations is null"); diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java index e4f347f355e..f9496dc515b 100644 --- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java @@ -372,6 +372,26 @@ public static void checkIteratorConflictsWithTablesInNamespace(TableOperations t } } + public static void checkScanIteratorConflicts(List iterInfos, + Map> iterOpts, IteratorSetting setting) throws AccumuloException { + Map props = new HashMap<>(); + for (var iterInfo : iterInfos) { + props.put( + String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX.getKey(), + IteratorScope.scan.name().toLowerCase(), iterInfo.getIterName()), + String.format("%s,%s", iterInfo.getPriority(), iterInfo.getClassName())); + var options = iterOpts.get(iterInfo.getIterName()); + if (options != null) { + for (var iterOpt : options.entrySet()) { + props.put(String.format("%s%s.%s.opt.%s", Property.TABLE_ITERATOR_PREFIX.getKey(), + IteratorScope.scan.name().toLowerCase(), iterInfo.getIterName(), iterOpt.getKey()), + iterOpt.getValue()); + } + } + } + checkIteratorConflicts(props, setting, EnumSet.of(IteratorScope.scan)); + } + public static void checkIteratorConflicts(Map props, IteratorSetting setting, EnumSet scopes) throws AccumuloException { for (IteratorScope scope : scopes) { @@ -444,11 +464,22 @@ public static boolean isIterProp(String propKey, String propVal) { return isNonOptionIterProp(propKey, propVal) || ITERATOR_PROP_OPT_REGEX.matches(propKey); } + /** + * Returns a new map of all the iterator props contained in the given map + */ + public static Map gatherIteratorProps(Map props) { + Map iterProps = new HashMap<>(); + props.entrySet().stream() + .filter(entry -> IteratorConfigUtil.isIterProp(entry.getKey(), entry.getValue())) + .forEach(entry -> iterProps.put(entry.getKey(), entry.getValue())); + return iterProps; + } + /** * returns a map of the options associated with the given iterator property key. Options of the * iterator are obtained by searching the given map */ - public static Map gatherOpts(String iterPropKey, Map map) { + public static Map gatherIterOpts(String iterPropKey, Map map) { Map opts = new HashMap<>(); for (var iteratorProp : map.entrySet()) { if (ITERATOR_PROP_OPT_REGEX.matches(iteratorProp.getKey()) diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java index 4dc8c82fa46..ce198a7e85c 100644 --- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java +++ b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java @@ -31,7 +31,6 @@ import java.util.SortedSet; import java.util.TreeMap; import java.util.TreeSet; -import java.util.function.Consumer; import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.AccumuloException; @@ -161,10 +160,6 @@ private void decodeRows(TreeMap encodedResults, TreeMap re } private static class SettableScannerOptions extends ScannerOptions { - public SettableScannerOptions(Consumer iteratorValidator) { - super(iteratorValidator); - } - public ScannerOptions setColumns(SortedSet locCols) { this.fetchedColumns = locCols; // see comment in lookupTablet about why iterator is used @@ -191,7 +186,7 @@ public List lookupTablets(ClientContext context, String tserver, }; ScannerOptions opts = null; - try (SettableScannerOptions unsetOpts = new SettableScannerOptions((setting) -> {})) { + try (SettableScannerOptions unsetOpts = new SettableScannerOptions()) { opts = unsetOpts.setColumns(locCols); } diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java index 597c55ff056..41189e8d663 100644 --- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java +++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java @@ -197,8 +197,7 @@ private TabletsMetadata buildNonRoot(AccumuloClient client) { String resolvedTable = table == null ? level.metaTable() : table; Scanner scanner = - new IsolatedScanner(client.createScanner(resolvedTable, Authorizations.EMPTY), - ((ClientContext) client).getScanIteratorValidator(resolvedTable)); + new IsolatedScanner(client.createScanner(resolvedTable, Authorizations.EMPTY)); scanner.setRange(range); boolean extentsPresent = extentsToFetch != null; diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerImplTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerImplTest.java index a7f774a302e..676f00c593c 100644 --- a/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerImplTest.java +++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerImplTest.java @@ -34,15 +34,11 @@ public class ScannerImplTest { @BeforeEach public void setup() { context = EasyMock.createMock(ClientContext.class); - EasyMock.expect(context.getScanIteratorValidator(EasyMock.anyString())) - .andReturn(iteratorSetting -> {}); - EasyMock.replay(context); } @Test public void testValidReadaheadValues() { - try (var s = new ScannerImpl(context, TableId.of("foo"), Authorizations.EMPTY, - context.getScanIteratorValidator("bar"))) { + try (var s = new ScannerImpl(context, TableId.of("foo"), Authorizations.EMPTY)) { s.setReadaheadThreshold(0); s.setReadaheadThreshold(10); s.setReadaheadThreshold(Long.MAX_VALUE); @@ -53,8 +49,7 @@ public void testValidReadaheadValues() { @Test public void testInValidReadaheadValues() { - try (var s = new ScannerImpl(context, TableId.of("foo"), Authorizations.EMPTY, - context.getScanIteratorValidator("bar"))) { + try (var s = new ScannerImpl(context, TableId.of("foo"), Authorizations.EMPTY)) { assertThrows(IllegalArgumentException.class, () -> s.setReadaheadThreshold(-1)); } } @@ -62,16 +57,15 @@ public void testInValidReadaheadValues() { @Test public void testGetAuthorizations() { Authorizations expected = new Authorizations("a,b"); - try (var s = new ScannerImpl(context, TableId.of("foo"), expected, - context.getScanIteratorValidator("bar"))) { + try (var s = new ScannerImpl(context, TableId.of("foo"), expected)) { assertEquals(expected, s.getAuthorizations()); } } @Test public void testNullAuthorizationsFails() { - assertThrows(IllegalArgumentException.class, () -> new ScannerImpl(context, TableId.of("foo"), - null, context.getScanIteratorValidator("bar"))); + assertThrows(IllegalArgumentException.class, + () -> new ScannerImpl(context, TableId.of("foo"), null)); } } diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerOptionsTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerOptionsTest.java index c3f56e3911d..5c484acc0df 100644 --- a/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerOptionsTest.java +++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerOptionsTest.java @@ -40,7 +40,7 @@ public class ScannerOptionsTest { */ @Test public void testAddRemoveIterator() { - try (ScannerOptions options = new ScannerOptions((setting) -> {})) { + try (ScannerOptions options = new ScannerOptions()) { options.addScanIterator(new IteratorSetting(1, "NAME", WholeRowIterator.class)); assertEquals(1, options.serverSideIteratorList.size()); options.removeScanIterator("NAME"); @@ -50,7 +50,7 @@ public void testAddRemoveIterator() { @Test public void testIteratorConflict() { - try (ScannerOptions options = new ScannerOptions((setting) -> {})) { + try (ScannerOptions options = new ScannerOptions()) { options.addScanIterator(new IteratorSetting(1, "NAME", DebugIterator.class)); assertThrows(IllegalArgumentException.class, () -> options.addScanIterator(new IteratorSetting(2, "NAME", DebugIterator.class))); @@ -61,7 +61,7 @@ public void testIteratorConflict() { @Test public void testFetchColumn() { - try (ScannerOptions options = new ScannerOptions((setting) -> {})) { + try (ScannerOptions options = new ScannerOptions()) { assertEquals(0, options.getFetchedColumns().size()); IteratorSetting.Column col = new IteratorSetting.Column(new Text("family"), new Text("qualifier")); @@ -76,7 +76,7 @@ public void testFetchColumn() { @Test public void testFetchNullColumn() { - try (ScannerOptions options = new ScannerOptions((setting) -> {})) { + try (ScannerOptions options = new ScannerOptions()) { // Require a non-null instance of Column assertThrows(IllegalArgumentException.class, () -> options.fetchColumn(null)); } diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderTest.java index 60558839bdc..cfc47322f5e 100644 --- a/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderTest.java +++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderTest.java @@ -37,23 +37,21 @@ public class TabletServerBatchReaderTest { public void setup() { context = EasyMock.createMock(ClientContext.class); EasyMock.expect(context.threadPools()).andReturn(ThreadPools.getServerThreadPools()); - EasyMock.expect(context.getScanIteratorValidator(EasyMock.anyString())) - .andReturn(iteratorSetting -> {}); EasyMock.replay(context); } @Test public void testGetAuthorizations() { Authorizations expected = new Authorizations("a,b"); - try (BatchScanner s = new TabletServerBatchReader(context, TableId.of("foo"), "fooName", - expected, 1, context.getScanIteratorValidator("fooName"))) { + try (BatchScanner s = + new TabletServerBatchReader(context, TableId.of("foo"), "fooName", expected, 1)) { assertEquals(expected, s.getAuthorizations()); } } @Test public void testNullAuthorizationsFails() { - assertThrows(IllegalArgumentException.class, () -> new TabletServerBatchReader(context, - TableId.of("foo"), "fooName", null, 1, context.getScanIteratorValidator("fooName"))); + assertThrows(IllegalArgumentException.class, + () -> new TabletServerBatchReader(context, TableId.of("foo"), "fooName", null, 1)); } } diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java index 4ec077d050b..2325c086a93 100644 --- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java +++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java @@ -184,22 +184,19 @@ public void initialize(InputSplit inSplit, JobConf job) throws IOException { } Scanner scanner; - var iteratorValidator = context.getScanIteratorValidator(table); try { if (isOffline) { - scanner = new OfflineScanner(context, TableId.of(baseSplit.getTableId()), authorizations, - iteratorValidator); + scanner = new OfflineScanner(context, TableId.of(baseSplit.getTableId()), authorizations); } else { - scanner = new ScannerImpl(context, TableId.of(baseSplit.getTableId()), authorizations, - iteratorValidator); + scanner = new ScannerImpl(context, TableId.of(baseSplit.getTableId()), authorizations); scanner.setConsistencyLevel(cl == null ? ConsistencyLevel.IMMEDIATE : cl); log.info("Using consistency level: {}", scanner.getConsistencyLevel()); } if (isIsolated) { log.info("Creating isolated scanner"); @SuppressWarnings("resource") - var wrapped = new IsolatedScanner(scanner, iteratorValidator); + var wrapped = new IsolatedScanner(scanner); scanner = wrapped; } if (usesLocalIterators) { diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java index 296670ad377..66445e62cdf 100644 --- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java +++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java @@ -203,22 +203,19 @@ public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IO } try { - var iteratorValidator = ((ClientContext) client).getScanIteratorValidator(table); if (isOffline) { - scanner = new OfflineScanner(context, TableId.of(split.getTableId()), authorizations, - iteratorValidator); + scanner = new OfflineScanner(context, TableId.of(split.getTableId()), authorizations); } else { // Not using public API to create scanner so that we can use table ID // Table ID is used in case of renames during M/R job - scanner = new ScannerImpl(context, TableId.of(split.getTableId()), authorizations, - iteratorValidator); + scanner = new ScannerImpl(context, TableId.of(split.getTableId()), authorizations); scanner.setConsistencyLevel(cl == null ? ConsistencyLevel.IMMEDIATE : cl); log.info("Using consistency level: {}", scanner.getConsistencyLevel()); } if (isIsolated) { log.info("Creating isolated scanner"); @SuppressWarnings("resource") - var wrapped = new IsolatedScanner(scanner, iteratorValidator); + var wrapped = new IsolatedScanner(scanner); scanner = wrapped; } if (usesLocalIterators) { diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java index a68301c1c54..22cb7a941fb 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java +++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java @@ -191,8 +191,7 @@ public void removeBulkLoadEntries(TableId tableId, long tid, Text firstSplit, Te Preconditions.checkArgument(DataLevel.of(tableId) == DataLevel.USER); try ( Scanner mscanner = - new IsolatedScanner(context.createScanner(MetadataTable.NAME, Authorizations.EMPTY), - context.getScanIteratorValidator(MetadataTable.NAME)); + new IsolatedScanner(context.createScanner(MetadataTable.NAME, Authorizations.EMPTY)); BatchWriter bw = context.createBatchWriter(MetadataTable.NAME)) { mscanner.setRange(new KeyExtent(tableId, lastSplit, firstSplit).toMetaRange()); mscanner.fetchColumnFamily(BulkFileColumnFamily.NAME); diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java index 3f0c9629d1a..c0b5e27fa65 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java @@ -132,8 +132,7 @@ private static KeyExtent fixSplit(ServerContext context, TableId tableId, Text m Key prevRowKey = new Key(new Text(TabletsSection.encodeRow(tableId, metadataPrevEndRow))); try (ScannerImpl scanner2 = - new ScannerImpl(context, Ample.DataLevel.of(tableId).metaTableId(), Authorizations.EMPTY, - context.getScanIteratorValidator(Ample.DataLevel.of(tableId).metaTable()))) { + new ScannerImpl(context, Ample.DataLevel.of(tableId).metaTableId(), Authorizations.EMPTY)) { scanner2.setRange(new Range(prevRowKey, prevRowKey.followingKey(PartialKey.ROW))); if (scanner2.iterator().hasNext()) { @@ -147,8 +146,7 @@ private static KeyExtent fixSplit(ServerContext context, TableId tableId, Text m Key rowKey = new Key(metadataEntry); try (Scanner scanner3 = new ScannerImpl(context, Ample.DataLevel.of(tableId).metaTableId(), - Authorizations.EMPTY, - context.getScanIteratorValidator(Ample.DataLevel.of(tableId).metaTable()))) { + Authorizations.EMPTY)) { scanner3.fetchColumnFamily(DataFileColumnFamily.NAME); scanner3.setRange(new Range(rowKey, rowKey.followingKey(PartialKey.ROW))); diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java index a07c16deb03..60b16e55738 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java @@ -311,9 +311,7 @@ public static void splitDatafiles(Text midRow, double splitRatio, public static void deleteTable(TableId tableId, boolean insertDeletes, ServerContext context, ServiceLock lock) throws AccumuloException { - try ( - Scanner ms = new ScannerImpl(context, MetadataTable.ID, Authorizations.EMPTY, - context.getScanIteratorValidator(MetadataTable.NAME)); + try (Scanner ms = new ScannerImpl(context, MetadataTable.ID, Authorizations.EMPTY); BatchWriter bw = new BatchWriterImpl(context, MetadataTable.ID, new BatchWriterConfig().setMaxMemory(1000000) .setMaxLatency(120000L, TimeUnit.MILLISECONDS).setMaxWriteThreads(2))) { diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java b/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java index 62e1b29cf3f..22275c9e119 100644 --- a/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java +++ b/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java @@ -174,8 +174,7 @@ public Stream getBlipPaths() throws TableNotFoundException { int blipPrefixLen = MetadataSchema.BlipSection.getRowPrefix().length(); var scanner = - new IsolatedScanner(context.createScanner(level.metaTable(), Authorizations.EMPTY), - context.getScanIteratorValidator(level.metaTable())); + new IsolatedScanner(context.createScanner(level.metaTable(), Authorizations.EMPTY)); scanner.setRange(MetadataSchema.BlipSection.getRange()); return scanner.stream() .map(entry -> entry.getKey().getRow().toString().substring(blipPrefixLen)) diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java index 52320225bab..6277a52c931 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java @@ -44,13 +44,16 @@ import java.util.Set; import java.util.stream.Collectors; +import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.NamespaceNotFoundException; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.admin.CompactionConfig; import org.apache.accumulo.core.client.admin.InitialTableState; import org.apache.accumulo.core.client.admin.TimeType; import org.apache.accumulo.core.clientImpl.Namespaces; +import org.apache.accumulo.core.clientImpl.TableOperationsHelper; import org.apache.accumulo.core.clientImpl.TableOperationsImpl; import org.apache.accumulo.core.clientImpl.UserCompactionUtils; import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode; @@ -63,6 +66,8 @@ import org.apache.accumulo.core.data.NamespaceId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus; +import org.apache.accumulo.core.iterators.IteratorUtil; +import org.apache.accumulo.core.iteratorsImpl.IteratorConfigUtil; import org.apache.accumulo.core.manager.state.tables.TableState; import org.apache.accumulo.core.manager.thrift.FateOperation; import org.apache.accumulo.core.manager.thrift.FateService; @@ -217,15 +222,11 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED); } + var namespaceIterProps = IteratorConfigUtil + .gatherIteratorProps(manager.getContext().getNamespaceConfiguration(namespaceId) + .getAllPropertiesWithPrefix(Property.TABLE_ITERATOR_PREFIX)); for (Map.Entry entry : options.entrySet()) { - if (!Property.isValidProperty(entry.getKey(), entry.getValue())) { - String errorMessage = "Property or value not valid "; - if (!Property.isValidTablePropertyKey(entry.getKey())) { - errorMessage = "Invalid Table Property "; - } - throw new ThriftPropertyException(entry.getKey(), entry.getValue(), - errorMessage + entry.getKey() + "=" + entry.getValue()); - } + validateTableProperty(entry.getKey(), entry.getValue(), options, namespaceIterProps); } goalMessage += "Create table " + tableName + " " + initialTableState + " with " + splitCount @@ -322,6 +323,18 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe Map propertiesToSet = new HashMap<>(); Set propertiesToExclude = new HashSet<>(); + // dest table will have the dest namespace props + src table props: need to check provided + // options to set for conflicts with this + var srcTableConfigIterProps = + new HashMap<>(manager.getContext().getTableConfiguration(srcTableId) + .getAllPropertiesWithPrefix(Property.TABLE_ITERATOR_PREFIX)); + var srcNamespaceConfigIterProps = + manager.getContext().getNamespaceConfiguration(srcNamespaceId) + .getAllPropertiesWithPrefix(Property.TABLE_ITERATOR_PREFIX); + srcNamespaceConfigIterProps.forEach((k, v) -> srcTableConfigIterProps.remove(k)); + var iterProps = new HashMap<>(manager.getContext().getNamespaceConfiguration(namespaceId) + .getAllPropertiesWithPrefix(Property.TABLE_ITERATOR_PREFIX)); + iterProps.putAll(srcTableConfigIterProps); for (Entry entry : options.entrySet()) { if (entry.getKey().startsWith(TableOperationsImpl.PROPERTY_EXCLUDE_PREFIX)) { propertiesToExclude.add( @@ -329,14 +342,8 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe continue; } - if (!Property.isValidProperty(entry.getKey(), entry.getValue())) { - String errorMessage = "Property or value not valid "; - if (!Property.isValidTablePropertyKey(entry.getKey())) { - errorMessage = "Invalid Table Property "; - } - throw new ThriftPropertyException(entry.getKey(), entry.getValue(), - errorMessage + entry.getKey() + "=" + entry.getValue()); - } + validateTableProperty(entry.getKey(), entry.getValue(), options, + IteratorConfigUtil.gatherIteratorProps(iterProps)); propertiesToSet.put(entry.getKey(), entry.getValue()); } @@ -843,6 +850,35 @@ private void writeSplitsToFile(Path splitsPath, final List arguments } } + private void validateTableProperty(String propKey, String propVal, Map propMap, + Map config) throws ThriftPropertyException { + // validating property as valid table property + if (!Property.isValidProperty(propKey, propVal)) { + String errorMessage = "Property or value not valid "; + if (!Property.isValidTablePropertyKey(propKey)) { + errorMessage = "Invalid Table Property "; + } + throw new ThriftPropertyException(propKey, propVal, errorMessage + propKey + "=" + propVal); + } + + // validating property does not create an iterator conflict with those in the config + if (IteratorConfigUtil.isNonOptionIterProp(propKey, propVal)) { + var iterPropKeyParts = propKey.split("\\."); + var iterPropValParts = propVal.split(","); + String iterName = iterPropKeyParts[iterPropKeyParts.length - 1]; + IteratorUtil.IteratorScope iterScope = + IteratorUtil.IteratorScope.valueOf(iterPropKeyParts[iterPropKeyParts.length - 2]); + Map opts = IteratorConfigUtil.gatherIterOpts(propKey, propMap); + var is = new IteratorSetting(Integer.parseInt(iterPropValParts[0]), iterName, + iterPropValParts[1], opts); + try { + TableOperationsHelper.checkIteratorConflicts(config, is, EnumSet.of(iterScope)); + } catch (AccumuloException e) { + throw new ThriftPropertyException(propKey, propVal, e.getMessage()); + } + } + } + /** * Creates a temporary directory for the given FaTE operation (deleting any existing, to avoid * issues in case of server retry). diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java index 389b2a2c4d3..52d99c3724d 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java @@ -122,8 +122,7 @@ public Repo call(long tid, Manager manager) throws Exception { // determine which failed files were loaded ServerContext client = manager.getContext(); try (Scanner mscanner = - new IsolatedScanner(client.createScanner(MetadataTable.NAME, Authorizations.EMPTY), - client.getScanIteratorValidator(MetadataTable.NAME))) { + new IsolatedScanner(client.createScanner(MetadataTable.NAME, Authorizations.EMPTY))) { mscanner.setRange(new KeyExtent(tableId, null, null).toMetaRange()); mscanner.fetchColumnFamily(BulkFileColumnFamily.NAME); diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java index 13c75157c63..2a06283f7e2 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java @@ -26,6 +26,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; +import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Value; @@ -243,6 +245,26 @@ private SortedKeyValueIterator createIterator() } else { // Scan time iterator options were set, so need to merge those with pre-parsed table // iterator options. + + // First ensure the set iterators do not conflict with the existing table iterators. + for (var scanParamIterInfo : scanParams.getSsiList()) { + IteratorSetting setting; + var scanParamIterOpts = scanParams.getSsio().get(scanParamIterInfo.getIterName()); + if (scanParamIterOpts != null) { + setting = new IteratorSetting(scanParamIterInfo.getPriority(), + scanParamIterInfo.getIterName(), scanParamIterInfo.getClassName(), + scanParamIterOpts); + } else { + setting = new IteratorSetting(scanParamIterInfo.getPriority(), + scanParamIterInfo.getIterName(), scanParamIterInfo.getClassName()); + } + try { + IteratorConfigUtil.checkScanIteratorConflicts(pic.getIterInfo(), pic.getOpts(), + setting); + } catch (AccumuloException e) { + throw new IllegalArgumentException(e); + } + } iterOpts = new HashMap<>(pic.getOpts().size() + scanParams.getSsio().size()); iterInfos = new ArrayList<>(pic.getIterInfo().size() + scanParams.getSsiList().size()); IteratorConfigUtil.mergeIteratorConfig(iterInfos, iterOpts, pic.getIterInfo(), diff --git a/test/src/main/java/org/apache/accumulo/test/ClientSideIteratorIT.java b/test/src/main/java/org/apache/accumulo/test/ClientSideIteratorIT.java index fc33e2e9696..824284ee981 100644 --- a/test/src/main/java/org/apache/accumulo/test/ClientSideIteratorIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ClientSideIteratorIT.java @@ -217,8 +217,8 @@ private void runPluginEnvTest(Set expected) throws Exception { // accumulo config from iterators also. client.tableOperations().offline(tableName, true); var context = (ClientContext) client; - try (OfflineScanner offlineScanner = new OfflineScanner(context, context.getTableId(tableName), - Authorizations.EMPTY, context.getScanIteratorValidator(tableName))) { + try (OfflineScanner offlineScanner = + new OfflineScanner(context, context.getTableId(tableName), Authorizations.EMPTY)) { initCalled.set(false); offlineScanner.addScanIterator(new IteratorSetting(100, "filter", TestPropFilter.class)); assertEquals(expected, offlineScanner.stream().map(e -> e.getKey().getRowData().toString()) diff --git a/test/src/main/java/org/apache/accumulo/test/CloseScannerIT.java b/test/src/main/java/org/apache/accumulo/test/CloseScannerIT.java index 0532fd25692..6a4c4f68121 100644 --- a/test/src/main/java/org/apache/accumulo/test/CloseScannerIT.java +++ b/test/src/main/java/org/apache/accumulo/test/CloseScannerIT.java @@ -28,7 +28,6 @@ import org.apache.accumulo.core.client.IsolatedScanner; import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.client.ScannerBase; -import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.security.Authorizations; @@ -127,8 +126,7 @@ private static Scanner createScanner(AccumuloClient client, String tableName, in throws Exception { Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY); if (i % 2 == 0) { - scanner = new IsolatedScanner(scanner, - ((ClientContext) client).getScanIteratorValidator(tableName)); + scanner = new IsolatedScanner(scanner); } return scanner; } diff --git a/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java b/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java index 6e9fd1498df..bccbc0ced25 100644 --- a/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java @@ -69,9 +69,7 @@ import org.apache.accumulo.core.client.TableOfflineException; import org.apache.accumulo.core.client.admin.NewTableConfiguration; import org.apache.accumulo.core.client.security.tokens.PasswordToken; -import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.clientImpl.ConditionalWriterImpl; -import org.apache.accumulo.core.clientImpl.Namespace; import org.apache.accumulo.core.conf.ClientProperty; import org.apache.accumulo.core.data.ArrayByteSequence; import org.apache.accumulo.core.data.ByteSequence; @@ -90,7 +88,6 @@ import org.apache.accumulo.core.iterators.user.VersioningIterator; import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.security.ColumnVisibility; -import org.apache.accumulo.core.security.NamespacePermission; import org.apache.accumulo.core.security.SystemPermission; import org.apache.accumulo.core.security.TablePermission; import org.apache.accumulo.core.util.FastFormat; @@ -268,8 +265,6 @@ public void testFields() throws Exception { client1.securityOperations().changeUserAuthorizations(user, auths); client1.securityOperations().grantSystemPermission(user, SystemPermission.CREATE_TABLE); - client1.securityOperations().grantNamespacePermission(user, Namespace.DEFAULT.name(), - NamespacePermission.ALTER_NAMESPACE); try (AccumuloClient client2 = Accumulo.newClient().from(client1.properties()).as(user, user1.getToken()).build()) { @@ -1200,8 +1195,7 @@ public MutatorTask(String tableName, AccumuloClient client, ArrayList client.tableOperations() + var exception = assertThrows(IllegalArgumentException.class, () -> client.tableOperations() .create(table, new NewTableConfiguration().attachIterator(iterator1))); assertTrue(exception.getMessage().contains("conflict with default table iterator")); // add an iterator with same name as the default iterator var iterator2 = new IteratorSetting(10, "vers", "foo.bar"); - exception = assertThrows(AccumuloException.class, () -> client.tableOperations().create(table, - new NewTableConfiguration().attachIterator(iterator2))); + exception = assertThrows(IllegalArgumentException.class, () -> client.tableOperations() + .create(table, new NewTableConfiguration().attachIterator(iterator2))); assertTrue(exception.getMessage().contains("conflict with default table iterator")); // try to attach the exact default iterators, should not present a conflict as they are // equivalent to what would be added @@ -561,7 +561,7 @@ public void testConflictsWithDefaults() throws Exception { for (IteratorScope iterScope : IteratorScope.values()) { props.put(Property.TABLE_ITERATOR_PREFIX + iterScope.name() + ".foo", "20,foo.bar"); } - exception = assertThrows(AccumuloException.class, () -> client.tableOperations() + exception = assertThrows(IllegalArgumentException.class, () -> client.tableOperations() .create(table2, new NewTableConfiguration().setProperties(props))); assertTrue(exception.getMessage().contains("conflict with default table iterator")); props.clear(); @@ -569,7 +569,7 @@ public void testConflictsWithDefaults() throws Exception { for (IteratorScope iterScope : IteratorScope.values()) { props.put(Property.TABLE_ITERATOR_PREFIX + iterScope.name() + ".vers", "10,foo.bar"); } - exception = assertThrows(AccumuloException.class, () -> client.tableOperations() + exception = assertThrows(IllegalArgumentException.class, () -> client.tableOperations() .create(table2, new NewTableConfiguration().setProperties(props))); assertTrue(exception.getMessage().contains("conflict with default table iterator")); props.clear(); @@ -583,7 +583,7 @@ public void testConflictsWithDefaults() throws Exception { */ // setting a value different from default should throw props.put(Property.TABLE_CONSTRAINT_PREFIX + "1", "foo"); - exception = assertThrows(AccumuloException.class, () -> client.tableOperations() + exception = assertThrows(IllegalArgumentException.class, () -> client.tableOperations() .create(table3, new NewTableConfiguration().setProperties(props))); assertTrue(exception.getMessage() .contains("conflict for property " + Property.TABLE_CONSTRAINT_PREFIX + "1")); diff --git a/test/src/main/java/org/apache/accumulo/test/SampleIT.java b/test/src/main/java/org/apache/accumulo/test/SampleIT.java index bb30fd2605b..369444ac0a5 100644 --- a/test/src/main/java/org/apache/accumulo/test/SampleIT.java +++ b/test/src/main/java/org/apache/accumulo/test/SampleIT.java @@ -137,8 +137,7 @@ public void testBasic() throws Exception { Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY); Scanner isoScanner = - new IsolatedScanner(client.createScanner(tableName, Authorizations.EMPTY), - ((ClientContext) client).getScanIteratorValidator(tableName)); + new IsolatedScanner(client.createScanner(tableName, Authorizations.EMPTY)); Scanner csiScanner = new ClientSideIteratorScanner(client.createScanner(tableName, Authorizations.EMPTY)); scanner.setSamplerConfiguration(SC1); @@ -211,8 +210,8 @@ private Scanner newOfflineScanner(AccumuloClient client, String tableName, Strin client.tableOperations().clone(tableName, clone, false, em, es); client.tableOperations().offline(clone, true); TableId cloneID = TableId.of(client.tableOperations().tableIdMap().get(clone)); - OfflineScanner oScanner = new OfflineScanner((ClientContext) client, cloneID, - Authorizations.EMPTY, ((ClientContext) client).getScanIteratorValidator(clone)); + OfflineScanner oScanner = + new OfflineScanner((ClientContext) client, cloneID, Authorizations.EMPTY); if (sc != null) { oScanner.setSamplerConfiguration(sc); } @@ -316,8 +315,7 @@ public void testIterator() throws Exception { Scanner oScanner = null; try { scanner = client.createScanner(tableName, Authorizations.EMPTY); - isoScanner = new IsolatedScanner(client.createScanner(tableName, Authorizations.EMPTY), - ((ClientContext) client).getScanIteratorValidator(tableName)); + isoScanner = new IsolatedScanner(client.createScanner(tableName, Authorizations.EMPTY)); csiScanner = new ClientSideIteratorScanner(client.createScanner(tableName, Authorizations.EMPTY)); bScanner = client.createBatchScanner(tableName, Authorizations.EMPTY, 2); @@ -416,8 +414,7 @@ public void testSampleNotPresent() throws Exception { } Scanner scanner = client.createScanner(tableName); - Scanner isoScanner = new IsolatedScanner(client.createScanner(tableName), - ((ClientContext) client).getScanIteratorValidator(tableName)); + Scanner isoScanner = new IsolatedScanner(client.createScanner(tableName)); isoScanner.setBatchSize(10); Scanner csiScanner = new ClientSideIteratorScanner(client.createScanner(tableName)); try (BatchScanner bScanner = client.createBatchScanner(tableName)) { diff --git a/test/src/main/java/org/apache/accumulo/test/ScanConsistencyIT.java b/test/src/main/java/org/apache/accumulo/test/ScanConsistencyIT.java index 9f1f90a2a30..0c03de99b02 100644 --- a/test/src/main/java/org/apache/accumulo/test/ScanConsistencyIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ScanConsistencyIT.java @@ -50,7 +50,6 @@ import org.apache.accumulo.core.client.admin.CompactionConfig; import org.apache.accumulo.core.client.rfile.RFile; import org.apache.accumulo.core.client.rfile.RFileWriter; -import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Range; @@ -354,9 +353,7 @@ private static ScanStats batchScanData(TestContext tctx, Range range) throws Exc private static ScanStats scanData(TestContext tctx, Random random, Range range, boolean scanIsolated) throws Exception { try (ExpectedScanData expectedScanData = tctx.dataTracker.beginScan(); - Scanner scanner = scanIsolated - ? new IsolatedScanner(tctx.client.createScanner(tctx.table), - ((ClientContext) tctx.client).getScanIteratorValidator(tctx.table)) + Scanner scanner = scanIsolated ? new IsolatedScanner(tctx.client.createScanner(tctx.table)) : tctx.client.createScanner(tctx.table)) { Set expected = expectedScanData.getExpectedData(range).collect(Collectors.toSet()); diff --git a/test/src/main/java/org/apache/accumulo/test/ScanFlushWithTimeIT.java b/test/src/main/java/org/apache/accumulo/test/ScanFlushWithTimeIT.java index 16c9f96c9bb..7c88ee40b34 100644 --- a/test/src/main/java/org/apache/accumulo/test/ScanFlushWithTimeIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ScanFlushWithTimeIT.java @@ -34,7 +34,6 @@ import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.client.ScannerBase; -import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.security.Authorizations; @@ -87,8 +86,7 @@ public void test() throws Exception { testScanner(s, 1200); log.info("IsolatedScanner"); - IsolatedScanner is = - new IsolatedScanner(s, ((ClientContext) c).getScanIteratorValidator(tableName)); + IsolatedScanner is = new IsolatedScanner(s); is.setReadaheadThreshold(1); // buffers an entire row testScanner(is, 2200); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java index e79d9981893..170a5872e16 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java @@ -19,10 +19,10 @@ package org.apache.accumulo.test.functional; import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.EnumSet; import java.util.List; +import java.util.Map; import java.util.function.Consumer; import org.apache.accumulo.core.client.Accumulo; @@ -30,6 +30,7 @@ import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.Scanner; +import org.apache.accumulo.core.client.admin.CloneConfiguration; import org.apache.accumulo.core.client.admin.NamespaceOperations; import org.apache.accumulo.core.client.admin.NewTableConfiguration; import org.apache.accumulo.core.client.admin.TableOperations; @@ -62,6 +63,8 @@ *

* - {@link NamespaceOperations#modifyProperties(String, Consumer)} *

+ * - {@link CloneConfiguration.Builder#setPropertiesToSet(Map)} + *

* All fail when conflicts arise from: *

* - Iterators attached directly to a table @@ -77,32 +80,35 @@ public class IteratorConflictsIT extends SharedMiniClusterBase { private static TableOperations tops; private static NamespaceOperations nops; private static AccumuloClient client; - private static final IteratorSetting iter1 = new IteratorSetting(99, "iter1name", "foo"); + // doesn't matter what iterator is used here + private static final String iterClass = SlowIterator.class.getName(); + private static final IteratorSetting iter1 = new IteratorSetting(99, "iter1name", iterClass); private static final String iter1Key = Property.TABLE_ITERATOR_PREFIX + IteratorUtil.IteratorScope.scan.name().toLowerCase() + "." + iter1.getName(); - private static final String iter1Val = "99,foo"; + private static final String iter1Val = "99," + iterClass; private static final IteratorSetting iter1PrioConflict = - new IteratorSetting(99, "othername", "foo"); + new IteratorSetting(99, "othername", iterClass); private static final IteratorSetting iter1NameConflict = - new IteratorSetting(101, iter1.getName(), "foo"); + new IteratorSetting(101, iter1.getName(), iterClass); private static final String iter1PrioConflictKey = Property.TABLE_ITERATOR_PREFIX + IteratorUtil.IteratorScope.scan.name().toLowerCase() + ".othername"; - private static final String iter1PrioConflictVal = "99,foo"; + private static final String iter1PrioConflictVal = "99," + iterClass; private static final String iter1NameConflictKey = Property.TABLE_ITERATOR_PREFIX + IteratorUtil.IteratorScope.scan.name().toLowerCase() + "." + iter1.getName(); - private static final String iter1NameConflictVal = "101,foo"; + private static final String iter1NameConflictVal = "101," + iterClass; private static final IteratorSetting defaultIterPrioConflict = - new IteratorSetting(20, "bar", "foo"); + new IteratorSetting(20, "bar", iterClass); private static final IteratorSetting defaultIterNameConflict = - new IteratorSetting(101, "vers", "foo"); + new IteratorSetting(101, "vers", iterClass); private static final IteratorSetting defaultTableIter = IteratorConfigUtil.getInitialTableIteratorSettings().keySet().iterator().next(); private static final String defaultIterPrioConflictKey = Property.TABLE_ITERATOR_PREFIX + IteratorUtil.IteratorScope.scan.name().toLowerCase() + ".foo"; - private static final String defaultIterPrioConflictVal = defaultTableIter.getPriority() + ",bar"; + private static final String defaultIterPrioConflictVal = + defaultTableIter.getPriority() + "," + iterClass; private static final String defaultIterNameConflictKey = Property.TABLE_ITERATOR_PREFIX + IteratorUtil.IteratorScope.scan.name().toLowerCase() + "." + defaultTableIter.getName(); - private static final String defaultIterNameConflictVal = "99,bar"; + private static final String defaultIterNameConflictVal = "99," + iterClass; private static final String defaultIterKey = Property.TABLE_ITERATOR_PREFIX.getKey() + IteratorUtil.IteratorScope.scan.name().toLowerCase() + "." + defaultTableIter.getName(); private static final String defaultIterVal = @@ -124,7 +130,7 @@ public static void shutdown() throws Exception { @Test public void testTableIterConflict() throws Exception { - final String[] tableNames = getUniqueNames(10); + final String[] tableNames = getUniqueNames(12); String table1 = tableNames[0]; String table2 = tableNames[1]; String table3 = tableNames[2]; @@ -135,18 +141,23 @@ public void testTableIterConflict() throws Exception { String table6 = ns6 + "." + tableNames[7]; String ns7 = tableNames[8]; String table7 = ns7 + "." + tableNames[9]; + String table8 = tableNames[10]; for (String ns : List.of(ns5, ns6, ns7)) { nops.create(ns); } - for (String table : List.of(table1, table2, table3, table4, table5, table6, table7)) { + for (String table : List.of(table1, table2, table3, table4, table5, table6, table7, table8)) { tops.create(table); } // testing Scanner.addScanIterator - try (var scanner = client.createScanner(table1)) { - testTableIterConflict(table1, IllegalArgumentException.class, - () -> scanner.addScanIterator(iter1PrioConflict), - () -> scanner.addScanIterator(iter1NameConflict)); + try (var scanner1 = client.createScanner(table1); var scanner2 = client.createScanner(table1)) { + testTableIterConflict(table1, RuntimeException.class, () -> { + scanner1.addScanIterator(iter1PrioConflict); + scanner1.iterator().hasNext(); + }, () -> { + scanner2.addScanIterator(iter1NameConflict); + scanner2.iterator().hasNext(); + }); } // testing TableOperations.setProperty @@ -186,59 +197,57 @@ public void testTableIterConflict() throws Exception { props -> props.put(iter1PrioConflictKey, iter1PrioConflictVal)), () -> nops.modifyProperties(ns7, props -> props.put(iter1NameConflictKey, iter1NameConflictVal))); + + // testing CloneConfiguration.Builder.setPropertiesToSet + String table9 = tableNames[11]; + testTableIterConflict(table8, AccumuloException.class, + () -> tops.clone(table8, table9, + CloneConfiguration.builder() + .setPropertiesToSet(Map.of(iter1PrioConflictKey, iter1PrioConflictVal)).build()), + () -> tops.clone(table8, table9, CloneConfiguration.builder() + .setPropertiesToSet(Map.of(iter1NameConflictKey, iter1NameConflictVal)).build())); } private void testTableIterConflict(String table, Class exceptionClass, Executable iterPrioConflictExec, Executable iterNameConflictExec) throws Exception { tops.attachIterator(table, iter1); - var e = assertThrows(exceptionClass, iterPrioConflictExec); - assertTrue(e.getMessage().contains("iterator priority conflict") - && e.getMessage().contains(iter1.getName())); - e = assertThrows(exceptionClass, iterNameConflictExec); - assertTrue(e.getMessage().contains("iterator name conflict") - && e.getMessage().contains(iter1.getName())); + assertThrows(exceptionClass, iterPrioConflictExec); + assertThrows(exceptionClass, iterNameConflictExec); } @Test public void testNamespaceIterConflict() throws Exception { - final String[] names = getUniqueNames(16); - String ns1 = names[0]; - String table1 = ns1 + "." + names[1]; - String ns2 = names[2]; - String table2 = ns2 + "." + names[3]; - String ns3 = names[4]; - String table3 = ns3 + "." + names[5]; - String ns4 = names[6]; - String table4 = ns4 + "." + names[7]; - String ns5 = names[8]; - String table5 = ns5 + "." + names[9]; - String ns6 = names[10]; - String table6 = ns5 + "." + names[11]; - String ns7 = names[12]; - String table7 = ns5 + "." + names[13]; - String ns8 = names[14]; - String table8 = ns5 + "." + names[15]; - for (String ns : List.of(ns1, ns2, ns3, ns4, ns5, ns6, ns7, ns8)) { - nops.create(ns); - } - // don't create table4 - for (String table : List.of(table1, table2, table3, table5, table6, table7, table8)) { - tops.create(table); - } + final String[] names = getUniqueNames(25); // testing Scanner.addScanIterator - try (var scanner = client.createScanner(table1)) { - testNamespaceIterConflict(ns1, IllegalArgumentException.class, - () -> scanner.addScanIterator(iter1PrioConflict), - () -> scanner.addScanIterator(iter1NameConflict)); + String ns1 = names[0]; + nops.create(ns1); + String table1 = ns1 + "." + names[1]; + tops.create(table1); + try (var scanner1 = client.createScanner(table1); var scanner2 = client.createScanner(table1)) { + testNamespaceIterConflict(ns1, RuntimeException.class, () -> { + scanner1.addScanIterator(iter1PrioConflict); + scanner1.iterator().hasNext(); + }, () -> { + scanner2.addScanIterator(iter1NameConflict); + scanner2.iterator().hasNext(); + }); } // testing TableOperations.setProperty + String ns2 = names[2]; + nops.create(ns2); + String table2 = ns2 + "." + names[3]; + tops.create(table2); testNamespaceIterConflict(ns2, AccumuloException.class, () -> tops.setProperty(table2, iter1PrioConflictKey, iter1PrioConflictVal), () -> tops.setProperty(table2, iter1NameConflictKey, iter1NameConflictVal)); // testing TableOperations.modifyProperties + String ns3 = names[4]; + nops.create(ns3); + String table3 = ns3 + "." + names[5]; + tops.create(table3); testNamespaceIterConflict(ns3, AccumuloException.class, () -> tops.modifyProperties(table3, props -> props.put(iter1PrioConflictKey, iter1PrioConflictVal)), @@ -246,61 +255,124 @@ public void testNamespaceIterConflict() throws Exception { props -> props.put(iter1NameConflictKey, iter1NameConflictVal))); // testing NewTableConfiguration.attachIterator + String ns4 = names[6]; + nops.create(ns4); + String table4 = ns4 + "." + names[7]; testNamespaceIterConflict(ns4, AccumuloException.class, () -> tops.create(table4, new NewTableConfiguration().attachIterator(iter1PrioConflict)), () -> tops.create(table4, new NewTableConfiguration().attachIterator(iter1NameConflict))); // testing TableOperations.attachIterator + String ns5 = names[8]; + nops.create(ns5); + String table5 = ns5 + "." + names[9]; + tops.create(table5); testNamespaceIterConflict(ns5, AccumuloException.class, () -> tops.attachIterator(table5, iter1PrioConflict), () -> tops.attachIterator(table5, iter1NameConflict)); // testing NamespaceOperations.attachIterator + String ns6 = names[10]; + nops.create(ns6); testNamespaceIterConflict(ns6, AccumuloException.class, () -> nops.attachIterator(ns6, iter1PrioConflict), () -> nops.attachIterator(ns6, iter1NameConflict)); // testing NamespaceOperations.setProperty - testNamespaceIterConflict(ns6, AccumuloException.class, - () -> nops.setProperty(ns6, iter1PrioConflictKey, iter1PrioConflictVal), - () -> nops.setProperty(ns6, iter1NameConflictKey, iter1NameConflictVal)); + String ns7 = names[11]; + nops.create(ns7); + testNamespaceIterConflict(ns7, AccumuloException.class, + () -> nops.setProperty(ns7, iter1PrioConflictKey, iter1PrioConflictVal), + () -> nops.setProperty(ns7, iter1NameConflictKey, iter1NameConflictVal)); // testing NamespaceOperations.modifyProperties - testNamespaceIterConflict(ns6, AccumuloException.class, - () -> nops.modifyProperties(ns6, + String ns8 = names[12]; + nops.create(ns8); + testNamespaceIterConflict(ns8, AccumuloException.class, + () -> nops.modifyProperties(ns8, props -> props.put(iter1PrioConflictKey, iter1PrioConflictVal)), - () -> nops.modifyProperties(ns6, + () -> nops.modifyProperties(ns8, props -> props.put(iter1NameConflictKey, iter1NameConflictVal))); + + // testing CloneConfiguration.Builder.setPropertiesToSet + // testing same src and dst namespace: should conflict + String dstAndSrcNamespace1 = names[13]; + nops.create(dstAndSrcNamespace1); + String src1 = dstAndSrcNamespace1 + "." + names[14]; + tops.create(src1); + String dst1 = dstAndSrcNamespace1 + "." + names[15]; + testNamespaceIterConflict(dstAndSrcNamespace1, AccumuloException.class, + () -> tops.clone(src1, dst1, + CloneConfiguration.builder() + .setPropertiesToSet(Map.of(iter1PrioConflictKey, iter1PrioConflictVal)).build()), + () -> tops.clone(src1, dst1, CloneConfiguration.builder() + .setPropertiesToSet(Map.of(iter1NameConflictKey, iter1NameConflictVal)).build())); + // testing attached to src namespace, different dst namespace: should not conflict + String srcNamespace2 = names[16]; + nops.create(srcNamespace2); + nops.attachIterator(srcNamespace2, iter1); + String src2 = srcNamespace2 + "." + names[17]; + tops.create(src2); + String dstNamespace2 = names[18]; + nops.create(dstNamespace2); + String dst2 = dstNamespace2 + "." + names[19]; + String dst3 = dstNamespace2 + "." + names[20]; + // should NOT throw + tops.clone(src2, dst2, CloneConfiguration.builder() + .setPropertiesToSet(Map.of(iter1PrioConflictKey, iter1PrioConflictVal)).build()); + // should NOT throw + tops.clone(src2, dst3, CloneConfiguration.builder() + .setPropertiesToSet(Map.of(iter1NameConflictKey, iter1NameConflictVal)).build()); + // testing attached to dst namespace, different src namespace: should conflict + String srcNamespace3 = names[21]; + nops.create(srcNamespace3); + String src3 = srcNamespace3 + "." + names[22]; + tops.create(src3); + String dstNamespace3 = names[23]; + nops.create(dstNamespace3); + String dst4 = dstNamespace3 + "." + names[24]; + testNamespaceIterConflict(dstNamespace3, AccumuloException.class, + () -> tops.clone(src3, dst4, + CloneConfiguration.builder() + .setPropertiesToSet(Map.of(iter1PrioConflictKey, iter1PrioConflictVal)).build()), + () -> tops.clone(src3, dst4, CloneConfiguration.builder() + .setPropertiesToSet(Map.of(iter1NameConflictKey, iter1NameConflictVal)).build())); } private void testNamespaceIterConflict(String ns, Class exceptionClass, Executable iterPrioConflictExec, Executable iterNameConflictExec) throws Exception { nops.attachIterator(ns, iter1); - var e = assertThrows(exceptionClass, iterPrioConflictExec); - assertTrue(e.getMessage().contains("iterator priority conflict") - && e.getMessage().contains(iter1.getName())); - e = assertThrows(exceptionClass, iterNameConflictExec); - assertTrue(e.getMessage().contains("iterator name conflict") - && e.getMessage().contains(iter1.getName())); + assertThrows(exceptionClass, iterPrioConflictExec); + assertThrows(exceptionClass, iterNameConflictExec); } @Test public void testDefaultIterConflict() throws Throwable { - final String[] tables = getUniqueNames(23); + final String[] tables = getUniqueNames(27); // testing Scanner.addScanIterator String defaultsTable1 = tables[0]; tops.create(defaultsTable1); String noDefaultsTable1 = tables[1]; tops.create(noDefaultsTable1, new NewTableConfiguration().withoutDefaults()); - try (var defaultsScanner = client.createScanner(defaultsTable1); - var noDefaultsScanner = client.createScanner(noDefaultsTable1)) { - testDefaultIterConflict(IllegalArgumentException.class, - () -> defaultsScanner.addScanIterator(defaultIterPrioConflict), - () -> defaultsScanner.addScanIterator(defaultIterNameConflict), - () -> noDefaultsScanner.addScanIterator(defaultIterPrioConflict), - () -> noDefaultsScanner.addScanIterator(defaultIterNameConflict)); + try (var defaultsScanner1 = client.createScanner(defaultsTable1); + var noDefaultsScanner1 = client.createScanner(noDefaultsTable1); + var defaultsScanner2 = client.createScanner(defaultsTable1); + var noDefaultsScanner2 = client.createScanner(noDefaultsTable1)) { + testDefaultIterConflict(RuntimeException.class, () -> { + defaultsScanner1.addScanIterator(defaultIterPrioConflict); + defaultsScanner1.iterator().hasNext(); + }, () -> { + defaultsScanner2.addScanIterator(defaultIterNameConflict); + defaultsScanner2.iterator().hasNext(); + }, () -> { + noDefaultsScanner1.addScanIterator(defaultIterPrioConflict); + noDefaultsScanner1.iterator().hasNext(); + }, () -> { + noDefaultsScanner2.addScanIterator(defaultIterNameConflict); + noDefaultsScanner2.iterator().hasNext(); + }); } // testing TableOperations.setProperty @@ -337,7 +409,7 @@ public void testDefaultIterConflict() throws Throwable { String defaultsTable4 = tables[6]; String noDefaultsTable4 = tables[7]; String noDefaultsTable5 = tables[8]; - testDefaultIterConflict(AccumuloException.class, + testDefaultIterConflict(IllegalArgumentException.class, () -> tops.create(defaultsTable4, new NewTableConfiguration().attachIterator(defaultIterPrioConflict)), () -> tops.create(defaultsTable4, @@ -402,37 +474,65 @@ public void testDefaultIterConflict() throws Throwable { () -> nops.setProperty(ns11, defaultIterNameConflictKey, defaultIterNameConflictVal), () -> nops.setProperty(ns12, defaultIterPrioConflictKey, defaultIterPrioConflictVal), () -> nops.setProperty(ns12, defaultIterNameConflictKey, defaultIterNameConflictVal)); + + // testing CloneConfiguration.Builder.setPropertiesToSet + String dst1 = tables[23]; + String dst2 = tables[24]; + String defaultsTable12 = tables[25]; + tops.create(defaultsTable12); + String noDefaultsTable13 = tables[26]; + tops.create(noDefaultsTable13, new NewTableConfiguration().withoutDefaults()); + testDefaultIterConflict(AccumuloException.class, + () -> tops + .clone(defaultsTable12, dst1, + CloneConfiguration.builder() + .setPropertiesToSet( + Map.of(defaultIterPrioConflictKey, defaultIterPrioConflictVal)) + .build()), + () -> tops + .clone(defaultsTable12, dst1, + CloneConfiguration.builder() + .setPropertiesToSet( + Map.of(defaultIterNameConflictKey, defaultIterNameConflictVal)) + .build()), + () -> tops + .clone(noDefaultsTable13, dst1, + CloneConfiguration.builder() + .setPropertiesToSet( + Map.of(defaultIterPrioConflictKey, defaultIterPrioConflictVal)) + .build()), + () -> tops.clone(noDefaultsTable13, dst2, + CloneConfiguration.builder() + .setPropertiesToSet(Map.of(defaultIterNameConflictKey, defaultIterNameConflictVal)) + .build())); } private void testDefaultIterConflict(Class exceptionClass, Executable defaultsTableOp1, Executable defaultsTableOp2, Executable noDefaultsTableOp1, Executable noDefaultsTableOp2) throws Throwable { - var e = assertThrows(exceptionClass, defaultsTableOp1); - // exception message different depending on operation, just checking essential info - assertTrue( - e.getMessage().contains("VersioningIterator") && e.getMessage().contains("conflict")); - - e = assertThrows(exceptionClass, defaultsTableOp2); - // exception message different depending on operation, just checking essential info - assertTrue( - e.getMessage().contains("VersioningIterator") && e.getMessage().contains("conflict")); + assertThrows(exceptionClass, defaultsTableOp1); + assertThrows(exceptionClass, defaultsTableOp2); noDefaultsTableOp1.execute(); // should NOT fail - noDefaultsTableOp2.execute(); // should NOT fail } @Test public void testSameIterNoConflict() throws Throwable { - final String[] names = getUniqueNames(13); + final String[] names = getUniqueNames(16); // testing Scanner.addScanIterator final String table1 = names[0]; tops.create(table1); tops.attachIterator(table1, iter1); - try (var scanner = client.createScanner(table1)) { - testSameIterNoConflict(() -> scanner.addScanIterator(iter1), - () -> scanner.addScanIterator(defaultTableIter)); + try (var scanner1 = client.createScanner(table1); var scanner2 = client.createScanner(table1)) { + testSameIterNoConflict(() -> { + scanner1.addScanIterator(iter1); + scanner1.iterator().hasNext(); + }, () -> { + scanner2.addScanIterator(defaultTableIter); + scanner2.iterator().hasNext(); + }); } // testing TableOperations.setProperty @@ -499,6 +599,18 @@ public void testSameIterNoConflict() throws Throwable { tops.attachIterator(table9, iter1); testSameIterNoConflict(() -> nops.setProperty(ns4, iter1Key, iter1Val), () -> nops.setProperty(ns4, defaultIterKey, defaultIterVal)); + + // testing CloneConfiguration.Builder.setPropertiesToSet + final String src = names[13]; + final String dst1 = names[14]; + final String dst2 = names[15]; + tops.create(src); + tops.attachIterator(src, iter1); + testSameIterNoConflict( + () -> tops.clone(src, dst1, + CloneConfiguration.builder().setPropertiesToSet(Map.of(iter1Key, iter1Val)).build()), + () -> tops.clone(src, dst2, CloneConfiguration.builder() + .setPropertiesToSet(Map.of(defaultIterKey, defaultIterVal)).build())); } private void testSameIterNoConflict(Executable addIter1Executable, diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java index b1ae0a2fad2..50827f3dacf 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java @@ -142,8 +142,7 @@ public void testShutdownOnlyTServerWithUserTable() throws Exception { Runnable task = () -> { while (true) { - try (var scanner = new IsolatedScanner(client.createScanner(tableName), - ((ClientContext) client).getScanIteratorValidator(tableName))) { + try (var scanner = new IsolatedScanner(client.createScanner(tableName))) { // TODO maybe do not close scanner? The following limit was placed on the stream to // avoid reading all the data possibly leaving a scan session active on the tserver AtomicInteger count = new AtomicInteger(0); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/PerTableCryptoIT.java b/test/src/main/java/org/apache/accumulo/test/functional/PerTableCryptoIT.java index 368cebc3312..53306cf33d0 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/PerTableCryptoIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/PerTableCryptoIT.java @@ -163,8 +163,7 @@ public void testOfflineIterator() throws Exception { TableId tableId = TableId.of(c.tableOperations().tableIdMap().get(tableName)); c.tableOperations().offline(tableName, true); - try (var oScanner = new OfflineScanner((ClientContext) c, tableId, Authorizations.EMPTY, - ((ClientContext) c).getScanIteratorValidator(tableName))) { + try (var oScanner = new OfflineScanner((ClientContext) c, tableId, Authorizations.EMPTY)) { long count = oScanner.stream().count(); assertEquals(count, 100_000); } diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ScanIteratorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ScanIteratorIT.java index 647c4dfca5a..53fb54dc98c 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/ScanIteratorIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/ScanIteratorIT.java @@ -93,8 +93,6 @@ public void setup() throws Exception { accumuloClient.securityOperations().grantTablePermission(user, tableName, TablePermission.READ); accumuloClient.securityOperations().grantTablePermission(user, tableName, TablePermission.WRITE); - accumuloClient.securityOperations().grantTablePermission(user, tableName, - TablePermission.ALTER_TABLE); accumuloClient.securityOperations().changeUserAuthorizations(user, AuthsIterator.AUTHS); } diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java index 50e4ada2d1c..7b5b59accc4 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java @@ -254,8 +254,7 @@ private void splitPartiallyAndRecover(ServerContext context, KeyExtent extent, K private void ensureTabletHasNoUnexpectedMetadataEntries(ServerContext context, KeyExtent extent, SortedMap expectedMapFiles) throws Exception { - try (Scanner scanner = new ScannerImpl(context, MetadataTable.ID, Authorizations.EMPTY, - context.getScanIteratorValidator(MetadataTable.NAME))) { + try (Scanner scanner = new ScannerImpl(context, MetadataTable.ID, Authorizations.EMPTY)) { scanner.setRange(extent.toMetaRange()); HashSet expectedColumns = new HashSet<>(); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java index cd4e07fb2e2..1982707cdd7 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java @@ -278,8 +278,7 @@ private void copyTable(AccumuloClient client, String source, String copy) List mutations = new ArrayList<>(); try (Scanner scanner = client.createScanner(source, Authorizations.EMPTY)) { - RowIterator rows = new RowIterator( - new IsolatedScanner(scanner, ((ClientContext) client).getScanIteratorValidator(source))); + RowIterator rows = new RowIterator(new IsolatedScanner(scanner)); while (rows.hasNext()) { Iterator> row = rows.next(); diff --git a/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java index a7d0713f730..00e7cfdc033 100644 --- a/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java +++ b/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java @@ -357,7 +357,6 @@ public void user() throws Exception { String perms = ts.exec("userpermissions -u xyzzy", true); assertTrue(perms.contains("Table permissions (" + MetadataTable.NAME + "): Table.READ")); ts.exec("grant -u xyzzy -s System.CREATE_TABLE", true); - ts.exec("grant -u xyzzy -ns \"\" Namespace.ALTER_NAMESPACE", true); perms = ts.exec("userpermissions -u xyzzy", true); assertTrue(perms.contains("")); ts.exec("grant -u " + getPrincipal() + " -t " + MetadataTable.NAME + " Table.WRITE", true); From d518c3c700665ff414b9707bcd43bd49e0e5e532 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Tue, 13 Jan 2026 14:03:59 -0500 Subject: [PATCH 05/21] minor fix and cleanup --- .../core/clientImpl/TableOperationsImpl.java | 1 + .../iteratorsImpl/IteratorConfigUtil.java | 20 +++++++++++++++---- .../admin/NewTableConfigurationTest.java | 5 ++--- .../manager/tableOps/bulkVer1/CopyFailed.java | 4 ++-- 4 files changed, 21 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java index 5a2c77cac15..f8b1642c62e 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java @@ -231,6 +231,7 @@ public void create(String tableName, NewTableConfiguration ntc) throws AccumuloException, AccumuloSecurityException, TableExistsException { NEW_TABLE_NAME.validate(tableName); checkArgument(ntc != null, "ntc is null"); + List args = new ArrayList<>(); args.add(ByteBuffer.wrap(tableName.getBytes(UTF_8))); args.add(ByteBuffer.wrap(ntc.getTimeType().name().getBytes(UTF_8))); diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java index f9496dc515b..a8e353a400d 100644 --- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java @@ -347,8 +347,14 @@ public static void checkIteratorConflicts(TableOperations tableOps, NamespaceOpe public static void checkIteratorConflictsWithTablesInNamespace(TableOperations tableOps, String namespace, IteratorSetting is, EnumSet scopes) throws AccumuloException { - var tablesInNamespace = tableOps.list().stream() - .filter(t -> t.startsWith(namespace + Namespace.SEPARATOR)).collect(Collectors.toSet()); + Set tablesInNamespace; + if (namespace.equals(Namespace.DEFAULT.name())) { + tablesInNamespace = + tableOps.list().stream().filter(t -> t.startsWith(namespace)).collect(Collectors.toSet()); + } else { + tablesInNamespace = tableOps.list().stream() + .filter(t -> t.startsWith(namespace + Namespace.SEPARATOR)).collect(Collectors.toSet()); + } try { for (var table : tablesInNamespace) { IteratorConfigUtil.checkIteratorConflicts(tableOps.getTableProperties(table), is, scopes); @@ -360,8 +366,14 @@ public static void checkIteratorConflictsWithTablesInNamespace(TableOperations t public static void checkIteratorConflictsWithTablesInNamespace(TableOperations tableOps, String namespace, String property, String value) throws AccumuloException { - var tablesInNamespace = tableOps.list().stream() - .filter(t -> t.startsWith(namespace + Namespace.SEPARATOR)).collect(Collectors.toSet()); + Set tablesInNamespace; + if (namespace.equals(Namespace.DEFAULT.name())) { + tablesInNamespace = + tableOps.list().stream().filter(t -> t.startsWith(namespace)).collect(Collectors.toSet()); + } else { + tablesInNamespace = tableOps.list().stream() + .filter(t -> t.startsWith(namespace + Namespace.SEPARATOR)).collect(Collectors.toSet()); + } try { for (var table : tablesInNamespace) { IteratorConfigUtil.checkIteratorConflicts(tableOps.getTableProperties(table), property, diff --git a/core/src/test/java/org/apache/accumulo/core/client/admin/NewTableConfigurationTest.java b/core/src/test/java/org/apache/accumulo/core/client/admin/NewTableConfigurationTest.java index d940a21d8e1..89930fb37ac 100644 --- a/core/src/test/java/org/apache/accumulo/core/client/admin/NewTableConfigurationTest.java +++ b/core/src/test/java/org/apache/accumulo/core/client/admin/NewTableConfigurationTest.java @@ -31,7 +31,6 @@ import java.util.SortedSet; import java.util.TreeSet; -import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.sample.SamplerConfiguration; import org.apache.accumulo.core.client.summary.Summarizer; @@ -125,7 +124,7 @@ public void populateOptions() { * Verify enableSampling returns */ @Test - public void testEnableSampling() throws AccumuloException { + public void testEnableSampling() { SamplerConfiguration sha1SamplerConfig = new SamplerConfiguration("com.mysampler"); sha1SamplerConfig.setOptions(options); NewTableConfiguration ntcSample2 = @@ -139,7 +138,7 @@ public void testEnableSampling() throws AccumuloException { * Verify enableSummarization returns SummarizerConfiguration with the expected class name(s). */ @Test - public void testEnableSummarization() throws AccumuloException { + public void testEnableSummarization() { SummarizerConfiguration summarizerConfig1 = SummarizerConfiguration .builder("com.test.summarizer").setPropertyId("s1").addOption("opt1", "v1").build(); NewTableConfiguration ntcSummarization1 = diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java index 52d99c3724d..b53d54edacf 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java @@ -27,6 +27,7 @@ import java.util.Set; import org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.client.AccumuloClient; import org.apache.accumulo.core.client.IsolatedScanner; import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.data.Key; @@ -43,7 +44,6 @@ import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.manager.Manager; import org.apache.accumulo.manager.tableOps.ManagerRepo; -import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.fs.VolumeManager; import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection; import org.apache.accumulo.server.zookeeper.DistributedWorkQueue; @@ -120,7 +120,7 @@ public Repo call(long tid, Manager manager) throws Exception { */ // determine which failed files were loaded - ServerContext client = manager.getContext(); + AccumuloClient client = manager.getContext(); try (Scanner mscanner = new IsolatedScanner(client.createScanner(MetadataTable.NAME, Authorizations.EMPTY))) { mscanner.setRange(new KeyExtent(tableId, null, null).toMetaRange()); From b0c0644bf687e215f91a5865eafcd9c71f39d0cb Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Tue, 13 Jan 2026 14:15:52 -0500 Subject: [PATCH 06/21] trivial --- .../apache/accumulo/core/client/admin/NewTableConfiguration.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java index 4acde47a041..c35aca8c0f2 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java +++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NewTableConfiguration.java @@ -198,7 +198,6 @@ public Map getProperties() { var initTableProps = IteratorConfigUtil.getInitialTableProperties(); // check the properties for conflicts with default iterators var defaultIterSettings = IteratorConfigUtil.getInitialTableIteratorSettings(); - // if a default prop already exists, don't want to consider that a conflict for (var defaultIterSetting : defaultIterSettings.entrySet()) { var setting = defaultIterSetting.getKey(); var scopes = defaultIterSetting.getValue(); From f422862630a4817f40d33b427193ba988cfef71e Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Thu, 15 Jan 2026 12:44:47 -0500 Subject: [PATCH 07/21] Improved efficiency of iterator conflict check at scan time Also fixed a bug where I was calling regex.matches(str) instead of str.matches(regex) --- .../iteratorsImpl/IteratorConfigUtil.java | 124 ++++++++---------- .../tserver/tablet/ScanDataSource.java | 34 +++-- .../test/functional/IteratorConflictsIT.java | 47 ++++--- 3 files changed, 106 insertions(+), 99 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java index a8e353a400d..b0d7ba563f7 100644 --- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java @@ -384,84 +384,61 @@ public static void checkIteratorConflictsWithTablesInNamespace(TableOperations t } } - public static void checkScanIteratorConflicts(List iterInfos, - Map> iterOpts, IteratorSetting setting) throws AccumuloException { - Map props = new HashMap<>(); - for (var iterInfo : iterInfos) { - props.put( - String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX.getKey(), - IteratorScope.scan.name().toLowerCase(), iterInfo.getIterName()), - String.format("%s,%s", iterInfo.getPriority(), iterInfo.getClassName())); - var options = iterOpts.get(iterInfo.getIterName()); - if (options != null) { - for (var iterOpt : options.entrySet()) { - props.put(String.format("%s%s.%s.opt.%s", Property.TABLE_ITERATOR_PREFIX.getKey(), - IteratorScope.scan.name().toLowerCase(), iterInfo.getIterName(), iterOpt.getKey()), - iterOpt.getValue()); - } - } - } - checkIteratorConflicts(props, setting, EnumSet.of(IteratorScope.scan)); - } - - public static void checkIteratorConflicts(Map props, IteratorSetting setting, - EnumSet scopes) throws AccumuloException { - for (IteratorScope scope : scopes) { - String scopeStr = - String.format("%s%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase()); - String nameStr = String.format("%s.%s", scopeStr, setting.getName()); - String optStr = String.format("%s.opt.", nameStr); - String valStr = String.format("%s,%s", setting.getPriority(), setting.getIteratorClass()); - Map optionConflicts = new TreeMap<>(); - // skip if the setting is present in the map... not a conflict if exactly the same - if (props.containsKey(nameStr) && props.get(nameStr).equals(valStr) - && IteratorConfigUtil.containsSameIterOpts(props, setting, optStr)) { + public static void checkIteratorConflicts(IteratorSetting iterToCheck, + EnumSet iterToCheckScopes, + Map> existingIters) throws AccumuloException { + for (var scope : iterToCheckScopes) { + var existingItersForScope = existingIters.get(scope); + if (existingItersForScope == null) { continue; } - for (Entry property : props.entrySet()) { - if (property.getKey().startsWith(scopeStr)) { - if (property.getKey().equals(nameStr)) { - throw new AccumuloException(new IllegalArgumentException("iterator name conflict for " - + setting.getName() + ": " + property.getKey() + "=" + property.getValue())); - } - if (property.getKey().startsWith(optStr)) { - optionConflicts.put(property.getKey(), property.getValue()); - } - if (property.getKey().contains(".opt.")) { - continue; - } - String[] parts = property.getValue().split(","); - if (parts.length != 2) { - throw new AccumuloException("Bad value for existing iterator setting: " - + property.getKey() + "=" + property.getValue()); - } - try { - if (Integer.parseInt(parts[0]) == setting.getPriority()) { - throw new AccumuloException(new IllegalArgumentException( - "iterator priority conflict: " + property.getKey() + "=" + property.getValue())); - } - } catch (NumberFormatException e) { - throw new AccumuloException("Bad value for existing iterator setting: " - + property.getKey() + "=" + property.getValue()); - } + for (var existingIter : existingItersForScope) { + // not a conflict if exactly the same + if (iterToCheck.equals(existingIter)) { + continue; + } + if (iterToCheck.getName().equals(existingIter.getName())) { + String msg = + String.format("iterator name conflict at %s scope. %s conflicts with existing %s", + scope, iterToCheck, existingIter); + throw new AccumuloException(new IllegalArgumentException(msg)); + } + if (iterToCheck.getPriority() == existingIter.getPriority()) { + String msg = + String.format("iterator priority conflict at %s scope. %s conflicts with existing %s", + scope, iterToCheck, existingIter); + throw new AccumuloException(new IllegalArgumentException(msg)); } - } - if (!optionConflicts.isEmpty()) { - throw new AccumuloException(new IllegalArgumentException( - "iterator options conflict for " + setting.getName() + ": " + optionConflicts)); } } } - public static boolean containsSameIterOpts(Map props, IteratorSetting setting, - String optStr) { - for (var opt : setting.getOptions().entrySet()) { - final String optKey = optStr + opt.getKey(); - if (!props.containsKey(optKey) || !props.get(optKey).equals(opt.getValue())) { - return false; + public static void checkIteratorConflicts(Map props, IteratorSetting iterToCheck, + EnumSet iterToCheckScopes) throws AccumuloException { + // parse the props map + Map> existingIters = + new HashMap<>(IteratorScope.values().length); + for (var prop : props.entrySet()) { + if (isNonOptionIterProp(prop.getKey(), prop.getValue())) { + var propKeyParts = prop.getKey().split("\\."); + var scope = IteratorScope.valueOf(propKeyParts[2]); + var name = propKeyParts[3]; + var propValParts = prop.getValue().split(","); + var priority = Integer.parseInt(propValParts[0]); + var clazz = propValParts[1]; + var existingIter = + new IteratorSetting(priority, name, clazz, gatherIterOpts(prop.getKey(), props)); + if (existingIters.get(scope) == null) { + List iters = new ArrayList<>(); + iters.add(existingIter); + existingIters.put(scope, iters); + } else { + existingIters.get(scope).add(existingIter); + } } } - return true; + // check for conflicts + checkIteratorConflicts(iterToCheck, iterToCheckScopes, existingIters); } /** @@ -472,8 +449,12 @@ public static boolean isNonOptionIterProp(String propKey, String propVal) { && propVal.matches(IteratorConfigUtil.ITERATOR_PROP_VAL_REGEX); } + public static boolean isOptionIterProp(String propKey) { + return propKey.matches(ITERATOR_PROP_OPT_REGEX); + } + public static boolean isIterProp(String propKey, String propVal) { - return isNonOptionIterProp(propKey, propVal) || ITERATOR_PROP_OPT_REGEX.matches(propKey); + return isNonOptionIterProp(propKey, propVal) || isOptionIterProp(propKey); } /** @@ -494,8 +475,7 @@ public static Map gatherIteratorProps(Map props) { public static Map gatherIterOpts(String iterPropKey, Map map) { Map opts = new HashMap<>(); for (var iteratorProp : map.entrySet()) { - if (ITERATOR_PROP_OPT_REGEX.matches(iteratorProp.getKey()) - && iteratorProp.getKey().contains(iterPropKey)) { + if (isOptionIterProp(iteratorProp.getKey()) && iteratorProp.getKey().contains(iterPropKey)) { String[] parts = iteratorProp.getKey().split("\\."); opts.put(parts[parts.length - 1], iteratorProp.getValue()); } diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java index 2a06283f7e2..f7ff5de9754 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.EnumSet; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -247,20 +248,15 @@ private SortedKeyValueIterator createIterator() // iterator options. // First ensure the set iterators do not conflict with the existing table iterators. + List picIteratorSettings = new ArrayList<>(pic.getIterInfo().size()); + for (var picIterInfo : pic.getIterInfo()) { + picIteratorSettings.add(getIteratorSetting(picIterInfo, pic.getOpts())); + } for (var scanParamIterInfo : scanParams.getSsiList()) { - IteratorSetting setting; - var scanParamIterOpts = scanParams.getSsio().get(scanParamIterInfo.getIterName()); - if (scanParamIterOpts != null) { - setting = new IteratorSetting(scanParamIterInfo.getPriority(), - scanParamIterInfo.getIterName(), scanParamIterInfo.getClassName(), - scanParamIterOpts); - } else { - setting = new IteratorSetting(scanParamIterInfo.getPriority(), - scanParamIterInfo.getIterName(), scanParamIterInfo.getClassName()); - } try { - IteratorConfigUtil.checkScanIteratorConflicts(pic.getIterInfo(), pic.getOpts(), - setting); + IteratorConfigUtil.checkIteratorConflicts( + getIteratorSetting(scanParamIterInfo, scanParams.getSsio()), + EnumSet.of(IteratorScope.scan), Map.of(IteratorScope.scan, picIteratorSettings)); } catch (AccumuloException e) { throw new IllegalArgumentException(e); } @@ -294,6 +290,20 @@ private SortedKeyValueIterator createIterator() } } + private IteratorSetting getIteratorSetting(IterInfo iterInfo, + Map> iterOpts) { + IteratorSetting setting; + var opts = iterOpts.get(iterInfo.getIterName()); + if (opts != null) { + setting = new IteratorSetting(iterInfo.getPriority(), iterInfo.getIterName(), + iterInfo.getClassName(), opts); + } else { + setting = new IteratorSetting(iterInfo.getPriority(), iterInfo.getIterName(), + iterInfo.getClassName()); + } + return setting; + } + private void returnIterators() { if (memIters != null) { log.trace("Returning mem iterators for {}, scanId:{}, fid:{}", tablet.getExtent(), diff --git a/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java index 170a5872e16..d2c096257e2 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java @@ -113,6 +113,11 @@ public class IteratorConflictsIT extends SharedMiniClusterBase { + IteratorUtil.IteratorScope.scan.name().toLowerCase() + "." + defaultTableIter.getName(); private static final String defaultIterVal = defaultTableIter.getPriority() + "," + defaultTableIter.getIteratorClass(); + private static final String defaultIterOptKey = Property.TABLE_ITERATOR_PREFIX.getKey() + + IteratorUtil.IteratorScope.scan.name().toLowerCase() + "." + defaultTableIter.getName() + + ".opt." + defaultTableIter.getOptions().entrySet().iterator().next().getKey(); + private static final String defaultIterOptVal = + defaultTableIter.getOptions().entrySet().iterator().next().getValue(); @BeforeAll public static void startup() throws Exception { @@ -470,10 +475,14 @@ public void testDefaultIterConflict() throws Throwable { String noDefaultsTable12 = ns12 + "." + tables[22]; tops.create(noDefaultsTable12, new NewTableConfiguration().withoutDefaults()); testDefaultIterConflict(AccumuloException.class, - () -> nops.setProperty(ns11, defaultIterPrioConflictKey, defaultIterPrioConflictVal), - () -> nops.setProperty(ns11, defaultIterNameConflictKey, defaultIterNameConflictVal), - () -> nops.setProperty(ns12, defaultIterPrioConflictKey, defaultIterPrioConflictVal), - () -> nops.setProperty(ns12, defaultIterNameConflictKey, defaultIterNameConflictVal)); + () -> nops.modifyProperties(ns11, + props -> props.put(defaultIterPrioConflictKey, defaultIterPrioConflictVal)), + () -> nops.modifyProperties(ns11, + props -> props.put(defaultIterNameConflictKey, defaultIterNameConflictVal)), + () -> nops.modifyProperties(ns12, + props -> props.put(defaultIterPrioConflictKey, defaultIterPrioConflictVal)), + () -> nops.modifyProperties(ns12, + props -> props.put(defaultIterNameConflictKey, defaultIterNameConflictVal))); // testing CloneConfiguration.Builder.setPropertiesToSet String dst1 = tables[23]; @@ -519,6 +528,11 @@ private void testDefaultIterConflict(Class exceptionCla @Test public void testSameIterNoConflict() throws Throwable { + // note about setProperty calls in this test. The default table iter has an option so the + // property representation of this iter is a property for the iter and a property for the + // option (2 properties). Obviously we cannot call setProperty with both of these properties, + // but calling setProperty for one of these properties should be fine as it has no effect on + // the config. final String[] names = getUniqueNames(16); // testing Scanner.addScanIterator @@ -536,9 +550,6 @@ public void testSameIterNoConflict() throws Throwable { } // testing TableOperations.setProperty - // note that this is not technically the exact same iterator since the default iterator has - // options (which are separate properties), but this call has no effect on the - // property map/iterators, so this call should not throw final String table2 = names[1]; tops.create(table2); tops.attachIterator(table2, iter1); @@ -546,15 +557,15 @@ public void testSameIterNoConflict() throws Throwable { () -> tops.setProperty(table2, defaultIterKey, defaultIterVal)); // testing TableOperations.modifyProperties - // note that this is not technically the exact same iterator since the default iterator has - // options (which are separate properties), but this call has no effect on the - // property map/iterators, so this call should not throw final String table3 = names[2]; tops.create(table3); tops.attachIterator(table3, iter1); testSameIterNoConflict( () -> tops.modifyProperties(table3, props -> props.put(iter1Key, iter1Val)), - () -> tops.modifyProperties(table3, props -> props.put(defaultIterKey, defaultIterVal))); + () -> tops.modifyProperties(table3, props -> { + props.put(defaultIterKey, defaultIterVal); + props.put(defaultIterOptKey, defaultIterOptVal); + })); // testing NewTableConfiguration.attachIterator final String ns1 = names[3]; @@ -597,8 +608,11 @@ public void testSameIterNoConflict() throws Throwable { nops.create(ns4); tops.create(table9); tops.attachIterator(table9, iter1); - testSameIterNoConflict(() -> nops.setProperty(ns4, iter1Key, iter1Val), - () -> nops.setProperty(ns4, defaultIterKey, defaultIterVal)); + testSameIterNoConflict(() -> nops.modifyProperties(ns4, props -> props.put(iter1Key, iter1Val)), + () -> nops.modifyProperties(ns4, props -> { + props.put(defaultIterKey, defaultIterVal); + props.put(defaultIterOptKey, defaultIterOptVal); + })); // testing CloneConfiguration.Builder.setPropertiesToSet final String src = names[13]; @@ -609,8 +623,11 @@ public void testSameIterNoConflict() throws Throwable { testSameIterNoConflict( () -> tops.clone(src, dst1, CloneConfiguration.builder().setPropertiesToSet(Map.of(iter1Key, iter1Val)).build()), - () -> tops.clone(src, dst2, CloneConfiguration.builder() - .setPropertiesToSet(Map.of(defaultIterKey, defaultIterVal)).build())); + () -> tops.clone(src, dst2, + CloneConfiguration.builder() + .setPropertiesToSet( + Map.of(defaultIterKey, defaultIterVal, defaultIterOptKey, defaultIterOptVal)) + .build())); } private void testSameIterNoConflict(Executable addIter1Executable, From 6a3f9f8c795cbd2ff687a4e997525c4418b56c44 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Fri, 16 Jan 2026 16:50:43 -0500 Subject: [PATCH 08/21] Fix bug: Iterator option conflicts were not considered in new impl of checkIteratorConflicts --- .../iteratorsImpl/IteratorConfigUtil.java | 34 +++++++++++++------ 1 file changed, 23 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java index b0d7ba563f7..f516b2e67d3 100644 --- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java @@ -385,9 +385,9 @@ public static void checkIteratorConflictsWithTablesInNamespace(TableOperations t } public static void checkIteratorConflicts(IteratorSetting iterToCheck, - EnumSet iterToCheckScopes, + EnumSet iterScopesToCheck, Map> existingIters) throws AccumuloException { - for (var scope : iterToCheckScopes) { + for (var scope : iterScopesToCheck) { var existingItersForScope = existingIters.get(scope); if (existingItersForScope == null) { continue; @@ -414,7 +414,7 @@ public static void checkIteratorConflicts(IteratorSetting iterToCheck, } public static void checkIteratorConflicts(Map props, IteratorSetting iterToCheck, - EnumSet iterToCheckScopes) throws AccumuloException { + EnumSet iterScopesToCheck) throws AccumuloException { // parse the props map Map> existingIters = new HashMap<>(IteratorScope.values().length); @@ -428,17 +428,29 @@ public static void checkIteratorConflicts(Map props, IteratorSett var clazz = propValParts[1]; var existingIter = new IteratorSetting(priority, name, clazz, gatherIterOpts(prop.getKey(), props)); - if (existingIters.get(scope) == null) { - List iters = new ArrayList<>(); - iters.add(existingIter); - existingIters.put(scope, iters); - } else { - existingIters.get(scope).add(existingIter); - } + existingIters.computeIfAbsent(scope, s -> new ArrayList<>()).add(existingIter); } } + // check for conflicts - checkIteratorConflicts(iterToCheck, iterToCheckScopes, existingIters); + // any iterator option property not part of an existing iterator is an option conflict + for (var prop : props.entrySet()) { + if (isOptionIterProp(prop.getKey())) { + var iterOptPropParts = prop.getKey().split("\\."); + var scope = IteratorScope.valueOf(iterOptPropParts[2]); + var optKey = iterOptPropParts[iterOptPropParts.length - 1]; + var iterName = iterOptPropParts[3]; + if (!existingIters.containsKey(scope) || existingIters.get(scope).stream() + .noneMatch(is -> is.getName().equals(iterName) && is.getOptions().containsKey(optKey) + && is.getOptions().get(optKey).equals(prop.getValue()))) { + String msg = String.format("iterator options conflict for %s : %s=%s", + iterToCheck.getName(), prop.getKey(), prop.getValue()); + throw new AccumuloException(new IllegalArgumentException(msg)); + } + } + } + // check if the given iterator conflicts with any existing iterators + checkIteratorConflicts(iterToCheck, iterScopesToCheck, existingIters); } /** From 2e5da9b5fff365e0b72fdfc580ba7967b364d966 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Tue, 20 Jan 2026 14:40:58 -0500 Subject: [PATCH 09/21] Implemented more efficient alg for iterator conflict checks at scan time Co-authored-by: Keith Turner --- .../server/conf/TableConfiguration.java | 15 ++++++++ .../tserver/tablet/ScanDataSource.java | 38 +++++++++++-------- 2 files changed, 38 insertions(+), 15 deletions(-) diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java index d0699e8a1d7..97917015719 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java +++ b/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Objects; +import java.util.Set; import java.util.stream.Collectors; import org.apache.accumulo.core.classloader.ClassLoaderUtil; @@ -116,6 +117,8 @@ public static class ParsedIteratorConfig { private final List tableIters; private final Map> tableOpts; private final String context; + private final Set uniqueNames; + private final Set uniquePriorities; private ParsedIteratorConfig(List ii, Map> opts, String context) { @@ -123,6 +126,10 @@ private ParsedIteratorConfig(List ii, Map> o tableOpts = opts.entrySet().stream() .collect(Collectors.toUnmodifiableMap(Entry::getKey, e -> Map.copyOf(e.getValue()))); this.context = context; + uniqueNames = + tableIters.stream().map(IterInfo::getIterName).collect(Collectors.toUnmodifiableSet()); + uniquePriorities = + tableIters.stream().map(IterInfo::getPriority).collect(Collectors.toUnmodifiableSet()); } public List getIterInfo() { @@ -136,6 +143,14 @@ public Map> getOpts() { public String getServiceEnv() { return context; } + + public Set getUniqueNames() { + return uniqueNames; + } + + public Set getUniquePriorities() { + return uniquePriorities; + } } public ParsedIteratorConfig getParsedIteratorConfig(IteratorScope scope) { diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java index f7ff5de9754..fa59e1c02c0 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java @@ -248,17 +248,27 @@ private SortedKeyValueIterator createIterator() // iterator options. // First ensure the set iterators do not conflict with the existing table iterators. - List picIteratorSettings = new ArrayList<>(pic.getIterInfo().size()); - for (var picIterInfo : pic.getIterInfo()) { - picIteratorSettings.add(getIteratorSetting(picIterInfo, pic.getOpts())); - } + List picIteratorSettings = null; for (var scanParamIterInfo : scanParams.getSsiList()) { - try { - IteratorConfigUtil.checkIteratorConflicts( - getIteratorSetting(scanParamIterInfo, scanParams.getSsio()), - EnumSet.of(IteratorScope.scan), Map.of(IteratorScope.scan, picIteratorSettings)); - } catch (AccumuloException e) { - throw new IllegalArgumentException(e); + // Quick check for a potential iterator conflict (does not consider iterator scope). + // This avoids the more expensive check method call most of the time. + if (pic.getUniqueNames().contains(scanParamIterInfo.getIterName()) + || pic.getUniquePriorities().contains(scanParamIterInfo.getPriority())) { + if (picIteratorSettings == null) { + picIteratorSettings = new ArrayList<>(pic.getIterInfo().size()); + for (var picIterInfo : pic.getIterInfo()) { + picIteratorSettings.add( + getIteratorSetting(picIterInfo, pic.getOpts().get(picIterInfo.getIterName()))); + } + } + try { + IteratorConfigUtil.checkIteratorConflicts( + getIteratorSetting(scanParamIterInfo, + scanParams.getSsio().get(scanParamIterInfo.getIterName())), + EnumSet.of(IteratorScope.scan), Map.of(IteratorScope.scan, picIteratorSettings)); + } catch (AccumuloException e) { + throw new IllegalArgumentException(e); + } } } iterOpts = new HashMap<>(pic.getOpts().size() + scanParams.getSsio().size()); @@ -290,13 +300,11 @@ private SortedKeyValueIterator createIterator() } } - private IteratorSetting getIteratorSetting(IterInfo iterInfo, - Map> iterOpts) { + private IteratorSetting getIteratorSetting(IterInfo iterInfo, Map iterOpts) { IteratorSetting setting; - var opts = iterOpts.get(iterInfo.getIterName()); - if (opts != null) { + if (iterOpts != null) { setting = new IteratorSetting(iterInfo.getPriority(), iterInfo.getIterName(), - iterInfo.getClassName(), opts); + iterInfo.getClassName(), iterOpts); } else { setting = new IteratorSetting(iterInfo.getPriority(), iterInfo.getIterName(), iterInfo.getClassName()); From 469a83ebcddc8af3371eefc872aded0d1470dd67 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Wed, 21 Jan 2026 11:26:31 -0500 Subject: [PATCH 10/21] Newly added iterator conflict checks now log a warning instead of throwing --- .../clientImpl/NamespaceOperationsHelper.java | 2 +- .../clientImpl/TableOperationsHelper.java | 2 +- .../core/clientImpl/TableOperationsImpl.java | 10 +-- .../iteratorsImpl/IteratorConfigUtil.java | 63 +++++++++++-------- .../tserver/tablet/ScanDataSource.java | 3 +- 5 files changed, 45 insertions(+), 35 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsHelper.java index 360f05df11e..9df25d3251a 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsHelper.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsHelper.java @@ -148,7 +148,7 @@ public void checkIteratorConflicts(String namespace, IteratorSetting setting, throw new NamespaceNotFoundException(null, namespace, null); } var props = this.getNamespaceProperties(namespace); - IteratorConfigUtil.checkIteratorConflicts(props, setting, scopes); + IteratorConfigUtil.checkIteratorConflicts(props, setting, scopes, true); } @Override diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsHelper.java index bf138ffe7f8..6cf7e5e3d78 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsHelper.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsHelper.java @@ -140,7 +140,7 @@ public static void checkIteratorConflicts(Map props, IteratorSett EnumSet scopes) throws AccumuloException { checkArgument(setting != null, "setting is null"); checkArgument(scopes != null, "scopes is null"); - IteratorConfigUtil.checkIteratorConflicts(props, setting, scopes); + IteratorConfigUtil.checkIteratorConflicts(props, setting, scopes, true); } @Override diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java index f8b1642c62e..ac7fe53f8ea 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java @@ -1041,13 +1041,9 @@ private Map tryToModifyProperties(String tableName, // from here on the code is assured to always be dealing with the same map. vProperties.setProperties(Map.copyOf(vProperties.getProperties())); - try { - for (var property : vProperties.getProperties().entrySet()) { - IteratorConfigUtil.checkIteratorConflicts(configBeforeMut, property.getKey(), - property.getValue()); - } - } catch (TableNotFoundException e) { - throw new AccumuloException(e); + for (var property : vProperties.getProperties().entrySet()) { + IteratorConfigUtil.checkIteratorConflicts(configBeforeMut, property.getKey(), + property.getValue()); } try { diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java index f516b2e67d3..869e09890bf 100644 --- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java @@ -43,7 +43,6 @@ import org.apache.accumulo.core.client.admin.TableOperations; import org.apache.accumulo.core.clientImpl.Namespace; import org.apache.accumulo.core.clientImpl.NamespaceOperationsHelper; -import org.apache.accumulo.core.clientImpl.TableOperationsHelper; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Key; @@ -77,6 +76,8 @@ public class IteratorConfigUtil { + Arrays.stream(IteratorScope.values()).map(scope -> scope.name().toLowerCase()) .collect(Collectors.joining(".|")) + ".)").replace(".", "\\.") + "[^.]+\\.opt\\.[^.]+$"; + private static final String WARNING_MSG = + ". Iterator was set as requested, but may lead to non-deterministic behavior."; /** * Fetch the correct configuration key prefix for the given scope. Throws an @@ -296,12 +297,12 @@ private static Class> loadClass(boolean useAcc } public static void checkIteratorConflicts(Map props, String property, String value) - throws AccumuloException, TableNotFoundException, IllegalArgumentException { + throws AccumuloException { if (props.containsKey(property) && props.get(property).equals(value)) { // setting a property that already exists (i.e., no change) return; } - if (IteratorConfigUtil.isNonOptionIterProp(property, value)) { + if (isNonOptionIterProp(property, value)) { String[] iterPropParts = property.split("\\."); IteratorScope scope = IteratorScope.valueOf(iterPropParts[2]); String iterName = iterPropParts[3]; @@ -311,7 +312,7 @@ public static void checkIteratorConflicts(Map props, String prope // iterator is if the existing iterator has no options (opts are set as separate props) IteratorSetting givenIter = new IteratorSetting(Integer.parseInt(priorityAndClass[0]), iterName, priorityAndClass[1]); - TableOperationsHelper.checkIteratorConflicts(props, givenIter, EnumSet.of(scope)); + checkIteratorConflicts(props, givenIter, EnumSet.of(scope), false); } } } @@ -326,7 +327,7 @@ public static void checkIteratorConflicts(TableOperations tableOps, NamespaceOpe } // checking for conflicts in the namespace - if (IteratorConfigUtil.isNonOptionIterProp(property, value)) { + if (isNonOptionIterProp(property, value)) { String[] iterPropParts = property.split("\\."); IteratorScope scope = IteratorScope.valueOf(iterPropParts[2]); String iterName = iterPropParts[3]; @@ -336,7 +337,7 @@ public static void checkIteratorConflicts(TableOperations tableOps, NamespaceOpe // iterator is if the existing iterator has no options (opts are set as separate props) IteratorSetting givenIter = new IteratorSetting(Integer.parseInt(priorityAndClass[0]), iterName, priorityAndClass[1]); - noh.checkIteratorConflicts(namespace, givenIter, EnumSet.of(scope)); + checkIteratorConflicts(props, givenIter, EnumSet.of(scope), false); } } @@ -349,17 +350,17 @@ public static void checkIteratorConflictsWithTablesInNamespace(TableOperations t throws AccumuloException { Set tablesInNamespace; if (namespace.equals(Namespace.DEFAULT.name())) { - tablesInNamespace = - tableOps.list().stream().filter(t -> t.startsWith(namespace)).collect(Collectors.toSet()); + tablesInNamespace = tableOps.list().stream().filter(t -> !t.contains(Namespace.SEPARATOR)) + .collect(Collectors.toSet()); } else { tablesInNamespace = tableOps.list().stream() .filter(t -> t.startsWith(namespace + Namespace.SEPARATOR)).collect(Collectors.toSet()); } try { for (var table : tablesInNamespace) { - IteratorConfigUtil.checkIteratorConflicts(tableOps.getTableProperties(table), is, scopes); + checkIteratorConflicts(tableOps.getTableProperties(table), is, scopes, false); } - } catch (TableNotFoundException | IllegalArgumentException e) { + } catch (TableNotFoundException e) { throw new AccumuloException(e); } } @@ -368,25 +369,27 @@ public static void checkIteratorConflictsWithTablesInNamespace(TableOperations t String namespace, String property, String value) throws AccumuloException { Set tablesInNamespace; if (namespace.equals(Namespace.DEFAULT.name())) { - tablesInNamespace = - tableOps.list().stream().filter(t -> t.startsWith(namespace)).collect(Collectors.toSet()); + tablesInNamespace = tableOps.list().stream().filter(t -> !t.contains(Namespace.SEPARATOR)) + .collect(Collectors.toSet()); } else { tablesInNamespace = tableOps.list().stream() .filter(t -> t.startsWith(namespace + Namespace.SEPARATOR)).collect(Collectors.toSet()); } try { for (var table : tablesInNamespace) { - IteratorConfigUtil.checkIteratorConflicts(tableOps.getTableProperties(table), property, - value); + checkIteratorConflicts(tableOps.getTableProperties(table), property, value); } - } catch (TableNotFoundException | IllegalArgumentException e) { + } catch (TableNotFoundException e) { throw new AccumuloException(e); } } public static void checkIteratorConflicts(IteratorSetting iterToCheck, EnumSet iterScopesToCheck, - Map> existingIters) throws AccumuloException { + Map> existingIters, boolean shouldThrow) + throws AccumuloException { + // The reason for the 'shouldThrow' var is to prevent newly added 2.x checks from breaking + // existing user code. Just log the problem and proceed. Major version > 2 will always throw for (var scope : iterScopesToCheck) { var existingItersForScope = existingIters.get(scope); if (existingItersForScope == null) { @@ -401,20 +404,28 @@ public static void checkIteratorConflicts(IteratorSetting iterToCheck, String msg = String.format("iterator name conflict at %s scope. %s conflicts with existing %s", scope, iterToCheck, existingIter); - throw new AccumuloException(new IllegalArgumentException(msg)); + if (shouldThrow) { + throw new AccumuloException(new IllegalArgumentException(msg)); + } else { + log.warn(msg + WARNING_MSG); + } } if (iterToCheck.getPriority() == existingIter.getPriority()) { String msg = String.format("iterator priority conflict at %s scope. %s conflicts with existing %s", scope, iterToCheck, existingIter); - throw new AccumuloException(new IllegalArgumentException(msg)); + if (shouldThrow) { + throw new AccumuloException(new IllegalArgumentException(msg)); + } else { + log.warn(msg + WARNING_MSG); + } } } } } public static void checkIteratorConflicts(Map props, IteratorSetting iterToCheck, - EnumSet iterScopesToCheck) throws AccumuloException { + EnumSet iterScopesToCheck, boolean shouldThrow) throws AccumuloException { // parse the props map Map> existingIters = new HashMap<>(IteratorScope.values().length); @@ -445,20 +456,23 @@ public static void checkIteratorConflicts(Map props, IteratorSett && is.getOptions().get(optKey).equals(prop.getValue()))) { String msg = String.format("iterator options conflict for %s : %s=%s", iterToCheck.getName(), prop.getKey(), prop.getValue()); - throw new AccumuloException(new IllegalArgumentException(msg)); + if (shouldThrow) { + throw new AccumuloException(new IllegalArgumentException(msg)); + } else { + log.warn(msg + WARNING_MSG); + } } } } // check if the given iterator conflicts with any existing iterators - checkIteratorConflicts(iterToCheck, iterScopesToCheck, existingIters); + checkIteratorConflicts(iterToCheck, iterScopesToCheck, existingIters, shouldThrow); } /** * Returns true if the property is an iterator property not including iterator option properties */ public static boolean isNonOptionIterProp(String propKey, String propVal) { - return propKey.matches(IteratorConfigUtil.ITERATOR_PROP_REGEX) - && propVal.matches(IteratorConfigUtil.ITERATOR_PROP_VAL_REGEX); + return propKey.matches(ITERATOR_PROP_REGEX) && propVal.matches(ITERATOR_PROP_VAL_REGEX); } public static boolean isOptionIterProp(String propKey) { @@ -474,8 +488,7 @@ public static boolean isIterProp(String propKey, String propVal) { */ public static Map gatherIteratorProps(Map props) { Map iterProps = new HashMap<>(); - props.entrySet().stream() - .filter(entry -> IteratorConfigUtil.isIterProp(entry.getKey(), entry.getValue())) + props.entrySet().stream().filter(entry -> isIterProp(entry.getKey(), entry.getValue())) .forEach(entry -> iterProps.put(entry.getKey(), entry.getValue())); return iterProps; } diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java index fa59e1c02c0..f999feff0c3 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java @@ -265,7 +265,8 @@ private SortedKeyValueIterator createIterator() IteratorConfigUtil.checkIteratorConflicts( getIteratorSetting(scanParamIterInfo, scanParams.getSsio().get(scanParamIterInfo.getIterName())), - EnumSet.of(IteratorScope.scan), Map.of(IteratorScope.scan, picIteratorSettings)); + EnumSet.of(IteratorScope.scan), Map.of(IteratorScope.scan, picIteratorSettings), + false); } catch (AccumuloException e) { throw new IllegalArgumentException(e); } From d6e9e1cd00e883202bc74892828d5e827f92ef5c Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Thu, 22 Jan 2026 14:43:16 -0500 Subject: [PATCH 11/21] Updated IteratorConflictsIT based on switch from throwing exception to logging warning --- .../accumulo/manager/FateServiceHandler.java | 3 +- .../test/functional/IteratorConflictsIT.java | 371 +++++++++++++----- 2 files changed, 275 insertions(+), 99 deletions(-) diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java index f5a7a0a46b9..d0ee1e9a123 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java @@ -53,7 +53,6 @@ import org.apache.accumulo.core.client.admin.InitialTableState; import org.apache.accumulo.core.client.admin.TimeType; import org.apache.accumulo.core.clientImpl.Namespaces; -import org.apache.accumulo.core.clientImpl.TableOperationsHelper; import org.apache.accumulo.core.clientImpl.TableOperationsImpl; import org.apache.accumulo.core.clientImpl.UserCompactionUtils; import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode; @@ -874,7 +873,7 @@ private void validateTableProperty(String propKey, String propVal, Map events = new CopyOnWriteArrayList<>(); + + public TestAppender() { + super("TestAppender", null, PatternLayout.createDefaultLayout(), false, null); + } + + @Override + public void append(LogEvent event) { + events.add(event.toImmutable()); + } + + public List events() { + return events; + } + } + @BeforeAll public static void startup() throws Exception { SharedMiniClusterBase.startMiniCluster(); client = Accumulo.newClient().from(getClientProps()).build(); tops = client.tableOperations(); nops = client.namespaceOperations(); + appender.start(); + loggerConfig.getRootLogger().addAppender(appender, Level.WARN, null); + loggerContext.updateLoggers(); } @AfterAll public static void shutdown() throws Exception { client.close(); SharedMiniClusterBase.stopMiniCluster(); + loggerConfig.getRootLogger().removeAppender(appender.getName()); + appender.stop(); + loggerContext.updateLoggers(); } @Test - public void testTableIterConflict() throws Exception { - final String[] tableNames = getUniqueNames(12); + public void testTableIterConflict() throws Throwable { + final String[] tableNames = getUniqueNames(13); String table1 = tableNames[0]; String table2 = tableNames[1]; String table3 = tableNames[2]; @@ -162,20 +211,21 @@ public void testTableIterConflict() throws Exception { }, () -> { scanner2.addScanIterator(iter1NameConflict); scanner2.iterator().hasNext(); - }); + }, false); } // testing TableOperations.setProperty testTableIterConflict(table2, AccumuloException.class, () -> tops.setProperty(table2, iter1PrioConflictKey, iter1PrioConflictVal), - () -> tops.setProperty(table2, iter1NameConflictKey, iter1NameConflictVal)); + () -> tops.setProperty(table2, iter1NameConflictKey, iter1NameConflictVal), false); // testing TableOperations.modifyProperties testTableIterConflict(table3, AccumuloException.class, () -> tops.modifyProperties(table3, props -> props.put(iter1PrioConflictKey, iter1PrioConflictVal)), () -> tops.modifyProperties(table3, - props -> props.put(iter1NameConflictKey, iter1NameConflictVal))); + props -> props.put(iter1NameConflictKey, iter1NameConflictVal)), + false); // NewTableConfiguration.attachIterator is not applicable for this test // Attaching the iterator to the table requires the table to exist, but testing @@ -184,45 +234,57 @@ public void testTableIterConflict() throws Exception { // testing TableOperations.attachIterator testTableIterConflict(table4, AccumuloException.class, () -> tops.attachIterator(table4, iter1PrioConflict), - () -> tops.attachIterator(table4, iter1NameConflict)); + () -> tops.attachIterator(table4, iter1NameConflict), true); // testing NamespaceOperations.attachIterator testTableIterConflict(table5, AccumuloException.class, () -> nops.attachIterator(ns5, iter1PrioConflict), - () -> nops.attachIterator(ns5, iter1NameConflict)); + () -> nops.attachIterator(ns5, iter1NameConflict), false); // testing NamespaceOperations.setProperty testTableIterConflict(table6, AccumuloException.class, () -> nops.setProperty(ns6, iter1PrioConflictKey, iter1PrioConflictVal), - () -> nops.setProperty(ns6, iter1NameConflictKey, iter1NameConflictVal)); + () -> nops.setProperty(ns6, iter1NameConflictKey, iter1NameConflictVal), false); // testing NamespaceOperations.modifyProperties testTableIterConflict(table7, AccumuloException.class, () -> nops.modifyProperties(ns7, props -> props.put(iter1PrioConflictKey, iter1PrioConflictVal)), () -> nops.modifyProperties(ns7, - props -> props.put(iter1NameConflictKey, iter1NameConflictVal))); + props -> props.put(iter1NameConflictKey, iter1NameConflictVal)), + false); // testing CloneConfiguration.Builder.setPropertiesToSet String table9 = tableNames[11]; + String table10 = tableNames[12]; testTableIterConflict(table8, AccumuloException.class, () -> tops.clone(table8, table9, CloneConfiguration.builder() .setPropertiesToSet(Map.of(iter1PrioConflictKey, iter1PrioConflictVal)).build()), - () -> tops.clone(table8, table9, CloneConfiguration.builder() - .setPropertiesToSet(Map.of(iter1NameConflictKey, iter1NameConflictVal)).build())); + () -> tops.clone(table8, table10, + CloneConfiguration.builder() + .setPropertiesToSet(Map.of(iter1NameConflictKey, iter1NameConflictVal)).build()), + false); } private void testTableIterConflict(String table, Class exceptionClass, - Executable iterPrioConflictExec, Executable iterNameConflictExec) throws Exception { + Executable iterPrioConflictExec, Executable iterNameConflictExec, boolean shouldThrow) + throws Throwable { tops.attachIterator(table, iter1); - assertThrows(exceptionClass, iterPrioConflictExec); - assertThrows(exceptionClass, iterNameConflictExec); + if (shouldThrow) { + var e = assertThrows(exceptionClass, iterPrioConflictExec); + assertTrue(e.toString().contains("iterator priority conflict")); + e = assertThrows(exceptionClass, iterNameConflictExec); + assertTrue(e.toString().contains("iterator name conflict")); + } else { + assertTrue(logsContain(List.of("iterator priority conflict"), iterPrioConflictExec)); + assertTrue(logsContain(List.of("iterator name conflict"), iterNameConflictExec)); + } } @Test - public void testNamespaceIterConflict() throws Exception { - final String[] names = getUniqueNames(25); + public void testNamespaceIterConflict() throws Throwable { + final String[] names = getUniqueNames(28); // testing Scanner.addScanIterator String ns1 = names[0]; @@ -236,7 +298,7 @@ public void testNamespaceIterConflict() throws Exception { }, () -> { scanner2.addScanIterator(iter1NameConflict); scanner2.iterator().hasNext(); - }); + }, false); } // testing TableOperations.setProperty @@ -246,7 +308,7 @@ public void testNamespaceIterConflict() throws Exception { tops.create(table2); testNamespaceIterConflict(ns2, AccumuloException.class, () -> tops.setProperty(table2, iter1PrioConflictKey, iter1PrioConflictVal), - () -> tops.setProperty(table2, iter1NameConflictKey, iter1NameConflictVal)); + () -> tops.setProperty(table2, iter1NameConflictKey, iter1NameConflictVal), false); // testing TableOperations.modifyProperties String ns3 = names[4]; @@ -257,109 +319,127 @@ public void testNamespaceIterConflict() throws Exception { () -> tops.modifyProperties(table3, props -> props.put(iter1PrioConflictKey, iter1PrioConflictVal)), () -> tops.modifyProperties(table3, - props -> props.put(iter1NameConflictKey, iter1NameConflictVal))); + props -> props.put(iter1NameConflictKey, iter1NameConflictVal)), + false); // testing NewTableConfiguration.attachIterator String ns4 = names[6]; nops.create(ns4); String table4 = ns4 + "." + names[7]; + String table5 = ns4 + "." + names[8]; testNamespaceIterConflict(ns4, AccumuloException.class, () -> tops.create(table4, new NewTableConfiguration().attachIterator(iter1PrioConflict)), - () -> tops.create(table4, new NewTableConfiguration().attachIterator(iter1NameConflict))); + () -> tops.create(table5, new NewTableConfiguration().attachIterator(iter1NameConflict)), + false); // testing TableOperations.attachIterator - String ns5 = names[8]; + String ns5 = names[9]; nops.create(ns5); - String table5 = ns5 + "." + names[9]; - tops.create(table5); + String table6 = ns5 + "." + names[10]; + tops.create(table6); testNamespaceIterConflict(ns5, AccumuloException.class, - () -> tops.attachIterator(table5, iter1PrioConflict), - () -> tops.attachIterator(table5, iter1NameConflict)); + () -> tops.attachIterator(table6, iter1PrioConflict), + () -> tops.attachIterator(table6, iter1NameConflict), true); // testing NamespaceOperations.attachIterator - String ns6 = names[10]; + String ns6 = names[11]; nops.create(ns6); testNamespaceIterConflict(ns6, AccumuloException.class, () -> nops.attachIterator(ns6, iter1PrioConflict), - () -> nops.attachIterator(ns6, iter1NameConflict)); + () -> nops.attachIterator(ns6, iter1NameConflict), true); // testing NamespaceOperations.setProperty - String ns7 = names[11]; + String ns7 = names[12]; nops.create(ns7); testNamespaceIterConflict(ns7, AccumuloException.class, () -> nops.setProperty(ns7, iter1PrioConflictKey, iter1PrioConflictVal), - () -> nops.setProperty(ns7, iter1NameConflictKey, iter1NameConflictVal)); + () -> nops.setProperty(ns7, iter1NameConflictKey, iter1NameConflictVal), false); // testing NamespaceOperations.modifyProperties - String ns8 = names[12]; + String ns8 = names[13]; nops.create(ns8); testNamespaceIterConflict(ns8, AccumuloException.class, () -> nops.modifyProperties(ns8, props -> props.put(iter1PrioConflictKey, iter1PrioConflictVal)), () -> nops.modifyProperties(ns8, - props -> props.put(iter1NameConflictKey, iter1NameConflictVal))); + props -> props.put(iter1NameConflictKey, iter1NameConflictVal)), + false); // testing CloneConfiguration.Builder.setPropertiesToSet // testing same src and dst namespace: should conflict - String dstAndSrcNamespace1 = names[13]; + String dstAndSrcNamespace1 = names[14]; nops.create(dstAndSrcNamespace1); - String src1 = dstAndSrcNamespace1 + "." + names[14]; + String src1 = dstAndSrcNamespace1 + "." + names[15]; tops.create(src1); - String dst1 = dstAndSrcNamespace1 + "." + names[15]; + String dst1 = dstAndSrcNamespace1 + "." + names[16]; + String dst2 = dstAndSrcNamespace1 + "." + names[17]; testNamespaceIterConflict(dstAndSrcNamespace1, AccumuloException.class, () -> tops.clone(src1, dst1, CloneConfiguration.builder() .setPropertiesToSet(Map.of(iter1PrioConflictKey, iter1PrioConflictVal)).build()), - () -> tops.clone(src1, dst1, CloneConfiguration.builder() - .setPropertiesToSet(Map.of(iter1NameConflictKey, iter1NameConflictVal)).build())); + () -> tops.clone(src1, dst2, + CloneConfiguration.builder() + .setPropertiesToSet(Map.of(iter1NameConflictKey, iter1NameConflictVal)).build()), + false); // testing attached to src namespace, different dst namespace: should not conflict - String srcNamespace2 = names[16]; + String srcNamespace2 = names[18]; nops.create(srcNamespace2); nops.attachIterator(srcNamespace2, iter1); - String src2 = srcNamespace2 + "." + names[17]; + String src2 = srcNamespace2 + "." + names[19]; tops.create(src2); - String dstNamespace2 = names[18]; + String dstNamespace2 = names[20]; nops.create(dstNamespace2); - String dst2 = dstNamespace2 + "." + names[19]; - String dst3 = dstNamespace2 + "." + names[20]; + String dst3 = dstNamespace2 + "." + names[21]; + String dst4 = dstNamespace2 + "." + names[22]; // should NOT throw - tops.clone(src2, dst2, CloneConfiguration.builder() + tops.clone(src2, dst3, CloneConfiguration.builder() .setPropertiesToSet(Map.of(iter1PrioConflictKey, iter1PrioConflictVal)).build()); // should NOT throw - tops.clone(src2, dst3, CloneConfiguration.builder() + tops.clone(src2, dst4, CloneConfiguration.builder() .setPropertiesToSet(Map.of(iter1NameConflictKey, iter1NameConflictVal)).build()); // testing attached to dst namespace, different src namespace: should conflict - String srcNamespace3 = names[21]; + String srcNamespace3 = names[23]; nops.create(srcNamespace3); - String src3 = srcNamespace3 + "." + names[22]; + String src3 = srcNamespace3 + "." + names[24]; tops.create(src3); - String dstNamespace3 = names[23]; + String dstNamespace3 = names[25]; nops.create(dstNamespace3); - String dst4 = dstNamespace3 + "." + names[24]; + String dst5 = dstNamespace3 + "." + names[26]; + String dst6 = dstNamespace3 + "." + names[27]; testNamespaceIterConflict(dstNamespace3, AccumuloException.class, - () -> tops.clone(src3, dst4, + () -> tops.clone(src3, dst5, CloneConfiguration.builder() .setPropertiesToSet(Map.of(iter1PrioConflictKey, iter1PrioConflictVal)).build()), - () -> tops.clone(src3, dst4, CloneConfiguration.builder() - .setPropertiesToSet(Map.of(iter1NameConflictKey, iter1NameConflictVal)).build())); + () -> tops.clone(src3, dst6, + CloneConfiguration.builder() + .setPropertiesToSet(Map.of(iter1NameConflictKey, iter1NameConflictVal)).build()), + false); } private void testNamespaceIterConflict(String ns, Class exceptionClass, - Executable iterPrioConflictExec, Executable iterNameConflictExec) throws Exception { + Executable iterPrioConflictExec, Executable iterNameConflictExec, boolean shouldThrow) + throws Throwable { nops.attachIterator(ns, iter1); - assertThrows(exceptionClass, iterPrioConflictExec); - assertThrows(exceptionClass, iterNameConflictExec); + if (shouldThrow) { + var e = assertThrows(exceptionClass, iterPrioConflictExec); + assertTrue(e.toString().contains("iterator priority conflict")); + e = assertThrows(exceptionClass, iterNameConflictExec); + assertTrue(e.toString().contains("iterator name conflict")); + } else { + assertTrue(logsContain(List.of("iterator priority conflict"), iterPrioConflictExec)); + assertTrue(logsContain(List.of("iterator name conflict"), iterNameConflictExec)); + } } @Test public void testDefaultIterConflict() throws Throwable { - final String[] tables = getUniqueNames(27); + final String[] names = getUniqueNames(29); // testing Scanner.addScanIterator - String defaultsTable1 = tables[0]; + String defaultsTable1 = names[0]; tops.create(defaultsTable1); - String noDefaultsTable1 = tables[1]; + String noDefaultsTable1 = names[1]; tops.create(noDefaultsTable1, new NewTableConfiguration().withoutDefaults()); try (var defaultsScanner1 = client.createScanner(defaultsTable1); var noDefaultsScanner1 = client.createScanner(noDefaultsTable1); @@ -377,13 +457,13 @@ public void testDefaultIterConflict() throws Throwable { }, () -> { noDefaultsScanner2.addScanIterator(defaultIterNameConflict); noDefaultsScanner2.iterator().hasNext(); - }); + }, false); } // testing TableOperations.setProperty - String defaultsTable2 = tables[2]; + String defaultsTable2 = names[2]; tops.create(defaultsTable2); - String noDefaultsTable2 = tables[3]; + String noDefaultsTable2 = names[3]; tops.create(noDefaultsTable2, new NewTableConfiguration().withoutDefaults()); testDefaultIterConflict(AccumuloException.class, () -> tops.setProperty(defaultsTable2, defaultIterPrioConflictKey, @@ -393,12 +473,13 @@ public void testDefaultIterConflict() throws Throwable { () -> tops.setProperty(noDefaultsTable2, defaultIterPrioConflictKey, defaultIterPrioConflictVal), () -> tops.setProperty(noDefaultsTable2, defaultIterNameConflictKey, - defaultIterNameConflictVal)); + defaultIterNameConflictVal), + false); // testing TableOperations.modifyProperties - String defaultsTable3 = tables[4]; + String defaultsTable3 = names[4]; tops.create(defaultsTable3); - String noDefaultsTable3 = tables[5]; + String noDefaultsTable3 = names[5]; tops.create(noDefaultsTable3, new NewTableConfiguration().withoutDefaults()); testDefaultIterConflict(AccumuloException.class, () -> tops.modifyProperties(defaultsTable3, @@ -408,12 +489,13 @@ public void testDefaultIterConflict() throws Throwable { () -> tops.modifyProperties(noDefaultsTable3, props -> props.put(defaultIterPrioConflictKey, defaultIterPrioConflictVal)), () -> tops.modifyProperties(noDefaultsTable3, - props -> props.put(defaultIterNameConflictKey, defaultIterNameConflictVal))); + props -> props.put(defaultIterNameConflictKey, defaultIterNameConflictVal)), + false); // testing NewTableConfiguration.attachIterator - String defaultsTable4 = tables[6]; - String noDefaultsTable4 = tables[7]; - String noDefaultsTable5 = tables[8]; + String defaultsTable4 = names[6]; + String noDefaultsTable4 = names[7]; + String noDefaultsTable5 = names[8]; testDefaultIterConflict(IllegalArgumentException.class, () -> tops.create(defaultsTable4, new NewTableConfiguration().attachIterator(defaultIterPrioConflict)), @@ -422,57 +504,59 @@ public void testDefaultIterConflict() throws Throwable { () -> tops.create(noDefaultsTable4, new NewTableConfiguration().attachIterator(defaultIterPrioConflict).withoutDefaults()), () -> tops.create(noDefaultsTable5, - new NewTableConfiguration().attachIterator(defaultIterNameConflict).withoutDefaults())); + new NewTableConfiguration().attachIterator(defaultIterNameConflict).withoutDefaults()), + true); // testing TableOperations.attachIterator - String defaultsTable6 = tables[9]; + String defaultsTable6 = names[9]; tops.create(defaultsTable6); - String noDefaultsTable6 = tables[10]; + String noDefaultsTable6 = names[10]; tops.create(noDefaultsTable6, new NewTableConfiguration().withoutDefaults()); testDefaultIterConflict(AccumuloException.class, () -> tops.attachIterator(defaultsTable6, defaultIterPrioConflict), () -> tops.attachIterator(defaultsTable6, defaultIterNameConflict), () -> tops.attachIterator(noDefaultsTable6, defaultIterPrioConflict), - () -> tops.attachIterator(noDefaultsTable6, defaultIterNameConflict)); + () -> tops.attachIterator(noDefaultsTable6, defaultIterNameConflict), true); // testing NamespaceOperations.attachIterator - String ns7 = tables[11]; + String ns7 = names[11]; nops.create(ns7); - String defaultsTable7 = ns7 + "." + tables[12]; + String defaultsTable7 = ns7 + "." + names[12]; tops.create(defaultsTable7); - String ns8 = tables[13]; + String ns8 = names[13]; nops.create(ns8); - String noDefaultsTable8 = ns8 + "." + tables[14]; + String noDefaultsTable8 = ns8 + "." + names[14]; tops.create(noDefaultsTable8, new NewTableConfiguration().withoutDefaults()); testDefaultIterConflict(AccumuloException.class, () -> nops.attachIterator(ns7, defaultIterPrioConflict), () -> nops.attachIterator(ns7, defaultIterNameConflict), () -> nops.attachIterator(ns8, defaultIterPrioConflict), - () -> nops.attachIterator(ns8, defaultIterNameConflict)); + () -> nops.attachIterator(ns8, defaultIterNameConflict), false); // testing NamespaceOperations.setProperty - String ns9 = tables[15]; + String ns9 = names[15]; nops.create(ns9); - String defaultsTable9 = ns9 + "." + tables[16]; + String defaultsTable9 = ns9 + "." + names[16]; tops.create(defaultsTable9); - String ns10 = tables[17]; + String ns10 = names[17]; nops.create(ns10); - String noDefaultsTable10 = ns10 + "." + tables[18]; + String noDefaultsTable10 = ns10 + "." + names[18]; tops.create(noDefaultsTable10, new NewTableConfiguration().withoutDefaults()); testDefaultIterConflict(AccumuloException.class, () -> nops.setProperty(ns9, defaultIterPrioConflictKey, defaultIterPrioConflictVal), () -> nops.setProperty(ns9, defaultIterNameConflictKey, defaultIterNameConflictVal), () -> nops.setProperty(ns10, defaultIterPrioConflictKey, defaultIterPrioConflictVal), - () -> nops.setProperty(ns10, defaultIterNameConflictKey, defaultIterNameConflictVal)); + () -> nops.setProperty(ns10, defaultIterNameConflictKey, defaultIterNameConflictVal), + false); // testing NamespaceOperations.modifyProperties - String ns11 = tables[19]; + String ns11 = names[19]; nops.create(ns11); - String defaultsTable11 = ns11 + "." + tables[20]; + String defaultsTable11 = ns11 + "." + names[20]; tops.create(defaultsTable11); - String ns12 = tables[21]; + String ns12 = names[21]; nops.create(ns12); - String noDefaultsTable12 = ns12 + "." + tables[22]; + String noDefaultsTable12 = ns12 + "." + names[22]; tops.create(noDefaultsTable12, new NewTableConfiguration().withoutDefaults()); testDefaultIterConflict(AccumuloException.class, () -> nops.modifyProperties(ns11, @@ -482,14 +566,17 @@ public void testDefaultIterConflict() throws Throwable { () -> nops.modifyProperties(ns12, props -> props.put(defaultIterPrioConflictKey, defaultIterPrioConflictVal)), () -> nops.modifyProperties(ns12, - props -> props.put(defaultIterNameConflictKey, defaultIterNameConflictVal))); + props -> props.put(defaultIterNameConflictKey, defaultIterNameConflictVal)), + false); // testing CloneConfiguration.Builder.setPropertiesToSet - String dst1 = tables[23]; - String dst2 = tables[24]; - String defaultsTable12 = tables[25]; + String dst1 = names[23]; + String dst2 = names[24]; + String dst3 = names[25]; + String dst4 = names[26]; + String defaultsTable12 = names[27]; tops.create(defaultsTable12); - String noDefaultsTable13 = tables[26]; + String noDefaultsTable13 = names[28]; tops.create(noDefaultsTable13, new NewTableConfiguration().withoutDefaults()); testDefaultIterConflict(AccumuloException.class, () -> tops @@ -499,28 +586,42 @@ public void testDefaultIterConflict() throws Throwable { Map.of(defaultIterPrioConflictKey, defaultIterPrioConflictVal)) .build()), () -> tops - .clone(defaultsTable12, dst1, + .clone(defaultsTable12, dst2, CloneConfiguration.builder() .setPropertiesToSet( Map.of(defaultIterNameConflictKey, defaultIterNameConflictVal)) .build()), () -> tops - .clone(noDefaultsTable13, dst1, + .clone(noDefaultsTable13, dst3, CloneConfiguration.builder() .setPropertiesToSet( Map.of(defaultIterPrioConflictKey, defaultIterPrioConflictVal)) .build()), - () -> tops.clone(noDefaultsTable13, dst2, + () -> tops.clone(noDefaultsTable13, dst4, CloneConfiguration.builder() .setPropertiesToSet(Map.of(defaultIterNameConflictKey, defaultIterNameConflictVal)) - .build())); + .build()), + false); } private void testDefaultIterConflict(Class exceptionClass, Executable defaultsTableOp1, Executable defaultsTableOp2, Executable noDefaultsTableOp1, - Executable noDefaultsTableOp2) throws Throwable { - assertThrows(exceptionClass, defaultsTableOp1); - assertThrows(exceptionClass, defaultsTableOp2); + Executable noDefaultsTableOp2, boolean shouldThrow) throws Throwable { + if (shouldThrow) { + var e = assertThrows(exceptionClass, defaultsTableOp1); + assertTrue(e.toString().contains("conflict with default table iterator") + || e.toString().contains("iterator priority conflict")); + e = assertThrows(exceptionClass, defaultsTableOp2); + assertTrue(e.toString().contains("conflict with default table iterator") + || e.toString().contains("iterator name conflict")); + } else { + assertTrue( + logsContain(List.of("conflict with default table iterator", "iterator priority conflict"), + defaultsTableOp1)); + assertTrue( + logsContain(List.of("conflict with default table iterator", "iterator name conflict"), + defaultsTableOp2)); + } noDefaultsTableOp1.execute(); // should NOT fail noDefaultsTableOp2.execute(); // should NOT fail @@ -636,4 +737,80 @@ private void testSameIterNoConflict(Executable addIter1Executable, addIter1Executable.execute(); addDefaultIterExecutable.execute(); } + + private static boolean logsContain(List expectedStrs, Executable exec) throws Throwable { + var timeBeforeExec = LocalDateTime.now(); + var timeBeforeExecMillis = System.currentTimeMillis(); + exec.execute(); + + // check the logs from other processes for a log that occurred after the execution and + // contains one of the expected strings + List warnLogsAfterExec = warnLogsAfter(timeBeforeExec); + for (var warnLog : warnLogsAfterExec) { + if (expectedStrs.stream().anyMatch(warnLog::contains)) { + return true; + } + } + + // check the logs from the test process (this process) for a log that occurred after the + // execution and contains one of the expected strings + return appender.events().stream() + .anyMatch(logEvent -> logEvent.getTimeMillis() > timeBeforeExecMillis && expectedStrs + .stream().anyMatch(logEvent.getMessage().getFormattedMessage()::contains)); + } + + private static String getDatePattern() { + String datePattern = null; + for (var appender : loggerConfig.getAppenders().values()) { + if (appender.getLayout() instanceof PatternLayout) { + PatternLayout layout = (PatternLayout) appender.getLayout(); + String pattern = layout.getConversionPattern(); + if (pattern.contains("%d{ISO8601}")) { + datePattern = "yyyy-MM-dd'T'HH:mm:ss,SSS"; + break; + } + } + } + assertNotNull(datePattern, + "Format of dates in log4j config has changed. This test needs to be updated"); + return datePattern; + } + + private static List warnLogsAfter(LocalDateTime timeBeforeExec) throws Exception { + var filesIter = getCluster().getFileSystem() + .listFiles(new Path(getCluster().getConfig().getLogDir().toURI()), false); + List files = new ArrayList<>(); + List lines = new ArrayList<>(); + + // get all the WARN logs in the Manager and TabletServer logs that happened after the given + // time. We only care about the Manager and TabletServer as these are the only servers that + // will check for iterator conflicts + while (filesIter.hasNext()) { + var file = filesIter.next(); + if (file.getPath().getName().matches("(Manager_|TabletServer_).+\\.out")) { + files.add(file.getPath()); + } + } + for (var path : files) { + try (var in = getCluster().getFileSystem().open(path); BufferedReader reader = + new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8))) { + String line; + while ((line = reader.readLine()) != null) { + if (line.contains("WARN")) { + var words = line.split(" "); + try { + if (words.length >= 1 + && LocalDateTime.parse(words[0], dateTimeFormatter).isAfter(timeBeforeExec)) { + lines.add(line); + } + } catch (DateTimeParseException e) { + // ignore + } + } + } + } + } + + return lines; + } } From 8774df5c7e286a3ff933cd4eba3e1788b4ac00c8 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Thu, 22 Jan 2026 16:56:41 -0500 Subject: [PATCH 12/21] fix build failure: log4j-core and log4j-api deps added to test --- test/pom.xml | 8 ++++++++ .../accumulo/test/functional/IteratorConflictsIT.java | 6 +++--- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/test/pom.xml b/test/pom.xml index 4430f6d929e..30f5cf3d3c9 100644 --- a/test/pom.xml +++ b/test/pom.xml @@ -182,6 +182,14 @@ org.apache.logging.log4j log4j-1.2-api + + org.apache.logging.log4j + log4j-api + + + org.apache.logging.log4j + log4j-core + org.apache.thrift libthrift diff --git a/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java index dc8eed5d60f..514305e2955 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java @@ -18,13 +18,13 @@ */ package org.apache.accumulo.test.functional; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.BufferedReader; import java.io.InputStreamReader; -import java.nio.charset.StandardCharsets; import java.time.LocalDateTime; import java.time.format.DateTimeFormatter; import java.time.format.DateTimeParseException; @@ -792,8 +792,8 @@ private static List warnLogsAfter(LocalDateTime timeBeforeExec) throws E } } for (var path : files) { - try (var in = getCluster().getFileSystem().open(path); BufferedReader reader = - new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8))) { + try (var in = getCluster().getFileSystem().open(path); + BufferedReader reader = new BufferedReader(new InputStreamReader(in, UTF_8))) { String line; while ((line = reader.readLine()) != null) { if (line.contains("WARN")) { From f72d9a4834d4343efccc68658780f2fde5f14b02 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Fri, 23 Jan 2026 09:56:55 -0500 Subject: [PATCH 13/21] Address review: - Avoid two look up for some comparisons - Fix some bugs with the iterator conflict checks done on clone table Co-authored-by: Keith Turner --- .../iteratorsImpl/IteratorConfigUtil.java | 11 +++++---- .../accumulo/manager/FateServiceHandler.java | 23 ++++++++++++------- 2 files changed, 21 insertions(+), 13 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java index 869e09890bf..2c4bbb94f91 100644 --- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java @@ -30,6 +30,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Objects; import java.util.Set; import java.util.TreeMap; import java.util.stream.Collectors; @@ -298,7 +299,7 @@ private static Class> loadClass(boolean useAcc public static void checkIteratorConflicts(Map props, String property, String value) throws AccumuloException { - if (props.containsKey(property) && props.get(property).equals(value)) { + if (Objects.equals(props.get(property), value)) { // setting a property that already exists (i.e., no change) return; } @@ -321,7 +322,7 @@ public static void checkIteratorConflicts(TableOperations tableOps, NamespaceOpe String namespace, String property, String value) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException { var props = noh.getNamespaceProperties(namespace); - if (props.containsKey(property) && props.get(property).equals(value)) { + if (Objects.equals(props.get(property), value)) { // setting a property that already exists (i.e., no change) return; } @@ -451,9 +452,9 @@ public static void checkIteratorConflicts(Map props, IteratorSett var scope = IteratorScope.valueOf(iterOptPropParts[2]); var optKey = iterOptPropParts[iterOptPropParts.length - 1]; var iterName = iterOptPropParts[3]; - if (!existingIters.containsKey(scope) || existingIters.get(scope).stream() - .noneMatch(is -> is.getName().equals(iterName) && is.getOptions().containsKey(optKey) - && is.getOptions().get(optKey).equals(prop.getValue()))) { + if (!existingIters.containsKey(scope) + || existingIters.get(scope).stream().noneMatch(is -> is.getName().equals(iterName) + && Objects.equals(is.getOptions().get(optKey), prop.getValue()))) { String msg = String.format("iterator options conflict for %s : %s=%s", iterToCheck.getName(), prop.getKey(), prop.getValue()); if (shouldThrow) { diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java index d0ee1e9a123..59067d8048d 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java @@ -94,6 +94,7 @@ import org.apache.accumulo.manager.tableOps.tableExport.ExportTable; import org.apache.accumulo.manager.tableOps.tableImport.ImportTable; import org.apache.accumulo.server.client.ClientServiceHandler; +import org.apache.accumulo.server.conf.store.TablePropKey; import org.apache.accumulo.server.manager.state.MergeInfo; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.FSDataOutputStream; @@ -324,20 +325,26 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe // dest table will have the dest namespace props + src table props: need to check provided // options to set for conflicts with this - var srcTableConfigIterProps = - new HashMap<>(manager.getContext().getTableConfiguration(srcTableId) - .getAllPropertiesWithPrefix(Property.TABLE_ITERATOR_PREFIX)); - var srcNamespaceConfigIterProps = - manager.getContext().getNamespaceConfiguration(srcNamespaceId) - .getAllPropertiesWithPrefix(Property.TABLE_ITERATOR_PREFIX); - srcNamespaceConfigIterProps.forEach((k, v) -> srcTableConfigIterProps.remove(k)); var iterProps = new HashMap<>(manager.getContext().getNamespaceConfiguration(namespaceId) .getAllPropertiesWithPrefix(Property.TABLE_ITERATOR_PREFIX)); - iterProps.putAll(srcTableConfigIterProps); + // get only the source table props, not the merged view + var srcTableProps = manager.getContext().getPropStore() + .get(TablePropKey.of(manager.getContext(), srcTableId)).asMap(); + for (Entry entry : options.entrySet()) { if (entry.getKey().startsWith(TableOperationsImpl.PROPERTY_EXCLUDE_PREFIX)) { propertiesToExclude.add( entry.getKey().substring(TableOperationsImpl.PROPERTY_EXCLUDE_PREFIX.length())); + } + } + + // these props will not be cloned + srcTableProps.keySet().removeAll(propertiesToExclude); + // merge src table props into dest namespace props + iterProps.putAll(srcTableProps); + + for (Entry entry : options.entrySet()) { + if (entry.getKey().startsWith(TableOperationsImpl.PROPERTY_EXCLUDE_PREFIX)) { continue; } From d3cad32635dbd07bae4539435f6aa6ca444e26e8 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Fri, 23 Jan 2026 21:42:12 +0000 Subject: [PATCH 14/21] consolidated some duplicate iterator parsing code --- .../iteratorsImpl/IteratorConfigUtil.java | 139 ++++++------------ .../core/iteratorsImpl/IteratorProperty.java | 136 +++++++++++++++++ .../accumulo/manager/FateServiceHandler.java | 18 +-- .../test/functional/IteratorConflictsIT.java | 28 ++-- 4 files changed, 205 insertions(+), 116 deletions(-) create mode 100644 core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java index 2c4bbb94f91..d6a6d3c91b5 100644 --- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.EnumSet; @@ -67,16 +66,6 @@ public class IteratorConfigUtil { public static final Comparator ITER_INFO_COMPARATOR = Comparator.comparingInt(IterInfo::getPriority); - private static final String ITERATOR_PROP_REGEX = - ("^" + Property.TABLE_ITERATOR_PREFIX.getKey() + "(" + Arrays.stream(IteratorScope.values()) - .map(scope -> scope.name().toLowerCase()).collect(Collectors.joining(".|")) + ".)") - .replace(".", "\\.") + "[^.]+$"; - private static final String ITERATOR_PROP_VAL_REGEX = "^[0-9]+,[^,]+$"; - private static final String ITERATOR_PROP_OPT_REGEX = - ("^" + Property.TABLE_ITERATOR_PREFIX.getKey() + "(" - + Arrays.stream(IteratorScope.values()).map(scope -> scope.name().toLowerCase()) - .collect(Collectors.joining(".|")) - + ".)").replace(".", "\\.") + "[^.]+\\.opt\\.[^.]+$"; private static final String WARNING_MSG = ". Iterator was set as requested, but may lead to non-deterministic behavior."; @@ -145,24 +134,15 @@ public static List parseIterConf(IteratorScope scope, List i Map> allOptions, AccumuloConfiguration conf) { Map properties = conf.getAllPropertiesWithPrefix(getProperty(scope)); ArrayList iterators = new ArrayList<>(iters); - final Property scopeProperty = getProperty(scope); - final String scopePropertyKey = scopeProperty.getKey(); for (Entry entry : properties.entrySet()) { - String suffix = entry.getKey().substring(scopePropertyKey.length()); - String[] suffixSplit = suffix.split("\\.", 3); - - if (suffixSplit.length == 1) { - String[] sa = entry.getValue().split(","); - int prio = Integer.parseInt(sa[0]); - String className = sa[1]; - iterators.add(new IterInfo(prio, className, suffixSplit[0])); - } else if (suffixSplit.length == 3 && suffixSplit[1].equals("opt")) { - String iterName = suffixSplit[0]; - String optName = suffixSplit[2]; - allOptions.computeIfAbsent(iterName, k -> new HashMap<>()).put(optName, entry.getValue()); + var iterProp = IteratorProperty.parse(entry.getKey(), entry.getValue()); + if (iterProp.isOption()) { + allOptions.computeIfAbsent(iterProp.getName(), k -> new HashMap<>()) + .put(iterProp.getOptionKey(), iterProp.getOptionValue()); } else { - throw new IllegalArgumentException("Invalid iterator format: " + entry.getKey()); + iterators + .add(new IterInfo(iterProp.getPriority(), iterProp.getClassName(), iterProp.getName())); } } @@ -303,18 +283,10 @@ public static void checkIteratorConflicts(Map props, String prope // setting a property that already exists (i.e., no change) return; } - if (isNonOptionIterProp(property, value)) { - String[] iterPropParts = property.split("\\."); - IteratorScope scope = IteratorScope.valueOf(iterPropParts[2]); - String iterName = iterPropParts[3]; - String[] priorityAndClass; - if ((priorityAndClass = value.split(",")).length == 2) { - // given a single property, the only way for the property to be equivalent to an existing - // iterator is if the existing iterator has no options (opts are set as separate props) - IteratorSetting givenIter = new IteratorSetting(Integer.parseInt(priorityAndClass[0]), - iterName, priorityAndClass[1]); - checkIteratorConflicts(props, givenIter, EnumSet.of(scope), false); - } + + var iterProp = IteratorProperty.parse(property, value); + if (iterProp != null && !iterProp.isOption()) { + checkIteratorConflicts(props, iterProp.toSetting(), EnumSet.of(iterProp.getScope()), false); } } @@ -327,19 +299,11 @@ public static void checkIteratorConflicts(TableOperations tableOps, NamespaceOpe return; } - // checking for conflicts in the namespace - if (isNonOptionIterProp(property, value)) { - String[] iterPropParts = property.split("\\."); - IteratorScope scope = IteratorScope.valueOf(iterPropParts[2]); - String iterName = iterPropParts[3]; - String[] priorityAndClass; - if ((priorityAndClass = value.split(",")).length == 2) { - // given a single property, the only way for the property to be equivalent to an existing - // iterator is if the existing iterator has no options (opts are set as separate props) - IteratorSetting givenIter = new IteratorSetting(Integer.parseInt(priorityAndClass[0]), - iterName, priorityAndClass[1]); - checkIteratorConflicts(props, givenIter, EnumSet.of(scope), false); - } + var iterProp = IteratorProperty.parse(property, value); + if (iterProp != null && !iterProp.isOption()) { + // given a single property, the only way for the property to be equivalent to an existing + // iterator is if the existing iterator has no options (opts are set as separate props) + checkIteratorConflicts(props, iterProp.toSetting(), EnumSet.of(iterProp.getScope()), false); } // checking for conflicts for the tables in the namespace @@ -428,33 +392,29 @@ public static void checkIteratorConflicts(IteratorSetting iterToCheck, public static void checkIteratorConflicts(Map props, IteratorSetting iterToCheck, EnumSet iterScopesToCheck, boolean shouldThrow) throws AccumuloException { // parse the props map - Map> existingIters = - new HashMap<>(IteratorScope.values().length); + Map> iteratorSettings = new HashMap<>(); + Map> existingIters = new HashMap<>(); + for (var prop : props.entrySet()) { - if (isNonOptionIterProp(prop.getKey(), prop.getValue())) { - var propKeyParts = prop.getKey().split("\\."); - var scope = IteratorScope.valueOf(propKeyParts[2]); - var name = propKeyParts[3]; - var propValParts = prop.getValue().split(","); - var priority = Integer.parseInt(propValParts[0]); - var clazz = propValParts[1]; - var existingIter = - new IteratorSetting(priority, name, clazz, gatherIterOpts(prop.getKey(), props)); - existingIters.computeIfAbsent(scope, s -> new ArrayList<>()).add(existingIter); + var iterProp = IteratorProperty.parse(prop.getKey(), prop.getValue()); + if (iterProp != null && !iterProp.isOption() + && iterScopesToCheck.contains(iterProp.getScope())) { + var iterSetting = iterProp.toSetting(); + iteratorSettings.computeIfAbsent(iterProp.getScope(), s -> new HashMap<>()) + .put(iterProp.getName(), iterSetting); + existingIters.computeIfAbsent(iterProp.getScope(), s -> new ArrayList<>()).add(iterSetting); } } // check for conflicts // any iterator option property not part of an existing iterator is an option conflict for (var prop : props.entrySet()) { - if (isOptionIterProp(prop.getKey())) { - var iterOptPropParts = prop.getKey().split("\\."); - var scope = IteratorScope.valueOf(iterOptPropParts[2]); - var optKey = iterOptPropParts[iterOptPropParts.length - 1]; - var iterName = iterOptPropParts[3]; - if (!existingIters.containsKey(scope) - || existingIters.get(scope).stream().noneMatch(is -> is.getName().equals(iterName) - && Objects.equals(is.getOptions().get(optKey), prop.getValue()))) { + var iterProp = IteratorProperty.parse(prop.getKey(), prop.getValue()); + if (iterProp != null && iterProp.isOption() + && iterScopesToCheck.contains(iterProp.getScope())) { + var iterSetting = + iteratorSettings.getOrDefault(iterProp.getScope(), Map.of()).get(iterProp.getName()); + if (iterSetting == null) { String msg = String.format("iterator options conflict for %s : %s=%s", iterToCheck.getName(), prop.getKey(), prop.getValue()); if (shouldThrow) { @@ -462,6 +422,8 @@ public static void checkIteratorConflicts(Map props, IteratorSett } else { log.warn(msg + WARNING_MSG); } + } else { + iterSetting.addOption(iterProp.getOptionKey(), iterProp.getOptionValue()); } } } @@ -469,28 +431,17 @@ public static void checkIteratorConflicts(Map props, IteratorSett checkIteratorConflicts(iterToCheck, iterScopesToCheck, existingIters, shouldThrow); } - /** - * Returns true if the property is an iterator property not including iterator option properties - */ - public static boolean isNonOptionIterProp(String propKey, String propVal) { - return propKey.matches(ITERATOR_PROP_REGEX) && propVal.matches(ITERATOR_PROP_VAL_REGEX); - } - - public static boolean isOptionIterProp(String propKey) { - return propKey.matches(ITERATOR_PROP_OPT_REGEX); - } - - public static boolean isIterProp(String propKey, String propVal) { - return isNonOptionIterProp(propKey, propVal) || isOptionIterProp(propKey); - } - /** * Returns a new map of all the iterator props contained in the given map */ public static Map gatherIteratorProps(Map props) { Map iterProps = new HashMap<>(); - props.entrySet().stream().filter(entry -> isIterProp(entry.getKey(), entry.getValue())) - .forEach(entry -> iterProps.put(entry.getKey(), entry.getValue())); + for (var e : props.entrySet()) { + var iterProp = IteratorProperty.parse(e.getKey(), e.getValue()); + if (iterProp != null) { + iterProps.put(e.getKey(), e.getValue()); + } + } return iterProps; } @@ -498,12 +449,14 @@ public static Map gatherIteratorProps(Map props) { * returns a map of the options associated with the given iterator property key. Options of the * iterator are obtained by searching the given map */ - public static Map gatherIterOpts(String iterPropKey, Map map) { + public static Map gatherIterOpts(IteratorProperty nameProp, + Map map) { Map opts = new HashMap<>(); - for (var iteratorProp : map.entrySet()) { - if (isOptionIterProp(iteratorProp.getKey()) && iteratorProp.getKey().contains(iterPropKey)) { - String[] parts = iteratorProp.getKey().split("\\."); - opts.put(parts[parts.length - 1], iteratorProp.getValue()); + for (var e : map.entrySet()) { + var iterProp = IteratorProperty.parse(e.getKey(), e.getValue()); + if (iterProp != null && iterProp.isOption() && nameProp.getName().equals(iterProp.getName()) + && nameProp.getScope().equals(iterProp.getScope())) { + opts.put(iterProp.getOptionKey(), iterProp.getOptionValue()); } } return opts; diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java new file mode 100644 index 00000000000..c306ef57887 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.accumulo.core.iteratorsImpl; + +import org.apache.accumulo.core.client.IteratorSetting; +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.core.iterators.IteratorUtil; + +import com.google.common.base.Preconditions; + +/** + * Utility for parsing a single iterator key/value property. + */ +public class IteratorProperty { + + private final IteratorUtil.IteratorScope scope; + private final String name; + + private final int priority; + private final String className; + + private final String optionKey; + private final String optionValue; + + public IteratorProperty(String iterName, IteratorUtil.IteratorScope scope, int priority, + String className) { + this.name = iterName; + this.scope = scope; + this.priority = priority; + this.className = className; + this.optionKey = null; + this.optionValue = null; + } + + public IteratorProperty(String iterName, IteratorUtil.IteratorScope scope, String optionName, + String optionValue) { + this.name = iterName; + this.scope = scope; + this.priority = -1; + this.className = null; + this.optionKey = optionName; + this.optionValue = optionValue; + } + + public boolean isOption() { + return optionKey != null; + } + + public String getClassName() { + Preconditions.checkState(!isOption()); + return className; + } + + public String getName() { + return name; + } + + public String getOptionKey() { + Preconditions.checkState(isOption()); + return optionKey; + } + + public String getOptionValue() { + Preconditions.checkState(isOption()); + return optionValue; + } + + public int getPriority() { + Preconditions.checkState(!isOption()); + return priority; + } + + public IteratorUtil.IteratorScope getScope() { + return scope; + } + + /** + * Creates an initial iterator setting without options. + * + * @throws IllegalStateException if {@link #isOption()} returns true + */ + public IteratorSetting toSetting() { + Preconditions.checkState(!isOption()); + return new IteratorSetting(getPriority(), getName(), getClassName()); + } + + private static void check(boolean b, String property, String value) { + if (!b) { + throw new IllegalArgumentException("Illegal iterator property: " + property + "=" + value); + } + } + + /** + * Parses an iterator key value property. + * + * @return parsed iterator property or null if the property does not start with the iterator + * property prefix. + * @throws IllegalArgumentException if the iterator property is malformed. + */ + public static IteratorProperty parse(String property, String value) { + if (!property.startsWith(Property.TABLE_ITERATOR_PREFIX.getKey())) { + return null; + } + + String[] iterPropParts = property.split("\\."); + IteratorUtil.IteratorScope scope = IteratorUtil.IteratorScope.valueOf(iterPropParts[2]); + String iterName = iterPropParts[3]; + + if (iterPropParts.length == 4) { + String[] valTokens = value.split(","); + check(valTokens.length == 2, property, value); + return new IteratorProperty(iterName, scope, Integer.parseInt(valTokens[0]), valTokens[1]); + } else if (iterPropParts.length == 6) { + check(iterPropParts[4].equals("opt"), property, value); + return new IteratorProperty(iterName, scope, iterPropParts[5], value); + } else { + throw new IllegalArgumentException("Illegal iterator property: " + property + "=" + value); + } + } +} diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java index 59067d8048d..8f840bba669 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java @@ -46,7 +46,6 @@ import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; -import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.NamespaceNotFoundException; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.admin.CompactionConfig; @@ -65,8 +64,8 @@ import org.apache.accumulo.core.data.NamespaceId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus; -import org.apache.accumulo.core.iterators.IteratorUtil; import org.apache.accumulo.core.iteratorsImpl.IteratorConfigUtil; +import org.apache.accumulo.core.iteratorsImpl.IteratorProperty; import org.apache.accumulo.core.manager.state.tables.TableState; import org.apache.accumulo.core.manager.thrift.FateOperation; import org.apache.accumulo.core.manager.thrift.FateService; @@ -870,17 +869,12 @@ private void validateTableProperty(String propKey, String propVal, Map opts = IteratorConfigUtil.gatherIterOpts(propKey, propMap); - var is = new IteratorSetting(Integer.parseInt(iterPropValParts[0]), iterName, - iterPropValParts[1], opts); + var iterProp = IteratorProperty.parse(propKey, propVal); + if (iterProp != null && !iterProp.isOption()) { + Map opts = IteratorConfigUtil.gatherIterOpts(iterProp, propMap); try { - IteratorConfigUtil.checkIteratorConflicts(config, is, EnumSet.of(iterScope), false); + IteratorConfigUtil.checkIteratorConflicts(config, iterProp.toSetting(), + EnumSet.of(iterProp.getScope()), false); } catch (AccumuloException e) { throw new ThriftTableOperationException(null, tableName, tableOp, TableOperationExceptionType.OTHER, e.getMessage()); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java index 514305e2955..4dd6b23c586 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java @@ -19,6 +19,7 @@ package org.apache.accumulo.test.functional; import static java.nio.charset.StandardCharsets.UTF_8; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -32,6 +33,7 @@ import java.util.EnumSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; import java.util.function.Consumer; @@ -45,7 +47,7 @@ import org.apache.accumulo.core.client.admin.NewTableConfiguration; import org.apache.accumulo.core.client.admin.TableOperations; import org.apache.accumulo.core.conf.Property; -import org.apache.accumulo.core.iterators.IteratorUtil; +import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; import org.apache.accumulo.core.iteratorsImpl.IteratorConfigUtil; import org.apache.accumulo.harness.SharedMiniClusterBase; import org.apache.hadoop.fs.Path; @@ -102,17 +104,17 @@ public class IteratorConflictsIT extends SharedMiniClusterBase { private static final String iterClass = SlowIterator.class.getName(); private static final IteratorSetting iter1 = new IteratorSetting(99, "iter1name", iterClass); private static final String iter1Key = Property.TABLE_ITERATOR_PREFIX - + IteratorUtil.IteratorScope.scan.name().toLowerCase() + "." + iter1.getName(); + + IteratorScope.scan.name().toLowerCase() + "." + iter1.getName(); private static final String iter1Val = "99," + iterClass; private static final IteratorSetting iter1PrioConflict = new IteratorSetting(99, "othername", iterClass); private static final IteratorSetting iter1NameConflict = new IteratorSetting(101, iter1.getName(), iterClass); - private static final String iter1PrioConflictKey = Property.TABLE_ITERATOR_PREFIX - + IteratorUtil.IteratorScope.scan.name().toLowerCase() + ".othername"; + private static final String iter1PrioConflictKey = + Property.TABLE_ITERATOR_PREFIX + IteratorScope.scan.name().toLowerCase() + ".othername"; private static final String iter1PrioConflictVal = "99," + iterClass; private static final String iter1NameConflictKey = Property.TABLE_ITERATOR_PREFIX - + IteratorUtil.IteratorScope.scan.name().toLowerCase() + "." + iter1.getName(); + + IteratorScope.scan.name().toLowerCase() + "." + iter1.getName(); private static final String iter1NameConflictVal = "101," + iterClass; private static final IteratorSetting defaultIterPrioConflict = new IteratorSetting(20, "bar", iterClass); @@ -120,20 +122,20 @@ public class IteratorConflictsIT extends SharedMiniClusterBase { new IteratorSetting(101, "vers", iterClass); private static final IteratorSetting defaultTableIter = IteratorConfigUtil.getInitialTableIteratorSettings().keySet().iterator().next(); - private static final String defaultIterPrioConflictKey = Property.TABLE_ITERATOR_PREFIX - + IteratorUtil.IteratorScope.scan.name().toLowerCase() + ".foo"; + private static final String defaultIterPrioConflictKey = + Property.TABLE_ITERATOR_PREFIX + IteratorScope.scan.name().toLowerCase() + ".foo"; private static final String defaultIterPrioConflictVal = defaultTableIter.getPriority() + "," + iterClass; private static final String defaultIterNameConflictKey = Property.TABLE_ITERATOR_PREFIX - + IteratorUtil.IteratorScope.scan.name().toLowerCase() + "." + defaultTableIter.getName(); + + IteratorScope.scan.name().toLowerCase() + "." + defaultTableIter.getName(); private static final String defaultIterNameConflictVal = "99," + iterClass; private static final String defaultIterKey = Property.TABLE_ITERATOR_PREFIX.getKey() - + IteratorUtil.IteratorScope.scan.name().toLowerCase() + "." + defaultTableIter.getName(); + + IteratorScope.scan.name().toLowerCase() + "." + defaultTableIter.getName(); private static final String defaultIterVal = defaultTableIter.getPriority() + "," + defaultTableIter.getIteratorClass(); private static final String defaultIterOptKey = Property.TABLE_ITERATOR_PREFIX.getKey() - + IteratorUtil.IteratorScope.scan.name().toLowerCase() + "." + defaultTableIter.getName() - + ".opt." + defaultTableIter.getOptions().entrySet().iterator().next().getKey(); + + IteratorScope.scan.name().toLowerCase() + "." + defaultTableIter.getName() + ".opt." + + defaultTableIter.getOptions().entrySet().iterator().next().getKey(); private static final String defaultIterOptVal = defaultTableIter.getOptions().entrySet().iterator().next().getValue(); @@ -426,6 +428,10 @@ private void testNamespaceIterConflict(String ns, Class assertTrue(e.toString().contains("iterator priority conflict")); e = assertThrows(exceptionClass, iterNameConflictExec); assertTrue(e.toString().contains("iterator name conflict")); + assertEquals(Set.of(iter1.getName()), nops.listIterators(ns).keySet()); + for (var scope : IteratorScope.values()) { + assertEquals(iter1, nops.getIteratorSetting(ns, iter1.getName(), scope)); + } } else { assertTrue(logsContain(List.of("iterator priority conflict"), iterPrioConflictExec)); assertTrue(logsContain(List.of("iterator name conflict"), iterNameConflictExec)); From f4925dbaec9438cc1fe0f6e298cafb6d04cd7f89 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Fri, 23 Jan 2026 21:59:44 +0000 Subject: [PATCH 15/21] fix bug --- .../org/apache/accumulo/manager/FateServiceHandler.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java index 8f840bba669..aa053b01b72 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java @@ -873,8 +873,10 @@ private void validateTableProperty(String propKey, String propVal, Map opts = IteratorConfigUtil.gatherIterOpts(iterProp, propMap); try { - IteratorConfigUtil.checkIteratorConflicts(config, iterProp.toSetting(), - EnumSet.of(iterProp.getScope()), false); + var is = iterProp.toSetting(); + is.addOptions(opts); + IteratorConfigUtil.checkIteratorConflicts(config, is, EnumSet.of(iterProp.getScope()), + false); } catch (AccumuloException e) { throw new ThriftTableOperationException(null, tableName, tableOp, TableOperationExceptionType.OTHER, e.getMessage()); From 4a4e5c2720986ca8a992460c967e16c6f07f2176 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Fri, 23 Jan 2026 22:09:53 +0000 Subject: [PATCH 16/21] improve parsing --- .../org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java index c306ef57887..eaae95c247e 100644 --- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java +++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java @@ -119,6 +119,7 @@ public static IteratorProperty parse(String property, String value) { } String[] iterPropParts = property.split("\\."); + check(iterPropParts.length >= 4, property, value); IteratorUtil.IteratorScope scope = IteratorUtil.IteratorScope.valueOf(iterPropParts[2]); String iterName = iterPropParts[3]; From 201318fb704174eb8699c4be88c880992069a94f Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Fri, 23 Jan 2026 22:18:25 +0000 Subject: [PATCH 17/21] make constructors private --- .../apache/accumulo/core/iteratorsImpl/IteratorProperty.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java index eaae95c247e..30db439f043 100644 --- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java +++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java @@ -38,7 +38,7 @@ public class IteratorProperty { private final String optionKey; private final String optionValue; - public IteratorProperty(String iterName, IteratorUtil.IteratorScope scope, int priority, + private IteratorProperty(String iterName, IteratorUtil.IteratorScope scope, int priority, String className) { this.name = iterName; this.scope = scope; @@ -48,7 +48,7 @@ public IteratorProperty(String iterName, IteratorUtil.IteratorScope scope, int p this.optionValue = null; } - public IteratorProperty(String iterName, IteratorUtil.IteratorScope scope, String optionName, + private IteratorProperty(String iterName, IteratorUtil.IteratorScope scope, String optionName, String optionValue) { this.name = iterName; this.scope = scope; From 7ba91a2a21d486f2b6c7a407e8f23bd73182f84d Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Mon, 26 Jan 2026 08:39:58 -0800 Subject: [PATCH 18/21] Update core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java Co-authored-by: Kevin Rathbun --- .../apache/accumulo/core/iteratorsImpl/IteratorProperty.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java index 30db439f043..1e46dd35e7e 100644 --- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java +++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java @@ -119,7 +119,7 @@ public static IteratorProperty parse(String property, String value) { } String[] iterPropParts = property.split("\\."); - check(iterPropParts.length >= 4, property, value); + check(iterPropParts.length == 4 || iterPropParts.length == 6, property, value); IteratorUtil.IteratorScope scope = IteratorUtil.IteratorScope.valueOf(iterPropParts[2]); String iterName = iterPropParts[3]; From 88a52b429ccd06cfe8b5072b0d6f9665bf8c4d54 Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Mon, 26 Jan 2026 08:42:41 -0800 Subject: [PATCH 19/21] Update core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java Co-authored-by: Kevin Rathbun --- .../apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java index d6a6d3c91b5..1ac3adcdf60 100644 --- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java @@ -286,6 +286,8 @@ public static void checkIteratorConflicts(Map props, String prope var iterProp = IteratorProperty.parse(property, value); if (iterProp != null && !iterProp.isOption()) { + // given a single property, the only way for the property to be equivalent to an existing + // iterator is if the existing iterator has no options (opts are set as separate props) checkIteratorConflicts(props, iterProp.toSetting(), EnumSet.of(iterProp.getScope()), false); } } From 8e4273e15f1c40bbba12f24147e1b8eda8cc44fe Mon Sep 17 00:00:00 2001 From: Keith Turner Date: Mon, 26 Jan 2026 16:58:38 +0000 Subject: [PATCH 20/21] code review update --- .../apache/accumulo/core/iteratorsImpl/IteratorProperty.java | 3 ++- .../apache/accumulo/test/functional/IteratorConflictsIT.java | 4 ++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java index 1e46dd35e7e..d9812fb3084 100644 --- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java +++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorProperty.java @@ -111,7 +111,8 @@ private static void check(boolean b, String property, String value) { * * @return parsed iterator property or null if the property does not start with the iterator * property prefix. - * @throws IllegalArgumentException if the iterator property is malformed. + * @throws RuntimeException if the iterator property is malformed. Will actually throw a few + * different subclasses of this exception. */ public static IteratorProperty parse(String property, String value) { if (!property.startsWith(Property.TABLE_ITERATOR_PREFIX.getKey())) { diff --git a/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java index 4dd6b23c586..7317405befa 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java @@ -278,6 +278,10 @@ private void testTableIterConflict(String table, Class assertTrue(e.toString().contains("iterator priority conflict")); e = assertThrows(exceptionClass, iterNameConflictExec); assertTrue(e.toString().contains("iterator name conflict")); + assertEquals(Set.of(iter1.getName(), "vers"), tops.listIterators(table).keySet()); + for (var scope : IteratorScope.values()) { + assertEquals(iter1, tops.getIteratorSetting(table, iter1.getName(), scope)); + } } else { assertTrue(logsContain(List.of("iterator priority conflict"), iterPrioConflictExec)); assertTrue(logsContain(List.of("iterator name conflict"), iterNameConflictExec)); From 3244196cc94562658760003627c1b5768067b623 Mon Sep 17 00:00:00 2001 From: Kevin Rathbun Date: Mon, 26 Jan 2026 13:02:06 -0500 Subject: [PATCH 21/21] address review --- .../test/functional/IteratorConflictsIT.java | 45 +++++++------------ 1 file changed, 17 insertions(+), 28 deletions(-) diff --git a/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java index 7317405befa..0cc56b2237d 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/IteratorConflictsIT.java @@ -28,7 +28,6 @@ import java.io.InputStreamReader; import java.time.LocalDateTime; import java.time.format.DateTimeFormatter; -import java.time.format.DateTimeParseException; import java.util.ArrayList; import java.util.EnumSet; import java.util.List; @@ -50,6 +49,7 @@ import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; import org.apache.accumulo.core.iteratorsImpl.IteratorConfigUtil; import org.apache.accumulo.harness.SharedMiniClusterBase; +import org.apache.accumulo.test.util.Wait; import org.apache.hadoop.fs.Path; import org.apache.logging.log4j.Level; import org.apache.logging.log4j.LogManager; @@ -177,11 +177,11 @@ public static void startup() throws Exception { @AfterAll public static void shutdown() throws Exception { - client.close(); - SharedMiniClusterBase.stopMiniCluster(); loggerConfig.getRootLogger().removeAppender(appender.getName()); appender.stop(); loggerContext.updateLoggers(); + client.close(); + SharedMiniClusterBase.stopMiniCluster(); } @Test @@ -202,7 +202,7 @@ public void testTableIterConflict() throws Throwable { nops.create(ns); } for (String table : List.of(table1, table2, table3, table4, table5, table6, table7, table8)) { - tops.create(table); + tops.create(table, new NewTableConfiguration().attachIterator(iter1)); } // testing Scanner.addScanIterator @@ -272,7 +272,6 @@ public void testTableIterConflict() throws Throwable { private void testTableIterConflict(String table, Class exceptionClass, Executable iterPrioConflictExec, Executable iterNameConflictExec, boolean shouldThrow) throws Throwable { - tops.attachIterator(table, iter1); if (shouldThrow) { var e = assertThrows(exceptionClass, iterPrioConflictExec); assertTrue(e.toString().contains("iterator priority conflict")); @@ -426,6 +425,7 @@ private void testNamespaceIterConflict(String ns, Class Executable iterPrioConflictExec, Executable iterNameConflictExec, boolean shouldThrow) throws Throwable { nops.attachIterator(ns, iter1); + Wait.waitFor(() -> nops.listIterators(ns).containsKey(iter1.getName())); if (shouldThrow) { var e = assertThrows(exceptionClass, iterPrioConflictExec); @@ -648,8 +648,7 @@ public void testSameIterNoConflict() throws Throwable { // testing Scanner.addScanIterator final String table1 = names[0]; - tops.create(table1); - tops.attachIterator(table1, iter1); + tops.create(table1, new NewTableConfiguration().attachIterator(iter1)); try (var scanner1 = client.createScanner(table1); var scanner2 = client.createScanner(table1)) { testSameIterNoConflict(() -> { scanner1.addScanIterator(iter1); @@ -662,15 +661,13 @@ public void testSameIterNoConflict() throws Throwable { // testing TableOperations.setProperty final String table2 = names[1]; - tops.create(table2); - tops.attachIterator(table2, iter1); + tops.create(table2, new NewTableConfiguration().attachIterator(iter1)); testSameIterNoConflict(() -> tops.setProperty(table2, iter1Key, iter1Val), () -> tops.setProperty(table2, defaultIterKey, defaultIterVal)); // testing TableOperations.modifyProperties final String table3 = names[2]; - tops.create(table3); - tops.attachIterator(table3, iter1); + tops.create(table3, new NewTableConfiguration().attachIterator(iter1)); testSameIterNoConflict( () -> tops.modifyProperties(table3, props -> props.put(iter1Key, iter1Val)), () -> tops.modifyProperties(table3, props -> { @@ -684,14 +681,14 @@ public void testSameIterNoConflict() throws Throwable { final String table5 = names[5]; nops.create(ns1); nops.attachIterator(ns1, iter1); + Wait.waitFor(() -> nops.listIterators(ns1).containsKey(iter1.getName())); testSameIterNoConflict( () -> tops.create(table4, new NewTableConfiguration().attachIterator(iter1)), () -> tops.create(table5, new NewTableConfiguration().attachIterator(defaultTableIter))); // testing TableOperations.attachIterator final String table6 = names[6]; - tops.create(table6); - tops.attachIterator(table6, iter1); + tops.create(table6, new NewTableConfiguration().attachIterator(iter1)); testSameIterNoConflict(() -> tops.attachIterator(table6, iter1), () -> tops.attachIterator(table6, defaultTableIter)); @@ -699,8 +696,7 @@ public void testSameIterNoConflict() throws Throwable { final String ns2 = names[7]; final String table7 = ns2 + "." + names[8]; nops.create(ns2); - tops.create(table7); - tops.attachIterator(table7, iter1); + tops.create(table7, new NewTableConfiguration().attachIterator(iter1)); testSameIterNoConflict(() -> nops.attachIterator(ns2, iter1), () -> nops.attachIterator(ns2, defaultTableIter)); @@ -708,8 +704,7 @@ public void testSameIterNoConflict() throws Throwable { final String ns3 = names[9]; final String table8 = ns3 + "." + names[10]; nops.create(ns3); - tops.create(table8); - tops.attachIterator(table8, iter1); + tops.create(table8, new NewTableConfiguration().attachIterator(iter1)); testSameIterNoConflict(() -> nops.setProperty(ns3, iter1Key, iter1Val), () -> nops.setProperty(ns3, defaultIterKey, defaultIterVal)); @@ -717,8 +712,7 @@ public void testSameIterNoConflict() throws Throwable { final String ns4 = names[11]; final String table9 = ns4 + "." + names[12]; nops.create(ns4); - tops.create(table9); - tops.attachIterator(table9, iter1); + tops.create(table9, new NewTableConfiguration().attachIterator(iter1)); testSameIterNoConflict(() -> nops.modifyProperties(ns4, props -> props.put(iter1Key, iter1Val)), () -> nops.modifyProperties(ns4, props -> { props.put(defaultIterKey, defaultIterVal); @@ -729,8 +723,7 @@ public void testSameIterNoConflict() throws Throwable { final String src = names[13]; final String dst1 = names[14]; final String dst2 = names[15]; - tops.create(src); - tops.attachIterator(src, iter1); + tops.create(src, new NewTableConfiguration().attachIterator(iter1)); testSameIterNoConflict( () -> tops.clone(src, dst1, CloneConfiguration.builder().setPropertiesToSet(Map.of(iter1Key, iter1Val)).build()), @@ -808,13 +801,9 @@ private static List warnLogsAfter(LocalDateTime timeBeforeExec) throws E while ((line = reader.readLine()) != null) { if (line.contains("WARN")) { var words = line.split(" "); - try { - if (words.length >= 1 - && LocalDateTime.parse(words[0], dateTimeFormatter).isAfter(timeBeforeExec)) { - lines.add(line); - } - } catch (DateTimeParseException e) { - // ignore + if (words.length >= 1 + && LocalDateTime.parse(words[0], dateTimeFormatter).isAfter(timeBeforeExec)) { + lines.add(line); } } }