diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/ClientIteratorEnvironment.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/ClientIteratorEnvironment.java index d0e25957a42..a4778441eac 100644 --- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/ClientIteratorEnvironment.java +++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/ClientIteratorEnvironment.java @@ -46,10 +46,10 @@ public static class Builder { private boolean isFullMajorCompaction = false; private Optional auths = Optional.empty(); private boolean isUserCompaction = false; - private Optional tableId = Optional.empty(); - private Optional samplerConfig = Optional.empty(); + protected Optional tableId = Optional.empty(); + protected Optional samplerConfig = Optional.empty(); private boolean samplingEnabled = false; - protected Optional env = Optional.empty(); + protected Optional env = Optional.empty(); public Builder withScope(IteratorScope scope) { checkState(this.scope.isEmpty(), "Scope has already been set"); @@ -115,11 +115,11 @@ public ClientIteratorEnvironment build() { private final Optional auths; private final boolean isUserCompaction; private final Optional tableId; - private final Optional samplerConfig; + protected Optional samplerConfig; private final boolean samplingEnabled; - private final Optional env; + private final Optional env; - private ClientIteratorEnvironment(Builder builder) { + protected ClientIteratorEnvironment(Builder builder) { this.scope = builder.scope; this.isFullMajorCompaction = builder.isFullMajorCompaction; this.auths = builder.auths; @@ -133,7 +133,7 @@ private ClientIteratorEnvironment(Builder builder) { /** * Copy constructor used for enabling sample. Only called from {@link #cloneWithSamplingEnabled}. */ - private ClientIteratorEnvironment(ClientIteratorEnvironment copy) { + protected ClientIteratorEnvironment(ClientIteratorEnvironment copy) { this.scope = copy.scope; this.isFullMajorCompaction = copy.isFullMajorCompaction; this.auths = copy.auths; @@ -173,7 +173,7 @@ public void registerSideChannel(SortedKeyValueIterator iter) { @Override public Authorizations getAuthorizations() { if (getIteratorScope() != IteratorScope.scan) { - throw new IllegalStateException("Iterator scope is not scan"); + throw new UnsupportedOperationException("Iterator scope is not scan"); } return auths.orElseThrow(); } @@ -201,10 +201,16 @@ public SamplerConfiguration getSamplerConfiguration() { @Override public boolean isUserCompaction() { + // check for scan scope if (getIteratorScope() == IteratorScope.scan) { throw new IllegalStateException( "scan iterator scope is incompatible with a possible user compaction"); } + // check for minc scope + if (getIteratorScope() != IteratorScope.majc) { + throw new IllegalStateException( + "Asked about user initiated compaction type when scope is " + getIteratorScope()); + } return this.isUserCompaction; } diff --git a/server/base/src/main/java/org/apache/accumulo/server/iterators/SystemIteratorEnvironmentImpl.java b/server/base/src/main/java/org/apache/accumulo/server/iterators/SystemIteratorEnvironmentImpl.java new file mode 100644 index 00000000000..2705703ad6f --- /dev/null +++ b/server/base/src/main/java/org/apache/accumulo/server/iterators/SystemIteratorEnvironmentImpl.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.accumulo.server.iterators; + +import java.util.ArrayList; +import java.util.Optional; + +import org.apache.accumulo.core.client.AccumuloClient; +import org.apache.accumulo.core.client.SampleNotPresentException; +import org.apache.accumulo.core.client.sample.SamplerConfiguration; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.iterators.IteratorEnvironment; +import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; +import org.apache.accumulo.core.iterators.SortedKeyValueIterator; +import org.apache.accumulo.core.iteratorsImpl.ClientIteratorEnvironment; +import org.apache.accumulo.core.iteratorsImpl.system.MultiIterator; +import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl; +import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.server.ServiceEnvironmentImpl; + +/** + * IteratorEnvironment used on the server side that has a little different handling for Sampling and + * provides access to the ServerContext and top level iterators. + */ +public class SystemIteratorEnvironmentImpl extends ClientIteratorEnvironment + implements SystemIteratorEnvironment { + + public static class Builder extends ClientIteratorEnvironment.Builder { + + private final ServerContext ctx; + private ArrayList> topLevelIterators = new ArrayList<>(); + + public Builder(ServerContext ctx) { + this.ctx = ctx; + this.env = Optional.of(new ServiceEnvironmentImpl(ctx)); + } + + public Builder + withTopLevelIterators(ArrayList> topLevelIterators) { + this.topLevelIterators = topLevelIterators; + return this; + } + + @Override + public Builder withClient(AccumuloClient client) { + // Does nothing, this was set in constructor + return this; + } + + @Override + public SystemIteratorEnvironmentImpl build() { + return new SystemIteratorEnvironmentImpl(this); + } + + } + + private final ServerContext ctx; + private final ArrayList> topLevelIterators; + + protected SystemIteratorEnvironmentImpl(SystemIteratorEnvironmentImpl.Builder builder) { + super(builder); + this.ctx = builder.ctx; + this.topLevelIterators = builder.topLevelIterators; + } + + private SystemIteratorEnvironmentImpl(SystemIteratorEnvironmentImpl copy) { + super(copy); + this.ctx = copy.ctx; + this.topLevelIterators = copy.topLevelIterators; + } + + @Override + public SamplerConfiguration getSamplerConfiguration() { + // Server-side we take into account the table configuration for sampling + if (samplerConfig == null || samplerConfig.isEmpty()) { + // only create this once so that it stays the same, even if config changes + SamplerConfigurationImpl sci = + SamplerConfigurationImpl.newSamplerConfig(ctx.getTableConfiguration(getTableId())); + if (sci == null) { + return null; + } + samplerConfig = Optional.of(sci.toSamplerConfiguration()); + } + return samplerConfig.orElse(null); + } + + @Override + public IteratorEnvironment cloneWithSamplingEnabled() { + if (!getIteratorScope().equals(IteratorScope.scan)) { + throw new UnsupportedOperationException(); + } + if (getSamplerConfiguration() == null) { + throw new SampleNotPresentException(); + } + return new SystemIteratorEnvironmentImpl(this); + } + + @Override + public ServerContext getServerContext() { + return this.ctx; + } + + @Override + public SortedKeyValueIterator + getTopLevelIterator(SortedKeyValueIterator iter) { + if (topLevelIterators.isEmpty()) { + return iter; + } + ArrayList> allIters = new ArrayList<>(topLevelIterators); + allIters.add(iter); + return new MultiIterator(allIters, false); + } + +} diff --git a/server/base/src/main/java/org/apache/accumulo/server/iterators/TabletIteratorEnvironment.java b/server/base/src/main/java/org/apache/accumulo/server/iterators/TabletIteratorEnvironment.java deleted file mode 100644 index a1b657b5628..00000000000 --- a/server/base/src/main/java/org/apache/accumulo/server/iterators/TabletIteratorEnvironment.java +++ /dev/null @@ -1,269 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * https://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.accumulo.server.iterators; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Map; - -import org.apache.accumulo.core.client.PluginEnvironment; -import org.apache.accumulo.core.client.SampleNotPresentException; -import org.apache.accumulo.core.client.sample.SamplerConfiguration; -import org.apache.accumulo.core.conf.AccumuloConfiguration; -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.TableId; -import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.iterators.IteratorEnvironment; -import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; -import org.apache.accumulo.core.iterators.SortedKeyValueIterator; -import org.apache.accumulo.core.iteratorsImpl.system.MultiIterator; -import org.apache.accumulo.core.metadata.TabletFile; -import org.apache.accumulo.core.metadata.schema.DataFileValue; -import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl; -import org.apache.accumulo.core.security.Authorizations; -import org.apache.accumulo.core.spi.common.ServiceEnvironment; -import org.apache.accumulo.core.spi.compaction.CompactionKind; -import org.apache.accumulo.server.ServerContext; -import org.apache.accumulo.server.ServiceEnvironmentImpl; -import org.apache.accumulo.server.fs.FileManager.ScanFileManager; -import org.apache.hadoop.fs.Path; - -public class TabletIteratorEnvironment implements SystemIteratorEnvironment { - - private final ServerContext context; - private final ServiceEnvironment serviceEnvironment; - private final ScanFileManager trm; - private final IteratorScope scope; - private final boolean fullMajorCompaction; - private boolean userCompaction; - private final AccumuloConfiguration tableConfig; - private final TableId tableId; - private final ArrayList> topLevelIterators; - private Map files; - - private final Authorizations authorizations; // these will only be supplied during scan scope - private SamplerConfiguration samplerConfig; - private boolean enableSampleForDeepCopy; - - public TabletIteratorEnvironment(ServerContext context, IteratorScope scope, - AccumuloConfiguration tableConfig, TableId tableId) { - if (scope == IteratorScope.majc) { - throw new IllegalArgumentException("must set if compaction is full"); - } - - this.context = context; - this.serviceEnvironment = new ServiceEnvironmentImpl(context); - this.scope = scope; - this.trm = null; - this.tableConfig = tableConfig; - this.tableId = tableId; - this.fullMajorCompaction = false; - this.userCompaction = false; - this.authorizations = Authorizations.EMPTY; - this.topLevelIterators = new ArrayList<>(); - } - - public TabletIteratorEnvironment(ServerContext context, IteratorScope scope, - AccumuloConfiguration tableConfig, TableId tableId, SamplerConfigurationImpl samplerConfig) { - if (scope == IteratorScope.majc) { - throw new IllegalArgumentException("must set if compaction is full"); - } - - this.context = context; - this.serviceEnvironment = new ServiceEnvironmentImpl(context); - this.scope = scope; - this.trm = null; - this.tableConfig = tableConfig; - this.tableId = tableId; - this.fullMajorCompaction = false; - this.userCompaction = false; - this.authorizations = Authorizations.EMPTY; - if (samplerConfig != null) { - enableSampleForDeepCopy = true; - this.samplerConfig = samplerConfig.toSamplerConfiguration(); - } else { - enableSampleForDeepCopy = false; - } - this.topLevelIterators = new ArrayList<>(); - } - - public TabletIteratorEnvironment(ServerContext context, IteratorScope scope, - AccumuloConfiguration tableConfig, TableId tableId, ScanFileManager trm, - Map files, Authorizations authorizations, - SamplerConfigurationImpl samplerConfig, - ArrayList> topLevelIterators) { - if (scope == IteratorScope.majc) { - throw new IllegalArgumentException("must set if compaction is full"); - } - - this.context = context; - this.serviceEnvironment = new ServiceEnvironmentImpl(context); - this.scope = scope; - this.trm = trm; - this.tableConfig = tableConfig; - this.tableId = tableId; - this.fullMajorCompaction = false; - this.files = files; - this.authorizations = authorizations; - if (samplerConfig != null) { - enableSampleForDeepCopy = true; - this.samplerConfig = samplerConfig.toSamplerConfiguration(); - } else { - enableSampleForDeepCopy = false; - } - - this.topLevelIterators = topLevelIterators; - } - - public TabletIteratorEnvironment(ServerContext context, IteratorScope scope, boolean fullMajC, - AccumuloConfiguration tableConfig, TableId tableId, CompactionKind kind) { - if (scope != IteratorScope.majc) { - throw new IllegalArgumentException( - "Tried to set maj compaction type when scope was " + scope); - } - - this.context = context; - this.serviceEnvironment = new ServiceEnvironmentImpl(context); - this.scope = scope; - this.trm = null; - this.tableConfig = tableConfig; - this.tableId = tableId; - this.fullMajorCompaction = fullMajC; - this.userCompaction = kind.equals(CompactionKind.USER); - this.authorizations = Authorizations.EMPTY; - this.topLevelIterators = new ArrayList<>(); - } - - @Deprecated(since = "2.0.0") - @Override - public AccumuloConfiguration getConfig() { - return tableConfig; - } - - @Override - public IteratorScope getIteratorScope() { - return scope; - } - - @Override - public boolean isFullMajorCompaction() { - if (scope != IteratorScope.majc) { - throw new IllegalStateException("Asked about major compaction type when scope is " + scope); - } - return fullMajorCompaction; - } - - @Override - public boolean isUserCompaction() { - if (scope != IteratorScope.majc) { - throw new IllegalStateException( - "Asked about user initiated compaction type when scope is " + scope); - } - return userCompaction; - } - - @Deprecated(since = "2.0.0") - @Override - public SortedKeyValueIterator reserveMapFileReader(String mapFileName) - throws IOException { - TabletFile ref = new TabletFile(new Path(mapFileName)); - return trm.openFiles(Collections.singletonMap(ref, files.get(ref)), false, null).get(0); - } - - @Deprecated(since = "2.0.0") - @Override - public void registerSideChannel(SortedKeyValueIterator iter) { - topLevelIterators.add(iter); - } - - @Override - public Authorizations getAuthorizations() { - if (scope != IteratorScope.scan) { - throw new UnsupportedOperationException( - "Authorizations may only be supplied when scope is scan but scope is " + scope); - } - return authorizations; - } - - @Override - public SortedKeyValueIterator - getTopLevelIterator(SortedKeyValueIterator iter) { - if (topLevelIterators.isEmpty()) { - return iter; - } - ArrayList> allIters = new ArrayList<>(topLevelIterators); - allIters.add(iter); - return new MultiIterator(allIters, false); - } - - @Override - public boolean isSamplingEnabled() { - return enableSampleForDeepCopy; - } - - @Override - public SamplerConfiguration getSamplerConfiguration() { - if (samplerConfig == null) { - // only create this once so that it stays the same, even if config changes - SamplerConfigurationImpl sci = SamplerConfigurationImpl.newSamplerConfig(tableConfig); - if (sci == null) { - return null; - } - samplerConfig = sci.toSamplerConfiguration(); - } - return samplerConfig; - } - - @Override - public IteratorEnvironment cloneWithSamplingEnabled() { - if (!scope.equals(IteratorScope.scan)) { - throw new UnsupportedOperationException(); - } - - SamplerConfigurationImpl sci = SamplerConfigurationImpl.newSamplerConfig(tableConfig); - if (sci == null) { - throw new SampleNotPresentException(); - } - - return new TabletIteratorEnvironment(context, scope, tableConfig, tableId, trm, files, - authorizations, sci, topLevelIterators); - } - - @Override - public ServerContext getServerContext() { - return context; - } - - @Deprecated(since = "2.1.0") - @Override - public ServiceEnvironment getServiceEnv() { - return serviceEnvironment; - } - - @Override - public TableId getTableId() { - return tableId; - } - - @Override - public PluginEnvironment getPluginEnv() { - return serviceEnvironment; - } -} diff --git a/server/compactor/src/main/java/org/apache/accumulo/compactor/ExtCEnv.java b/server/compactor/src/main/java/org/apache/accumulo/compactor/ExtCEnv.java index 260f2c69aff..d109c97d9ca 100644 --- a/server/compactor/src/main/java/org/apache/accumulo/compactor/ExtCEnv.java +++ b/server/compactor/src/main/java/org/apache/accumulo/compactor/ExtCEnv.java @@ -32,7 +32,7 @@ import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.compaction.FileCompactor.CompactionEnv; import org.apache.accumulo.server.iterators.SystemIteratorEnvironment; -import org.apache.accumulo.server.iterators.TabletIteratorEnvironment; +import org.apache.accumulo.server.iterators.SystemIteratorEnvironmentImpl; import com.google.common.annotations.VisibleForTesting; @@ -42,20 +42,36 @@ public class ExtCEnv implements CompactionEnv { private final TExternalCompactionJob job; private final String queueName; - public static class CompactorIterEnv extends TabletIteratorEnvironment { + public static class CompactorIterEnv extends SystemIteratorEnvironmentImpl { + + private static class Builder extends SystemIteratorEnvironmentImpl.Builder { + + private final String queueName; + + public Builder(ServerContext context, String queueName) { + super(context); + this.queueName = queueName; + } + + @Override + public SystemIteratorEnvironmentImpl build() { + return new CompactorIterEnv(this); + } + + } private final String queueName; - public CompactorIterEnv(ServerContext context, IteratorScope scope, boolean fullMajC, - AccumuloConfiguration tableConfig, TableId tableId, CompactionKind kind, String queueName) { - super(context, scope, fullMajC, tableConfig, tableId, kind); - this.queueName = queueName; + public CompactorIterEnv(Builder builder) { + super(builder); + this.queueName = builder.queueName; } @VisibleForTesting public String getQueueName() { return queueName; } + } ExtCEnv(CompactionJobHolder jobHolder, String queueName) { @@ -87,9 +103,19 @@ public RateLimiter getWriteLimiter() { @Override public SystemIteratorEnvironment createIteratorEnv(ServerContext context, AccumuloConfiguration acuTableConf, TableId tableId) { - return new CompactorIterEnv(context, IteratorScope.majc, - !jobHolder.getJob().isPropagateDeletes(), acuTableConf, tableId, - CompactionKind.valueOf(job.getKind().name()), queueName); + + CompactorIterEnv.Builder builder = new CompactorIterEnv.Builder(context, queueName); + builder.withScope(IteratorScope.majc).withTableId(tableId); + + if (CompactionKind.valueOf(job.getKind().name()) == CompactionKind.USER) { + builder.isUserCompaction(); + } + + if (!jobHolder.getJob().isPropagateDeletes()) { + builder.isFullMajorCompaction(); + } + + return builder.build(); } @Override diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionCheckerContext.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionCheckerContext.java index 7b0e24c206c..f395c4134ad 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionCheckerContext.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionCheckerContext.java @@ -39,6 +39,7 @@ import org.apache.accumulo.core.dataImpl.thrift.TCMResult; import org.apache.accumulo.core.dataImpl.thrift.TCMStatus; import org.apache.accumulo.core.dataImpl.thrift.TCondition; +import org.apache.accumulo.core.iterators.IteratorEnvironment; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; import org.apache.accumulo.core.iteratorsImpl.IteratorBuilder; @@ -46,7 +47,7 @@ import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.conf.TableConfiguration; import org.apache.accumulo.server.conf.TableConfiguration.ParsedIteratorConfig; -import org.apache.accumulo.server.iterators.TabletIteratorEnvironment; +import org.apache.accumulo.server.iterators.SystemIteratorEnvironmentImpl; import org.apache.accumulo.tserver.data.ServerConditionalMutation; import org.apache.hadoop.io.Text; @@ -55,7 +56,7 @@ public class ConditionCheckerContext { private final List tableIters; private final Map> tableIterOpts; - private final TabletIteratorEnvironment tie; + private final IteratorEnvironment ie; private final String context; private static class MergedIterConfig { @@ -80,8 +81,8 @@ private static class MergedIterConfig { tableIterOpts = pic.getOpts(); this.context = pic.getServiceEnv(); - tie = new TabletIteratorEnvironment(context, IteratorScope.scan, tableConf, - tableConf.getTableId()); + ie = new SystemIteratorEnvironmentImpl.Builder(context).withScope(IteratorScope.scan) + .withTableId(tableConf.getTableId()).build(); } SortedKeyValueIterator buildIterator(SortedKeyValueIterator systemIter, @@ -104,8 +105,8 @@ SortedKeyValueIterator buildIterator(SortedKeyValueIterator createIterator() throws IOException { MultiIterator multiIter = new MultiIterator(iters, tablet.getExtent()); - TabletIteratorEnvironment iterEnv = new TabletIteratorEnvironment(tablet.getContext(), - IteratorScope.scan, tablet.getTableConfiguration(), tablet.getExtent().tableId(), - fileManager, files, scanParams.getAuthorizations(), samplerConfig, new ArrayList<>()); + var builder = new SystemIteratorEnvironmentImpl.Builder(tablet.getContext()) + .withTopLevelIterators(new ArrayList<>()).withScope(IteratorScope.scan) + .withTableId(tablet.getExtent().tableId()) + .withAuthorizations(scanParams.getAuthorizations()); + if (samplerConfig != null) { + builder.withSamplingEnabled(); + builder.withSamplerConfiguration(samplerConfig.toSamplerConfiguration()); + } + SystemIteratorEnvironment iterEnv = (SystemIteratorEnvironment) builder.build(); statsIterator = new StatsIterator(multiIter, TabletServer.seekCount, tablet.getScannedCounter(), tablet.getScanMetrics().getScannedCounter()); diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/InMemoryMapTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/InMemoryMapTest.java index 5bd26d5b74f..79d58cf6b8c 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/InMemoryMapTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/InMemoryMapTest.java @@ -58,7 +58,7 @@ import org.apache.accumulo.core.util.LocalityGroupUtil; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.conf.TableConfiguration; -import org.apache.accumulo.server.iterators.TabletIteratorEnvironment; +import org.apache.accumulo.server.iterators.SystemIteratorEnvironmentImpl; import org.apache.accumulo.tserver.InMemoryMap.MemoryIterator; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; @@ -80,6 +80,7 @@ public static ServerContext getServerContext() { EasyMock.expect(context.getTableConfiguration(EasyMock.anyObject())).andReturn(tConf) .anyTimes(); EasyMock.expect(tConf.getCryptoService()).andReturn(NoCryptoServiceFactory.NONE).anyTimes(); + EasyMock.expect(tConf.get(Property.TABLE_SAMPLER)).andReturn("").anyTimes(); EasyMock.expect(context.getHadoopConf()).andReturn(hadoopConf).anyTimes(); EasyMock.replay(context, tConf); return context; @@ -297,8 +298,8 @@ public void test6() throws Exception { mutate(imm, "r1", "foo:cq5", 3, "bar5"); SortedKeyValueIterator dc = - ski1.deepCopy(new TabletIteratorEnvironment(getServerContext(), IteratorScope.scan, - getServerContext().getTableConfiguration(TableId.of("foo")), TableId.of("foo"))); + ski1.deepCopy(new SystemIteratorEnvironmentImpl.Builder(getServerContext()) + .withScope(IteratorScope.scan).withTableId(TableId.of("foo")).build()); ski1.seek(new Range(newKey("r1", "foo:cq1", 3), null), Set.of(), false); testAndCallNext(ski1, "r1", "foo:cq1", 3, "bar1"); @@ -351,8 +352,8 @@ private void deepCopyAndDelete(int interleaving, boolean interrupt) throws Excep } SortedKeyValueIterator dc = - ski1.deepCopy(new TabletIteratorEnvironment(getServerContext(), IteratorScope.scan, - getServerContext().getTableConfiguration(TableId.of("foo")), TableId.of("foo"))); + ski1.deepCopy(new SystemIteratorEnvironmentImpl.Builder(getServerContext()) + .withScope(IteratorScope.scan).withTableId(TableId.of("foo")).build()); if (interleaving == 2) { imm.delete(0); if (interrupt) { @@ -504,8 +505,8 @@ public void testLocalityGroups() throws Exception { seekLocalityGroups(iter1); SortedKeyValueIterator dc1 = - iter1.deepCopy(new TabletIteratorEnvironment(getServerContext(), IteratorScope.scan, - getServerContext().getTableConfiguration(TableId.of("foo")), TableId.of("foo"))); + iter1.deepCopy(new SystemIteratorEnvironmentImpl.Builder(getServerContext()) + .withScope(IteratorScope.scan).withTableId(TableId.of("foo")).build()); seekLocalityGroups(dc1); assertEquals(10, imm.getNumEntries()); @@ -559,26 +560,26 @@ public void testSample() throws Exception { MemoryIterator iter1 = imm.skvIterator(sampleConfig); MemoryIterator iter2 = imm.skvIterator(null); SortedKeyValueIterator iter0dc1 = - iter0.deepCopy(new TabletIteratorEnvironment(getServerContext(), IteratorScope.scan, - getServerContext().getTableConfiguration(TableId.of("foo")), TableId.of("foo"))); - SortedKeyValueIterator iter0dc2 = iter0.deepCopy(new TabletIteratorEnvironment(getServerContext(), - IteratorScope.scan, getServerContext().getTableConfiguration(TableId.of("foo")), - TableId.of("foo"), sampleConfig)); + iter0.deepCopy(new SystemIteratorEnvironmentImpl.Builder(getServerContext()) + .withScope(IteratorScope.scan).withTableId(TableId.of("foo")).build()); + SortedKeyValueIterator iter0dc2 = + iter0.deepCopy(new SystemIteratorEnvironmentImpl.Builder(getServerContext()) + .withScope(IteratorScope.scan).withTableId(TableId.of("foo")).withSamplingEnabled() + .withSamplerConfiguration(sampleConfig.toSamplerConfiguration()).build()); SortedKeyValueIterator iter1dc1 = - iter1.deepCopy(new TabletIteratorEnvironment(getServerContext(), IteratorScope.scan, - getServerContext().getTableConfiguration(TableId.of("foo")), TableId.of("foo"))); - SortedKeyValueIterator iter1dc2 = iter1.deepCopy(new TabletIteratorEnvironment(getServerContext(), - IteratorScope.scan, getServerContext().getTableConfiguration(TableId.of("foo")), - TableId.of("foo"), sampleConfig)); + iter1.deepCopy(new SystemIteratorEnvironmentImpl.Builder(getServerContext()) + .withScope(IteratorScope.scan).withTableId(TableId.of("foo")).build()); + SortedKeyValueIterator iter1dc2 = + iter1.deepCopy(new SystemIteratorEnvironmentImpl.Builder(getServerContext()) + .withScope(IteratorScope.scan).withTableId(TableId.of("foo")).withSamplingEnabled() + .withSamplerConfiguration(sampleConfig.toSamplerConfiguration()).build()); SortedKeyValueIterator iter2dc1 = - iter2.deepCopy(new TabletIteratorEnvironment(getServerContext(), IteratorScope.scan, - getServerContext().getTableConfiguration(TableId.of("foo")), TableId.of("foo"))); - SortedKeyValueIterator iter2dc2 = iter2.deepCopy(new TabletIteratorEnvironment(getServerContext(), - IteratorScope.scan, getServerContext().getTableConfiguration(TableId.of("foo")), - TableId.of("foo"), sampleConfig)); + iter2.deepCopy(new SystemIteratorEnvironmentImpl.Builder(getServerContext()) + .withScope(IteratorScope.scan).withTableId(TableId.of("foo")).build()); + SortedKeyValueIterator iter2dc2 = + iter2.deepCopy(new SystemIteratorEnvironmentImpl.Builder(getServerContext()) + .withScope(IteratorScope.scan).withTableId(TableId.of("foo")).withSamplingEnabled() + .withSamplerConfiguration(sampleConfig.toSamplerConfiguration()).build()); assertEquals(expectedNone, readAll(iter0)); assertEquals(expectedNone, readAll(iter0dc1)); @@ -603,26 +604,26 @@ IteratorScope.scan, getServerContext().getTableConfiguration(TableId.of("foo")), assertEquals(expectedSample, readAll(iter2dc2)); SortedKeyValueIterator iter0dc3 = - iter0.deepCopy(new TabletIteratorEnvironment(getServerContext(), IteratorScope.scan, - getServerContext().getTableConfiguration(TableId.of("foo")), TableId.of("foo"))); - SortedKeyValueIterator iter0dc4 = iter0.deepCopy(new TabletIteratorEnvironment(getServerContext(), - IteratorScope.scan, getServerContext().getTableConfiguration(TableId.of("foo")), - TableId.of("foo"), sampleConfig)); + iter0.deepCopy(new SystemIteratorEnvironmentImpl.Builder(getServerContext()) + .withScope(IteratorScope.scan).withTableId(TableId.of("foo")).build()); + SortedKeyValueIterator iter0dc4 = + iter0.deepCopy(new SystemIteratorEnvironmentImpl.Builder(getServerContext()) + .withScope(IteratorScope.scan).withTableId(TableId.of("foo")).withSamplingEnabled() + .withSamplerConfiguration(sampleConfig.toSamplerConfiguration()).build()); SortedKeyValueIterator iter1dc3 = - iter1.deepCopy(new TabletIteratorEnvironment(getServerContext(), IteratorScope.scan, - getServerContext().getTableConfiguration(TableId.of("foo")), TableId.of("foo"))); - SortedKeyValueIterator iter1dc4 = iter1.deepCopy(new TabletIteratorEnvironment(getServerContext(), - IteratorScope.scan, getServerContext().getTableConfiguration(TableId.of("foo")), - TableId.of("foo"), sampleConfig)); + iter1.deepCopy(new SystemIteratorEnvironmentImpl.Builder(getServerContext()) + .withScope(IteratorScope.scan).withTableId(TableId.of("foo")).build()); + SortedKeyValueIterator iter1dc4 = + iter1.deepCopy(new SystemIteratorEnvironmentImpl.Builder(getServerContext()) + .withScope(IteratorScope.scan).withTableId(TableId.of("foo")).withSamplingEnabled() + .withSamplerConfiguration(sampleConfig.toSamplerConfiguration()).build()); SortedKeyValueIterator iter2dc3 = - iter2.deepCopy(new TabletIteratorEnvironment(getServerContext(), IteratorScope.scan, - getServerContext().getTableConfiguration(TableId.of("foo")), TableId.of("foo"))); - SortedKeyValueIterator iter2dc4 = iter2.deepCopy(new TabletIteratorEnvironment(getServerContext(), - IteratorScope.scan, getServerContext().getTableConfiguration(TableId.of("foo")), - TableId.of("foo"), sampleConfig)); + iter2.deepCopy(new SystemIteratorEnvironmentImpl.Builder(getServerContext()) + .withScope(IteratorScope.scan).withTableId(TableId.of("foo")).build()); + SortedKeyValueIterator iter2dc4 = + iter2.deepCopy(new SystemIteratorEnvironmentImpl.Builder(getServerContext()) + .withScope(IteratorScope.scan).withTableId(TableId.of("foo")).withSamplingEnabled() + .withSamplerConfiguration(sampleConfig.toSamplerConfiguration()).build()); assertEquals(expectedNone, readAll(iter0dc3)); assertEquals(expectedNone, readAll(iter0dc4)); @@ -679,9 +680,9 @@ private void runInterruptSampleTest(boolean deepCopy, boolean delete, boolean dc } if (deepCopy) { - iter = iter.deepCopy(new TabletIteratorEnvironment(getServerContext(), IteratorScope.scan, - getServerContext().getTableConfiguration(TableId.of("foo")), TableId.of("foo"), - sampleConfig1)); + iter = iter.deepCopy(new SystemIteratorEnvironmentImpl.Builder(getServerContext()) + .withScope(IteratorScope.scan).withTableId(TableId.of("foo")).withSamplingEnabled() + .withSamplerConfiguration(sampleConfig1.toSamplerConfiguration()).build()); } if (delete && dcAfterDelete) { @@ -783,9 +784,9 @@ public void testDeferredSamplerCreation() throws Exception { assertEquals(expectedSample, readAll(iter)); SortedKeyValueIterator dc = iter.deepCopy(new TabletIteratorEnvironment(getServerContext(), - IteratorScope.scan, getServerContext().getTableConfiguration(TableId.of("foo")), - TableId.of("foo"), sampleConfig2)); + Value> dc = iter.deepCopy(new SystemIteratorEnvironmentImpl.Builder(getServerContext()) + .withScope(IteratorScope.scan).withTableId(TableId.of("foo")).withSamplingEnabled() + .withSamplerConfiguration(sampleConfig2.toSamplerConfiguration()).build()); dc.seek(new Range(), Set.of(), false); assertEquals(expectedSample, readAll(dc)); diff --git a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java index e34747e3aea..bba4b355bc0 100644 --- a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java +++ b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java @@ -52,6 +52,7 @@ import org.apache.accumulo.core.dataImpl.thrift.IterInfo; import org.apache.accumulo.core.file.FileOperations; import org.apache.accumulo.core.file.FileSKVIterator; +import org.apache.accumulo.core.iterators.IteratorEnvironment; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; import org.apache.accumulo.core.iteratorsImpl.IteratorConfigUtil; @@ -72,7 +73,7 @@ import org.apache.accumulo.server.cli.ServerUtilOpts; import org.apache.accumulo.server.conf.TableConfiguration; import org.apache.accumulo.server.fs.VolumeManager; -import org.apache.accumulo.server.iterators.TabletIteratorEnvironment; +import org.apache.accumulo.server.iterators.SystemIteratorEnvironmentImpl; import org.apache.accumulo.server.util.MetadataTableUtil; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FileStatus; @@ -447,8 +448,8 @@ private static SortedKeyValueIterator createScanIterator(KeyExtent ke if (useTableIterators) { var ibEnv = IteratorConfigUtil.loadIterConf(IteratorScope.scan, ssiList, ssio, conf); - TabletIteratorEnvironment iterEnv = - new TabletIteratorEnvironment(context, IteratorScope.scan, conf, ke.tableId()); + IteratorEnvironment iterEnv = new SystemIteratorEnvironmentImpl.Builder(context) + .withScope(IteratorScope.scan).withTableId(ke.tableId()).build(); var iteratorBuilder = ibEnv.env(iterEnv).useClassLoader("test").build(); return IteratorConfigUtil.loadIterators(visFilter, iteratorBuilder); }