Skip to content

Removed TabletIteratorEnvironment, replaced with smaller impl #5587

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 7 commits into from
Jun 5, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -46,10 +46,10 @@ public static class Builder {
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();
protected Optional<TableId> tableId = Optional.empty();
protected Optional<SamplerConfiguration> samplerConfig = Optional.empty();
private boolean samplingEnabled = false;
protected Optional<ClientServiceEnvironmentImpl> env = Optional.empty();
protected Optional<ServiceEnvironment> env = Optional.empty();

public Builder withScope(IteratorScope scope) {
checkState(this.scope.isEmpty(), "Scope has already been set");
Expand Down Expand Up @@ -115,11 +115,11 @@ public ClientIteratorEnvironment build() {
private final Optional<Authorizations> auths;
private final boolean isUserCompaction;
private final Optional<TableId> tableId;
private final Optional<SamplerConfiguration> samplerConfig;
protected Optional<SamplerConfiguration> samplerConfig;
private final boolean samplingEnabled;
private final Optional<ClientServiceEnvironmentImpl> env;
private final Optional<ServiceEnvironment> env;

private ClientIteratorEnvironment(Builder builder) {
protected ClientIteratorEnvironment(Builder builder) {
this.scope = builder.scope;
this.isFullMajorCompaction = builder.isFullMajorCompaction;
this.auths = builder.auths;
Expand All @@ -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;
Expand Down Expand Up @@ -173,7 +173,7 @@ public void registerSideChannel(SortedKeyValueIterator<Key,Value> 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();
}
Expand Down Expand Up @@ -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;
}

Expand Down
Original file line number Diff line number Diff line change
@@ -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<SortedKeyValueIterator<Key,Value>> topLevelIterators = new ArrayList<>();

public Builder(ServerContext ctx) {
this.ctx = ctx;
this.env = Optional.of(new ServiceEnvironmentImpl(ctx));
}

public Builder
withTopLevelIterators(ArrayList<SortedKeyValueIterator<Key,Value>> 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<SortedKeyValueIterator<Key,Value>> 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<Key,Value>
getTopLevelIterator(SortedKeyValueIterator<Key,Value> iter) {
if (topLevelIterators.isEmpty()) {
return iter;
}
ArrayList<SortedKeyValueIterator<Key,Value>> allIters = new ArrayList<>(topLevelIterators);
allIters.add(iter);
return new MultiIterator(allIters, false);
}

}
Loading