keith-turner commented on code in PR #5490: URL: https://github.com/apache/accumulo/pull/5490#discussion_r2054918721
########## core/src/main/java/org/apache/accumulo/core/iterators/ClientIteratorEnvironment.java: ########## @@ -0,0 +1,220 @@ +/* + * 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.iterators; + +import java.io.IOException; +import java.util.Optional; + +import org.apache.accumulo.core.client.AccumuloClient; +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.clientImpl.ClientContext; +import org.apache.accumulo.core.clientImpl.ClientServiceEnvironmentImpl; +import org.apache.accumulo.core.conf.Property; +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.IteratorUtil.IteratorScope; +import org.apache.accumulo.core.iteratorsImpl.system.MapFileIterator; +import org.apache.accumulo.core.security.Authorizations; +import org.apache.accumulo.core.spi.common.ServiceEnvironment; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; + +public class ClientIteratorEnvironment implements IteratorEnvironment { + + public static class Builder { + + private Optional<IteratorScope> scope = Optional.empty(); + private boolean isFullMajorCompaction = false; + private Optional<Authorizations> auths = Optional.empty(); + private boolean isUserCompaction = false; + private Optional<TableId> tableId = Optional.empty(); + private Optional<SamplerConfiguration> samplerConfig = Optional.empty(); + private boolean samplingEnabled = false; + protected Optional<ClientServiceEnvironmentImpl> env = Optional.empty(); + + public Builder withScope(IteratorScope scope) { + this.scope = Optional.of(scope); + return this; + } + + public Builder isFullMajorCompaction() { + this.isFullMajorCompaction = true; + return this; + } + + public Builder withAuthorizations(Authorizations auths) { + this.auths = Optional.of(auths); + return this; + } + + public Builder isUserCompaction() { + this.isUserCompaction = true; + return this; + } + + public Builder withTableId(TableId tableId) { + this.tableId = Optional.of(tableId); + return this; + } + + public Builder withSamplingEnabled() { + this.samplingEnabled = true; + return this; + } + + public Builder withSamplerConfiguration(SamplerConfiguration sc) { + this.samplerConfig = Optional.ofNullable(sc); + return this; + } + + public Builder withClient(AccumuloClient client) { + this.env = Optional.of(new ClientServiceEnvironmentImpl((ClientContext) client)); + return this; + } + + public ClientIteratorEnvironment build() { + return new ClientIteratorEnvironment(this); + } + + } + + public static final IteratorEnvironment DEFAULT = new Builder().build(); + + private final Optional<IteratorScope> scope; + private final boolean isFullMajorCompaction; + private final Optional<Authorizations> auths; + private final boolean isUserCompaction; + private final Optional<TableId> tableId; + private final Optional<SamplerConfiguration> samplerConfig; + private final boolean samplingEnabled; + private final Optional<ClientServiceEnvironmentImpl> env; + + private ClientIteratorEnvironment(Builder builder) { + this.scope = builder.scope; + this.isFullMajorCompaction = builder.isFullMajorCompaction; + this.auths = builder.auths; + this.isUserCompaction = builder.isUserCompaction; + this.tableId = builder.tableId; + this.samplerConfig = builder.samplerConfig; + this.env = builder.env; + this.samplingEnabled = builder.samplingEnabled; + } + + /** + * Copy constructor used for enabling sample. Only called from {@link cloneWithSamplingEnabled}. + */ + private ClientIteratorEnvironment(ClientIteratorEnvironment copy) { + this.scope = copy.scope; + this.isFullMajorCompaction = copy.isFullMajorCompaction; + this.auths = copy.auths; + this.isUserCompaction = copy.isUserCompaction; + this.tableId = copy.tableId; + this.samplerConfig = copy.samplerConfig; + this.env = copy.env; + this.samplingEnabled = true; + } + + @Override + @Deprecated(since = "2.0.0") + public SortedKeyValueIterator<Key,Value> reserveMapFileReader(String mapFileName) + throws IOException { + Configuration hadoopConf = new Configuration(); Review Comment: Was this implementation based on something else? On the server side this method can open an rfile or a mapfile (Looking at TabletIteratorEnvironment). According to my IDE nothing calls this method, so no test. Since this method has zero test, we may be better off throwing UnsupportedOperationException for now. Should probably test it if we add an impl. ########## core/src/main/java/org/apache/accumulo/core/iterators/ClientIteratorEnvironment.java: ########## @@ -0,0 +1,220 @@ +/* + * 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.iterators; + +import java.io.IOException; +import java.util.Optional; + +import org.apache.accumulo.core.client.AccumuloClient; +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.clientImpl.ClientContext; +import org.apache.accumulo.core.clientImpl.ClientServiceEnvironmentImpl; +import org.apache.accumulo.core.conf.Property; +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.IteratorUtil.IteratorScope; +import org.apache.accumulo.core.iteratorsImpl.system.MapFileIterator; +import org.apache.accumulo.core.security.Authorizations; +import org.apache.accumulo.core.spi.common.ServiceEnvironment; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; + +public class ClientIteratorEnvironment implements IteratorEnvironment { Review Comment: Is this class intended to be in the public API? This package is in the public API. Would need javadoc and since tags if that is the intent. Seems like all of its public methods only use public API types, so it could be in the public API. If this were in the public API, then it seems like it would be useful for users writing unit test. Would be good to kick the tires on that concept before releasing and try using on an external project in unit test to see if there any hiccups. Seems like it may work nicely w/ the following for external unit test. https://github.com/apache/accumulo/blob/9eda6d28f469fef0844eeb6ff4c765a2896cb41c/core/src/main/java/org/apache/accumulo/core/client/PluginEnvironment.java#L145 However those changes are in main. ########## core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java: ########## @@ -74,8 +79,65 @@ class RFileScanner extends ScannerOptions implements Scanner { + private static class RFileScannerIteratorEnvironmentBuilder + extends ClientIteratorEnvironment.Builder { + + public ClientIteratorEnvironment.Builder withEnvironment(ClientServiceEnvironmentImpl env) { Review Comment: Could push this method up into `ClientIteratorEnvironment.Builder` and then would not need this specialized class. Also may be able to generlize the type on the method to. ``` public ClientIteratorEnvironment.Builder withEnvironment(ServiceEnvironment env) ``` ########## core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java: ########## @@ -74,8 +79,65 @@ class RFileScanner extends ScannerOptions implements Scanner { + private static class RFileScannerIteratorEnvironmentBuilder + extends ClientIteratorEnvironment.Builder { + + public ClientIteratorEnvironment.Builder withEnvironment(ClientServiceEnvironmentImpl env) { + this.env = Optional.of(env); + return this; + } + + } + + private static class RFileScannerEnvironmentImpl extends ClientServiceEnvironmentImpl { + + private final Opts opts; + + public RFileScannerEnvironmentImpl(Opts opts) { + super(null); + this.opts = opts; + } + + @Override + public String getTableName(TableId tableId) throws TableNotFoundException { + Preconditions.checkArgument(tableId == TABLE_ID, "Expected " + TABLE_ID + " obtained" + + " from IteratorEnvironment.getTableId(), but got: " + tableId); + return TABLE_NAME; + } + + @Override + public <T> T instantiate(String className, Class<T> base) + throws ReflectiveOperationException, IOException { + return RFileScanner.class.getClassLoader().loadClass(className).asSubclass(base) + .getDeclaredConstructor().newInstance(); + } + + @Override + public <T> T instantiate(TableId tableId, String className, Class<T> base) + throws ReflectiveOperationException, IOException { + return instantiate(className, base); + } + + @Override + public Configuration getConfiguration() { + return new ConfigurationImpl(new ConfigurationCopy(DefaultConfiguration.getInstance())); + } + + @Override + public Configuration getConfiguration(TableId tableId) { + Preconditions.checkArgument(tableId == TABLE_ID, "Expected " + TABLE_ID + " obtained" + + " from IteratorEnvironment.getTableId(), but got: " + tableId); + ConfigurationCopy tableCC = new ConfigurationCopy(DefaultConfiguration.getInstance()); + opts.tableConfig.forEach(tableCC::set); + return new ConfigurationImpl(tableCC); + } Review Comment: Could memorize these or create them once in the constructor. Sometimes calling code will repeatedly call these methods for each property. ########## core/src/main/java/org/apache/accumulo/core/iterators/ClientIteratorEnvironment.java: ########## @@ -0,0 +1,220 @@ +/* + * 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.iterators; + +import java.io.IOException; +import java.util.Optional; + +import org.apache.accumulo.core.client.AccumuloClient; +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.clientImpl.ClientContext; +import org.apache.accumulo.core.clientImpl.ClientServiceEnvironmentImpl; +import org.apache.accumulo.core.conf.Property; +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.IteratorUtil.IteratorScope; +import org.apache.accumulo.core.iteratorsImpl.system.MapFileIterator; +import org.apache.accumulo.core.security.Authorizations; +import org.apache.accumulo.core.spi.common.ServiceEnvironment; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; + +public class ClientIteratorEnvironment implements IteratorEnvironment { + + public static class Builder { + + private Optional<IteratorScope> scope = Optional.empty(); + private boolean isFullMajorCompaction = false; + private Optional<Authorizations> auths = Optional.empty(); + private boolean isUserCompaction = false; + private Optional<TableId> tableId = Optional.empty(); + private Optional<SamplerConfiguration> samplerConfig = Optional.empty(); + private boolean samplingEnabled = false; + protected Optional<ClientServiceEnvironmentImpl> env = Optional.empty(); + + public Builder withScope(IteratorScope scope) { + this.scope = Optional.of(scope); + return this; + } + + public Builder isFullMajorCompaction() { + this.isFullMajorCompaction = true; + return this; + } + + public Builder withAuthorizations(Authorizations auths) { + this.auths = Optional.of(auths); + return this; + } + + public Builder isUserCompaction() { + this.isUserCompaction = true; + return this; + } + + public Builder withTableId(TableId tableId) { + this.tableId = Optional.of(tableId); + return this; + } + + public Builder withSamplingEnabled() { + this.samplingEnabled = true; + return this; + } + + public Builder withSamplerConfiguration(SamplerConfiguration sc) { + this.samplerConfig = Optional.ofNullable(sc); + return this; + } + + public Builder withClient(AccumuloClient client) { + this.env = Optional.of(new ClientServiceEnvironmentImpl((ClientContext) client)); + return this; + } + + public ClientIteratorEnvironment build() { + return new ClientIteratorEnvironment(this); + } + + } + + public static final IteratorEnvironment DEFAULT = new Builder().build(); + + private final Optional<IteratorScope> scope; + private final boolean isFullMajorCompaction; + private final Optional<Authorizations> auths; + private final boolean isUserCompaction; + private final Optional<TableId> tableId; + private final Optional<SamplerConfiguration> samplerConfig; + private final boolean samplingEnabled; + private final Optional<ClientServiceEnvironmentImpl> env; + + private ClientIteratorEnvironment(Builder builder) { + this.scope = builder.scope; + this.isFullMajorCompaction = builder.isFullMajorCompaction; + this.auths = builder.auths; + this.isUserCompaction = builder.isUserCompaction; + this.tableId = builder.tableId; + this.samplerConfig = builder.samplerConfig; + this.env = builder.env; + this.samplingEnabled = builder.samplingEnabled; + } + + /** + * Copy constructor used for enabling sample. Only called from {@link cloneWithSamplingEnabled}. + */ + private ClientIteratorEnvironment(ClientIteratorEnvironment copy) { + this.scope = copy.scope; + this.isFullMajorCompaction = copy.isFullMajorCompaction; + this.auths = copy.auths; + this.isUserCompaction = copy.isUserCompaction; + this.tableId = copy.tableId; + this.samplerConfig = copy.samplerConfig; + this.env = copy.env; + this.samplingEnabled = true; + } + + @Override + @Deprecated(since = "2.0.0") + public SortedKeyValueIterator<Key,Value> reserveMapFileReader(String mapFileName) + throws IOException { + Configuration hadoopConf = new Configuration(); + FileSystem fs = FileSystem.get(hadoopConf); + return new MapFileIterator(fs, mapFileName, hadoopConf); + } + + @Override + public IteratorScope getIteratorScope() { + return scope.orElseThrow(); + } + + @Override + public boolean isFullMajorCompaction() { + if (getIteratorScope() != IteratorScope.majc) { + throw new IllegalStateException("Iterator scope is not majc"); + } + return isFullMajorCompaction; + } + + @Override + @Deprecated(since = "2.0.0") + public void registerSideChannel(SortedKeyValueIterator<Key,Value> iter) { + throw new UnsupportedOperationException("Feature not supported"); + } + + @Override + public Authorizations getAuthorizations() { + if (getIteratorScope() != IteratorScope.scan) { + throw new IllegalStateException("Iterator scope is not scan"); + } + return auths.orElseThrow(); + } + + @Override + public IteratorEnvironment cloneWithSamplingEnabled() { + String samplerClass = + getPluginEnv().getConfiguration(getTableId()).get(Property.TABLE_SAMPLER.getKey()); + if (samplerClass == null || samplerClass.isBlank()) { + throw new SampleNotPresentException(); + } Review Comment: The following seems good for ClientSideIteratorScanner and is tested in `SampleIT.testIterator()`. ```suggestion if (samplerConfig.isEmpty()) { throw new SampleNotPresentException(); } ``` Inspecting table props seems like the right way to go, however the bit of code here is not complete for that. Looking into RFileScanner it may benefit from a secondary check of tables props. AFIACT there are no tests for calling `cloneWithSamplingEnabled()` when using RFileScanner. In a follow on PR I could add unit test that cover calling this code when using RFileScanner and maybe add some code that inspects the table config as a secondary step when samplerConfig.isEmpty(). ########## core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java: ########## @@ -74,8 +79,65 @@ class RFileScanner extends ScannerOptions implements Scanner { + private static class RFileScannerIteratorEnvironmentBuilder + extends ClientIteratorEnvironment.Builder { + + public ClientIteratorEnvironment.Builder withEnvironment(ClientServiceEnvironmentImpl env) { + this.env = Optional.of(env); + return this; + } + + } + + private static class RFileScannerEnvironmentImpl extends ClientServiceEnvironmentImpl { + + private final Opts opts; + + public RFileScannerEnvironmentImpl(Opts opts) { + super(null); + this.opts = opts; + } + + @Override + public String getTableName(TableId tableId) throws TableNotFoundException { + Preconditions.checkArgument(tableId == TABLE_ID, "Expected " + TABLE_ID + " obtained" + + " from IteratorEnvironment.getTableId(), but got: " + tableId); + return TABLE_NAME; + } + + @Override + public <T> T instantiate(String className, Class<T> base) + throws ReflectiveOperationException, IOException { + return RFileScanner.class.getClassLoader().loadClass(className).asSubclass(base) + .getDeclaredConstructor().newInstance(); + } + + @Override + public <T> T instantiate(TableId tableId, String className, Class<T> base) + throws ReflectiveOperationException, IOException { + return instantiate(className, base); + } Review Comment: These seem similar enough to the methods in the super class that they could be dropped here. ```suggestion ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: notifications-unsubscr...@accumulo.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org