Skip to content

Commit 40c5f8e

Browse files
committed
Merge branch '3.1'
2 parents 8a9c107 + 187733f commit 40c5f8e

File tree

13 files changed

+26
-32
lines changed

13 files changed

+26
-32
lines changed

core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java

-1
Original file line numberDiff line numberDiff line change
@@ -105,7 +105,6 @@
105105
import org.apache.accumulo.core.client.admin.NewTableConfiguration;
106106
import org.apache.accumulo.core.client.admin.SummaryRetriever;
107107
import org.apache.accumulo.core.client.admin.TableOperations;
108-
import org.apache.accumulo.core.client.admin.TableOperations.ImportDestinationArguments;
109108
import org.apache.accumulo.core.client.admin.TabletAvailability;
110109
import org.apache.accumulo.core.client.admin.TabletInformation;
111110
import org.apache.accumulo.core.client.admin.TabletMergeability;

core/src/main/java/org/apache/accumulo/core/data/LoadPlan.java

+2
Original file line numberDiff line numberDiff line change
@@ -249,6 +249,7 @@ private static class JsonDestination {
249249
String endRow;
250250
RangeType rangeType;
251251

252+
@SuppressWarnings("unused")
252253
JsonDestination() {}
253254

254255
JsonDestination(Destination destination) {
@@ -270,6 +271,7 @@ Destination toDestination() {
270271
private static final class JsonAll {
271272
List<JsonDestination> destinations;
272273

274+
@SuppressWarnings("unused")
273275
JsonAll() {}
274276

275277
JsonAll(List<Destination> destinations) {

core/src/main/java/org/apache/accumulo/core/fate/user/FateMutator.java

+6-5
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,6 @@
2323
import org.apache.accumulo.core.fate.FateStore;
2424
import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus;
2525
import org.apache.accumulo.core.fate.Repo;
26-
import org.apache.accumulo.core.fate.user.schema.FateSchema;
2726

2827
public interface FateMutator<T> {
2928

@@ -57,17 +56,19 @@ public interface FateMutator<T> {
5756
FateMutator<T> requireAbsentKey();
5857

5958
/**
60-
* Add a conditional mutation to {@link FateSchema.TxColumnFamily#RESERVATION_COLUMN} that will
61-
* put the reservation if there is not already a reservation present
59+
* Add a conditional mutation to
60+
* {@link org.apache.accumulo.core.fate.user.schema.FateSchema.TxColumnFamily#RESERVATION_COLUMN}
61+
* that will put the reservation if there is not already a reservation present
6262
*
6363
* @param reservation the reservation to attempt to put
6464
* @return the FateMutator with this added mutation
6565
*/
6666
FateMutator<T> putReservedTx(FateStore.FateReservation reservation);
6767

6868
/**
69-
* Add a conditional mutation to {@link FateSchema.TxColumnFamily#RESERVATION_COLUMN} that will
70-
* delete the column if the column value matches the given reservation
69+
* Add a conditional mutation to
70+
* {@link org.apache.accumulo.core.fate.user.schema.FateSchema.TxColumnFamily#RESERVATION_COLUMN}
71+
* that will delete the column if the column value matches the given reservation
7172
*
7273
* @param reservation the reservation to attempt to remove
7374
* @return the FateMutator with this added mutation

core/src/main/java/org/apache/accumulo/core/fate/user/UserFateStore.java

+3-2
Original file line numberDiff line numberDiff line change
@@ -79,8 +79,9 @@ public class UserFateStore<T> extends AbstractFateStore<T> {
7979
*
8080
* @param context the {@link ClientContext}
8181
* @param tableName the name of the table which will store the Fate data
82-
* @param lockID the {@link ZooUtil.LockID} held by the process creating this store. Should be
83-
* null if this store will be used as read-only (will not be used to reserve transactions)
82+
* @param lockID the {@link org.apache.accumulo.core.fate.zookeeper.ZooUtil.LockID} held by the
83+
* process creating this store. Should be null if this store will be used as read-only
84+
* (will not be used to reserve transactions)
8485
* @param isLockHeld the {@link Predicate} used to determine if the lockID is held or not at the
8586
* time of invocation. If the store is used for a {@link Fate} which runs a dead
8687
* reservation cleaner, this should be non-null, otherwise null is fine

core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java

+5-4
Original file line numberDiff line numberDiff line change
@@ -259,10 +259,11 @@ public interface TabletsMutator extends AutoCloseable {
259259
interface ConditionalResult {
260260

261261
/**
262-
* This enum was created instead of using {@link ConditionalWriter.Status} because Ample has
263-
* automated handling for most of the statuses of the conditional writer and therefore only a
264-
* subset are expected to be passed out of Ample. This enum represents the subset that Ample
265-
* will actually return.
262+
* This enum was created instead of using
263+
* {@link org.apache.accumulo.core.client.ConditionalWriter.Status} because Ample has automated
264+
* handling for most of the statuses of the conditional writer and therefore only a subset are
265+
* expected to be passed out of Ample. This enum represents the subset that Ample will actually
266+
* return.
266267
*/
267268
enum Status {
268269
ACCEPTED, REJECTED

core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionPlanImpl.java

+1-4
Original file line numberDiff line numberDiff line change
@@ -55,14 +55,11 @@ public static class BuilderImpl implements CompactionPlan.Builder {
5555

5656
private final CompactionKind kind;
5757
private final ArrayList<CompactionJob> jobs = new ArrayList<>();
58-
private final Set<CompactableFile> allFiles;
5958
private final Set<CompactableFile> seenFiles = new HashSet<>();
6059
private final Set<CompactableFile> candidates;
6160

62-
public BuilderImpl(CompactionKind kind, Set<CompactableFile> allFiles,
63-
Set<CompactableFile> candidates) {
61+
public BuilderImpl(CompactionKind kind, Set<CompactableFile> candidates) {
6462
this.kind = kind;
65-
this.allFiles = allFiles;
6663
this.candidates = candidates;
6764
}
6865

core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java

+1-10
Original file line numberDiff line numberDiff line change
@@ -266,21 +266,12 @@ private abstract class ZooRunnable<T> {
266266
* Runs an operation against ZooKeeper. Retries are performed by the retry method when
267267
* KeeperExceptions occur.
268268
*
269-
* Changes were made in ACCUMULO-4388 so that the run method no longer accepts Zookeeper as an
270-
* argument, and instead relies on the ZooRunnable implementation to call
271-
* {@link #getZooKeeper()}. Performing the call to retrieving a ZooKeeper Session after caches
272-
* are checked has the benefit of limiting ZK connections and blocking as a result of obtaining
273-
* these sessions.
274-
*
275269
* @return T the result of the runnable
276270
*/
277271
abstract T run() throws KeeperException, InterruptedException;
278272

279273
/**
280-
* Retry will attempt to call the run method. Run should make a call to {@link #getZooKeeper()}
281-
* after checks to cached information are made. This change, per ACCUMULO-4388 ensures that we
282-
* don't create a ZooKeeper session when information is cached, and access to ZooKeeper is
283-
* unnecessary.
274+
* Retry will attempt to call the run method.
284275
*
285276
* @return result of the runnable access success ( i.e. no exceptions ).
286277
*/

core/src/test/java/org/apache/accumulo/core/spi/compaction/RatioBasedCompactionPlannerTest.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -667,7 +667,7 @@ public void testMaxTableFilesFallback() {
667667

668668
private CompactionJob createJob(CompactionKind kind, Set<CompactableFile> all,
669669
Set<CompactableFile> files) {
670-
return new CompactionPlanImpl.BuilderImpl(kind, all, all)
670+
return new CompactionPlanImpl.BuilderImpl(kind, all)
671671
.addJob((short) all.size(), CompactorGroupId.of("small"), files).build().getJobs()
672672
.iterator().next();
673673
}
@@ -817,7 +817,7 @@ public Collection<CompactableFile> getAll() {
817817

818818
@Override
819819
public Builder createPlanBuilder() {
820-
return new CompactionPlanImpl.BuilderImpl(kind, all, candidates);
820+
return new CompactionPlanImpl.BuilderImpl(kind, candidates);
821821
}
822822
};
823823
}

server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionJobGenerator.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -296,7 +296,7 @@ public Map<String,String> getExecutionHints() {
296296

297297
@Override
298298
public CompactionPlan.Builder createPlanBuilder() {
299-
return new CompactionPlanImpl.BuilderImpl(kind, allFiles, candidates);
299+
return new CompactionPlanImpl.BuilderImpl(kind, candidates);
300300
}
301301
};
302302
return planCompactions(planner, params, serviceId);

server/base/src/main/java/org/apache/accumulo/server/util/ServiceStatusCmd.java

-1
Original file line numberDiff line numberDiff line change
@@ -237,7 +237,6 @@ Result<Set<String>> readAllNodesData(final ZooReader zooReader, final String pat
237237
* Provides explicit method names instead of generic getFirst to get the error count and getSecond
238238
* hosts information
239239
*
240-
* @param <A> errorCount
241240
* @param <B> hosts
242241
*/
243242
private static class Result<B> extends Pair<Integer,B> {

server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/CheckRunner.java

+2-1
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,8 @@ public interface CheckRunner {
3434
* @param opts server util opts. Only applicable for the checks on the root and metadata tables
3535
* @param fixFiles remove dangling file pointers. Only applicable for the checks on the system and
3636
* user files
37-
* @return the {@link Admin.CheckCommand.CheckStatus} resulting from running the check
37+
* @return the {@link org.apache.accumulo.server.util.Admin.CheckCommand.CheckStatus} resulting
38+
* from running the check
3839
*/
3940
Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtilOpts opts,
4041
boolean fixFiles) throws Exception;

test/src/main/java/org/apache/accumulo/test/functional/GracefulShutdownIT.java

+1
Original file line numberDiff line numberDiff line change
@@ -270,6 +270,7 @@ public void testGracefulShutdown() throws Exception {
270270
scanner.setBatchSize(1);
271271
int count = 0;
272272
for (Entry<Key,Value> e : scanner) {
273+
assertNotNull(e);
273274
count++;
274275
if (count == 2) {
275276
Admin.signalGracefulShutdown(ctx, sserver.toString());

test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedScanIT.java

+2-1
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
import static org.apache.accumulo.core.metrics.Metric.SCAN_RETURN_FOR_MEM;
2626
import static org.apache.accumulo.test.util.Wait.waitFor;
2727
import static org.junit.jupiter.api.Assertions.assertEquals;
28+
import static org.junit.jupiter.api.Assertions.assertNotEquals;
2829
import static org.junit.jupiter.api.Assertions.assertTrue;
2930

3031
import java.util.Collections;
@@ -187,7 +188,7 @@ static void freeServerMemory(AccumuloClient client) throws Exception {
187188
try (Scanner scanner = client.createScanner(AccumuloTable.METADATA.tableName())) {
188189
IteratorSetting is = new IteratorSetting(11, MemoryFreeingIterator.class, Map.of());
189190
scanner.addScanIterator(is);
190-
var unused = Iterables.size(scanner); // consume the key/values
191+
assertNotEquals(0, Iterables.size(scanner)); // consume the key/values
191192
}
192193
}
193194

0 commit comments

Comments
 (0)