Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -198,30 +198,31 @@ public Map<String,String> 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
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);
Comment on lines -207 to +205
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I could remove noDefaultsPropMap since I pushed the check for equality into checkIteratorConflicts

} catch (AccumuloException e) {
throw new IllegalStateException(String.format(
throw new IllegalArgumentException(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(
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);
}

return Collections.unmodifiableMap(propertyMap);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 {

Expand Down Expand Up @@ -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<String,String> optionConflicts = new TreeMap<>();
for (Entry<String,String> 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());
}
Comment on lines -177 to -180
Copy link
Contributor

@keith-turner keith-turner Jan 16, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems like the new code is not translating this NumberFormatException, is this change in exceptions going to ripple through to the client API? When we parse the props to IteratorSetting we could maintain these exceptions in the new code.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah this does change the exceptions the client might receive from this. They will always receive an AccumuloException before and after these changes, but they no longer receive AccumuloExceptions for two things: AccumuloExceptions if parts.length != 2 or the priority isn't a number. With my changes, I check properties against a regex (see new static vars in IteratorConfigUtil), ignoring those that don't match. So if parts.length != 2 or the priority isn't a number, it is quietly ignored as not being an iterator-related property.

I can add back AccumuloExceptions for these two cases if you think that is best.

}
}
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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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));
Expand Down Expand Up @@ -215,6 +218,11 @@ private Map<String,String> 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,
Expand Down Expand Up @@ -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);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 {

Expand Down Expand Up @@ -139,45 +140,7 @@ public static void checkIteratorConflicts(Map<String,String> props, IteratorSett
EnumSet<IteratorScope> 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);
Map<String,String> optionConflicts = new TreeMap<>();
for (Entry<String,String> 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));
}
}
IteratorConfigUtil.checkIteratorConflicts(props, setting, scopes);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -1008,10 +1009,13 @@ public void setProperty(final String tableName, final String property, final Str
checkArgument(value != null, "value is null");

try {
IteratorConfigUtil.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);
}
}
Expand All @@ -1022,6 +1026,13 @@ private Map<String,String> tryToModifyProperties(String tableName,
final TVersionedProperties vProperties =
ThriftClientTypes.CLIENT.execute(context, client -> client
.getVersionedTableProperties(TraceUtil.traceInfo(), context.rpcCreds(), tableName));
final Map<String,String> 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
Expand All @@ -1030,6 +1041,15 @@ private Map<String,String> 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);
}

try {
// Send to server
ThriftClientTypes.MANAGER.executeVoid(context,
Expand Down
Loading