Skip to content

Commit ec5954e

Browse files
dlmarionkevinrr888
andauthored
Modified IteratorEnvironment to no longer throw UOE (#5490)
Modified the IteratorEnvironment interface such that the methods no longer have default implementation that throw UnsupportedOperationException. Created the class ClientIteratorEnvironment to serve as the default implementation for the client objects that defined their own implementation and for use in the tests. Removed all test implementations of IteratorEnvironment. Closes #4810 Co-authored-by: Kevin Rathbun <[email protected]>
1 parent c2df3ba commit ec5954e

File tree

28 files changed

+504
-565
lines changed

28 files changed

+504
-565
lines changed

core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java

Lines changed: 9 additions & 69 deletions
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,6 @@
3434
import org.apache.accumulo.core.Constants;
3535
import org.apache.accumulo.core.client.sample.SamplerConfiguration;
3636
import org.apache.accumulo.core.clientImpl.ClientContext;
37-
import org.apache.accumulo.core.clientImpl.ClientServiceEnvironmentImpl;
3837
import org.apache.accumulo.core.clientImpl.ScannerImpl;
3938
import org.apache.accumulo.core.clientImpl.ScannerOptions;
4039
import org.apache.accumulo.core.data.ArrayByteSequence;
@@ -49,10 +48,10 @@
4948
import org.apache.accumulo.core.iterators.IteratorEnvironment;
5049
import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
5150
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
51+
import org.apache.accumulo.core.iteratorsImpl.ClientIteratorEnvironment;
5252
import org.apache.accumulo.core.iteratorsImpl.IteratorBuilder;
5353
import org.apache.accumulo.core.iteratorsImpl.IteratorConfigUtil;
5454
import org.apache.accumulo.core.security.Authorizations;
55-
import org.apache.accumulo.core.spi.common.ServiceEnvironment;
5655
import org.apache.hadoop.io.Text;
5756

5857
/**
@@ -87,70 +86,6 @@ public class ClientSideIteratorScanner extends ScannerOptions implements Scanner
8786
private final Supplier<ClientContext> context;
8887
private final Supplier<TableId> tableId;
8988

90-
private class ClientSideIteratorEnvironment implements IteratorEnvironment {
91-
92-
private final SamplerConfiguration samplerConfig;
93-
private final boolean sampleEnabled;
94-
95-
ClientSideIteratorEnvironment(boolean sampleEnabled, SamplerConfiguration samplerConfig) {
96-
this.sampleEnabled = sampleEnabled;
97-
this.samplerConfig = samplerConfig;
98-
}
99-
100-
@Override
101-
public IteratorScope getIteratorScope() {
102-
return IteratorScope.scan;
103-
}
104-
105-
@Override
106-
public boolean isFullMajorCompaction() {
107-
// The javadocs state this method will throw an ISE when scope is not majc
108-
throw new IllegalStateException(
109-
"Asked about major compaction type when scope is " + getIteratorScope());
110-
}
111-
112-
@Override
113-
public boolean isUserCompaction() {
114-
return false;
115-
}
116-
117-
@Override
118-
public Authorizations getAuthorizations() {
119-
return ClientSideIteratorScanner.this.getAuthorizations();
120-
}
121-
122-
@Override
123-
public IteratorEnvironment cloneWithSamplingEnabled() {
124-
return new ClientSideIteratorEnvironment(true, samplerConfig);
125-
}
126-
127-
@Override
128-
public boolean isSamplingEnabled() {
129-
return sampleEnabled;
130-
}
131-
132-
@Override
133-
public SamplerConfiguration getSamplerConfiguration() {
134-
return samplerConfig;
135-
}
136-
137-
@Deprecated(since = "2.1.0")
138-
@Override
139-
public ServiceEnvironment getServiceEnv() {
140-
return new ClientServiceEnvironmentImpl(context.get());
141-
}
142-
143-
@Override
144-
public PluginEnvironment getPluginEnv() {
145-
return new ClientServiceEnvironmentImpl(context.get());
146-
}
147-
148-
@Override
149-
public TableId getTableId() {
150-
return tableId.get();
151-
}
152-
}
153-
15489
/**
15590
* A class that wraps a Scanner in a SortedKeyValueIterator so that other accumulo iterators can
15691
* use it as a source.
@@ -295,9 +230,14 @@ public Iterator<Entry<Key,Value>> iterator() {
295230

296231
SortedKeyValueIterator<Key,Value> skvi;
297232
try {
298-
IteratorEnvironment iterEnv = new ClientSideIteratorEnvironment(
299-
getSamplerConfiguration() != null, getIteratorSamplerConfigurationInternal());
300-
233+
ClientIteratorEnvironment.Builder builder = new ClientIteratorEnvironment.Builder()
234+
.withClient(context.get()).withAuthorizations(getAuthorizations())
235+
.withScope(IteratorScope.scan).withTableId(tableId.get())
236+
.withSamplerConfiguration(getIteratorSamplerConfigurationInternal());
237+
if (getSamplerConfiguration() != null) {
238+
builder.withSamplingEnabled();
239+
}
240+
IteratorEnvironment iterEnv = builder.build();
301241
IteratorBuilder ib =
302242
IteratorBuilder.builder(tm.values()).opts(serverSideIteratorOptions).env(iterEnv).build();
303243

core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java

Lines changed: 53 additions & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -30,8 +30,9 @@
3030

3131
import org.apache.accumulo.core.client.IteratorSetting;
3232
import org.apache.accumulo.core.client.Scanner;
33+
import org.apache.accumulo.core.client.TableNotFoundException;
3334
import org.apache.accumulo.core.client.rfile.RFileScannerBuilder.InputArgs;
34-
import org.apache.accumulo.core.client.sample.SamplerConfiguration;
35+
import org.apache.accumulo.core.clientImpl.ClientServiceEnvironmentImpl;
3536
import org.apache.accumulo.core.clientImpl.ScannerOptions;
3637
import org.apache.accumulo.core.conf.AccumuloConfiguration;
3738
import org.apache.accumulo.core.conf.ConfigurationCopy;
@@ -42,6 +43,7 @@
4243
import org.apache.accumulo.core.data.Column;
4344
import org.apache.accumulo.core.data.Key;
4445
import org.apache.accumulo.core.data.Range;
46+
import org.apache.accumulo.core.data.TableId;
4547
import org.apache.accumulo.core.data.Value;
4648
import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheConfiguration;
4749
import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheManagerFactory;
@@ -55,6 +57,7 @@
5557
import org.apache.accumulo.core.iterators.IteratorEnvironment;
5658
import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
5759
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
60+
import org.apache.accumulo.core.iteratorsImpl.ClientIteratorEnvironment;
5861
import org.apache.accumulo.core.iteratorsImpl.IteratorBuilder;
5962
import org.apache.accumulo.core.iteratorsImpl.IteratorConfigUtil;
6063
import org.apache.accumulo.core.iteratorsImpl.system.MultiIterator;
@@ -66,6 +69,7 @@
6669
import org.apache.accumulo.core.spi.cache.CacheType;
6770
import org.apache.accumulo.core.spi.crypto.CryptoEnvironment;
6871
import org.apache.accumulo.core.spi.crypto.CryptoService;
72+
import org.apache.accumulo.core.util.ConfigurationImpl;
6973
import org.apache.accumulo.core.util.LocalityGroupUtil;
7074
import org.apache.hadoop.fs.FSDataInputStream;
7175
import org.apache.hadoop.io.Text;
@@ -74,8 +78,46 @@
7478

7579
class RFileScanner extends ScannerOptions implements Scanner {
7680

81+
private static class RFileScannerEnvironmentImpl extends ClientServiceEnvironmentImpl {
82+
83+
private final Configuration conf;
84+
private final Configuration tableConf;
85+
86+
public RFileScannerEnvironmentImpl(Opts opts) {
87+
super(null);
88+
conf = new ConfigurationImpl(new ConfigurationCopy(DefaultConfiguration.getInstance()));
89+
ConfigurationCopy tableCC = new ConfigurationCopy(DefaultConfiguration.getInstance());
90+
if (opts.tableConfig != null) {
91+
opts.tableConfig.forEach(tableCC::set);
92+
}
93+
tableConf = new ConfigurationImpl(tableCC);
94+
}
95+
96+
@Override
97+
public String getTableName(TableId tableId) throws TableNotFoundException {
98+
Preconditions.checkArgument(tableId == TABLE_ID, "Expected " + TABLE_ID + " obtained"
99+
+ " from IteratorEnvironment.getTableId(), but got: " + tableId);
100+
return TABLE_NAME;
101+
}
102+
103+
@Override
104+
public Configuration getConfiguration() {
105+
return conf;
106+
}
107+
108+
@Override
109+
public Configuration getConfiguration(TableId tableId) {
110+
Preconditions.checkArgument(tableId == TABLE_ID, "Expected " + TABLE_ID + " obtained"
111+
+ " from IteratorEnvironment.getTableId(), but got: " + tableId);
112+
return tableConf;
113+
}
114+
115+
}
116+
77117
private static final byte[] EMPTY_BYTES = new byte[0];
78118
private static final Range EMPTY_RANGE = new Range();
119+
private static final String TABLE_NAME = "rfileScanner";
120+
private static final TableId TABLE_ID = TableId.of(TABLE_NAME);
79121

80122
private Range range;
81123
private BlockCacheManager blockCacheManager = null;
@@ -225,33 +267,6 @@ public void updateScanIteratorOption(String iteratorName, String key, String val
225267
super.updateScanIteratorOption(iteratorName, key, value);
226268
}
227269

228-
private class IterEnv implements IteratorEnvironment {
229-
@Override
230-
public IteratorScope getIteratorScope() {
231-
return IteratorScope.scan;
232-
}
233-
234-
@Override
235-
public boolean isFullMajorCompaction() {
236-
return false;
237-
}
238-
239-
@Override
240-
public Authorizations getAuthorizations() {
241-
return opts.auths;
242-
}
243-
244-
@Override
245-
public boolean isSamplingEnabled() {
246-
return RFileScanner.this.getSamplerConfiguration() != null;
247-
}
248-
249-
@Override
250-
public SamplerConfiguration getSamplerConfiguration() {
251-
return RFileScanner.this.getSamplerConfiguration();
252-
}
253-
}
254-
255270
@Override
256271
public Iterator<Entry<Key,Value>> iterator() {
257272
try {
@@ -292,15 +307,23 @@ public Iterator<Entry<Key,Value>> iterator() {
292307
EMPTY_BYTES, tableConf);
293308
}
294309

310+
ClientIteratorEnvironment.Builder iterEnvBuilder = new ClientIteratorEnvironment.Builder()
311+
.withEnvironment(new RFileScannerEnvironmentImpl(opts)).withAuthorizations(opts.auths)
312+
.withScope(IteratorScope.scan).withTableId(TABLE_ID);
313+
if (getSamplerConfiguration() != null) {
314+
iterEnvBuilder.withSamplerConfiguration(getSamplerConfiguration());
315+
iterEnvBuilder.withSamplingEnabled();
316+
}
317+
IteratorEnvironment iterEnv = iterEnvBuilder.build();
295318
try {
296319
if (opts.tableConfig != null && !opts.tableConfig.isEmpty()) {
297320
var ibEnv = IteratorConfigUtil.loadIterConf(IteratorScope.scan, serverSideIteratorList,
298321
serverSideIteratorOptions, tableConf);
299-
var iteratorBuilder = ibEnv.env(new IterEnv()).build();
322+
var iteratorBuilder = ibEnv.env(iterEnv).build();
300323
iterator = IteratorConfigUtil.loadIterators(iterator, iteratorBuilder);
301324
} else {
302325
var iteratorBuilder = IteratorBuilder.builder(serverSideIteratorList)
303-
.opts(serverSideIteratorOptions).env(new IterEnv()).build();
326+
.opts(serverSideIteratorOptions).env(iterEnv).build();
304327
iterator = IteratorConfigUtil.loadIterators(iterator, iteratorBuilder);
305328
}
306329
} catch (IOException e) {

0 commit comments

Comments
 (0)